diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregates.scala index bf17791fdd0a0..3812dea20943b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregates.scala @@ -18,9 +18,9 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.analysis.PullOutNondeterministic -import org.apache.spark.sql.catalyst.expressions.{AliasHelper, AttributeSet} +import org.apache.spark.sql.catalyst.expressions.{Alias, AliasHelper, AttributeSet, ExpressionSet} import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression -import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern.AGGREGATE @@ -47,6 +47,17 @@ object RemoveRedundantAggregates extends Rule[LogicalPlan] with AliasHelper { } else { newAggregate } + + case agg @ Aggregate(groupingExps, _, child) + if agg.groupOnly && child.deterministic && + child.distinctKeys.exists(_.subsetOf(ExpressionSet(groupingExps))) => + Project(agg.aggregateExpressions, child) + + case agg @ Aggregate(groupingExps, aggregateExps, child) + if aggregateExps.forall(a => a.isInstanceOf[Alias] && a.children.forall(_.foldable)) && + child.deterministic && + child.distinctKeys.exists(_.subsetOf(ExpressionSet(groupingExps))) => + Project(agg.aggregateExpressions, child) } private def isLowerRedundant(upper: Aggregate, lower: Aggregate): Boolean = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DistinctKeyVisitor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DistinctKeyVisitor.scala new file mode 100644 index 0000000000000..f6b012a689955 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DistinctKeyVisitor.scala @@ -0,0 +1,129 @@ +/* + * 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.plans.logical + +import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, ExpressionSet, NamedExpression} +import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys +import org.apache.spark.sql.catalyst.plans.{Inner, LeftSemiOrAnti} + +/** + * A visitor pattern for traversing a [[LogicalPlan]] tree and propagate the distinct attributes. + */ +object DistinctKeyVisitor extends LogicalPlanVisitor[Set[ExpressionSet]] { + + private def projectDistinctKeys( + keys: Set[ExpressionSet], projectList: Seq[NamedExpression]): Set[ExpressionSet] = { + val outputSet = ExpressionSet(projectList.map(_.toAttribute)) + val distinctKeys = keys.filter(_.subsetOf(outputSet)) + val aliases = projectList.filter(_.isInstanceOf[Alias]) + if (aliases.isEmpty) { + distinctKeys + } else { + val aliasedDistinctKeys = keys.map { expressionSet => + expressionSet.map { expression => + expression transform { + case expr: Expression => + aliases + .collectFirst { case a: Alias if a.child.semanticEquals(expr) => a.toAttribute } + .getOrElse(expr) + } + } + } + aliasedDistinctKeys.collect { + case es: ExpressionSet if es.subsetOf(outputSet) => ExpressionSet(es) + } ++ distinctKeys + }.filter(_.nonEmpty) + } + + override def default(p: LogicalPlan): Set[ExpressionSet] = Set.empty[ExpressionSet] + + override def visitAggregate(p: Aggregate): Set[ExpressionSet] = { + val groupingExps = ExpressionSet(p.groupingExpressions) // handle group by a, a + projectDistinctKeys(Set(groupingExps), p.aggregateExpressions) + } + + override def visitDistinct(p: Distinct): Set[ExpressionSet] = Set(ExpressionSet(p.output)) + + override def visitExcept(p: Except): Set[ExpressionSet] = + if (!p.isAll && p.deterministic) Set(ExpressionSet(p.output)) else default(p) + + override def visitExpand(p: Expand): Set[ExpressionSet] = default(p) + + override def visitFilter(p: Filter): Set[ExpressionSet] = p.child.distinctKeys + + override def visitGenerate(p: Generate): Set[ExpressionSet] = default(p) + + override def visitGlobalLimit(p: GlobalLimit): Set[ExpressionSet] = { + p.maxRows match { + case Some(value) if value <= 1 => Set(ExpressionSet(p.output)) + case _ => p.child.distinctKeys + } + } + + override def visitIntersect(p: Intersect): Set[ExpressionSet] = { + if (!p.isAll && p.deterministic) Set(ExpressionSet(p.output)) else default(p) + } + + override def visitJoin(p: Join): Set[ExpressionSet] = { + p match { + case Join(_, _, LeftSemiOrAnti(_), _, _) => + p.left.distinctKeys + case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, _, _, _, _, _) + if p.left.distinctKeys.exists(_.subsetOf(ExpressionSet(leftKeys))) && + p.right.distinctKeys.exists(_.subsetOf(ExpressionSet(rightKeys))) => + Set(ExpressionSet(leftKeys), ExpressionSet(rightKeys)) + case _ => default(p) + } + } + + override def visitLocalLimit(p: LocalLimit): Set[ExpressionSet] = p.child.distinctKeys + + override def visitPivot(p: Pivot): Set[ExpressionSet] = default(p) + + override def visitProject(p: Project): Set[ExpressionSet] = { + if (p.child.distinctKeys.nonEmpty) { + projectDistinctKeys(p.child.distinctKeys.map(ExpressionSet(_)), p.projectList) + } else { + default(p) + } + } + + override def visitRepartition(p: Repartition): Set[ExpressionSet] = p.child.distinctKeys + + override def visitRepartitionByExpr(p: RepartitionByExpression): Set[ExpressionSet] = + p.child.distinctKeys + + override def visitSample(p: Sample): Set[ExpressionSet] = { + if (!p.withReplacement) p.child.distinctKeys else default(p) + } + + override def visitScriptTransform(p: ScriptTransformation): Set[ExpressionSet] = default(p) + + override def visitUnion(p: Union): Set[ExpressionSet] = default(p) + + override def visitWindow(p: Window): Set[ExpressionSet] = p.child.distinctKeys + + override def visitTail(p: Tail): Set[ExpressionSet] = p.child.distinctKeys + + override def visitSort(p: Sort): Set[ExpressionSet] = p.child.distinctKeys + + override def visitRebalancePartitions(p: RebalancePartitions): Set[ExpressionSet] = + p.child.distinctKeys + + override def visitWithCTE(p: WithCTE): Set[ExpressionSet] = p.plan.distinctKeys +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 5ae2a7da826a4..bce8bd35986bd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -31,6 +31,7 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with AnalysisHelper with LogicalPlanStats + with LogicalPlanDistinctKeys with QueryPlanConstraints with Logging { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlanDistinctKeys.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlanDistinctKeys.scala new file mode 100644 index 0000000000000..1843c2da478ef --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlanDistinctKeys.scala @@ -0,0 +1,34 @@ +/* + * 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.plans.logical + +import org.apache.spark.sql.catalyst.expressions.ExpressionSet +import org.apache.spark.sql.internal.SQLConf.PROPAGATE_DISTINCT_KEYS_ENABLED + +/** + * A trait to add distinct attributes to [[LogicalPlan]]. For example: + * {{{ + * SELECT a, b, SUM(c) FROM Tab1 GROUP BY a, b + * // returns a, b + * }}} + */ +trait LogicalPlanDistinctKeys { self: LogicalPlan => + lazy val distinctKeys: Set[ExpressionSet] = { + if (conf.getConf(PROPAGATE_DISTINCT_KEYS_ENABLED)) DistinctKeyVisitor.visit(self) else Set.empty + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 3a7ce650ea633..24edba82f4a8b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -732,6 +732,15 @@ object SQLConf { .booleanConf .createWithDefault(true) + val PROPAGATE_DISTINCT_KEYS_ENABLED = + buildConf("spark.sql.optimizer.propagateDistinctKeys.enabled") + .internal() + .doc("When true, the query optimizer will propagate a set of distinct attributes from the " + + "current node and use it to optimize query.") + .version("3.3.0") + .booleanConf + .createWithDefault(true) + val ESCAPED_STRING_LITERALS = buildConf("spark.sql.parser.escapedStringLiterals") .internal() .doc("When true, string literals (including regex patterns) remain escaped in our SQL " + diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala index d11ff16229b14..f997cdcf05cb8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala @@ -21,8 +21,9 @@ import org.apache.spark.api.python.PythonEvalType import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.expressions.{Expression, PythonUDF} -import org.apache.spark.sql.catalyst.plans.PlanTest -import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} +import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral +import org.apache.spark.sql.catalyst.plans.{LeftAnti, LeftSemi, PlanTest} +import org.apache.spark.sql.catalyst.plans.logical.{Distinct, LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.types.IntegerType @@ -33,6 +34,10 @@ class RemoveRedundantAggregatesSuite extends PlanTest { RemoveRedundantAggregates) :: Nil } + private val relation = LocalRelation('a.int, 'b.int) + private val x = relation.subquery('x) + private val y = relation.subquery('y) + private def aggregates(e: Expression): Seq[Expression] = { Seq( count(e), @@ -42,7 +47,6 @@ class RemoveRedundantAggregatesSuite extends PlanTest { } test("Remove redundant aggregate") { - val relation = LocalRelation('a.int, 'b.int) for (agg <- aggregates('b)) { val query = relation .groupBy('a)('a, agg) @@ -57,7 +61,6 @@ class RemoveRedundantAggregatesSuite extends PlanTest { } test("Remove 2 redundant aggregates") { - val relation = LocalRelation('a.int, 'b.int) for (agg <- aggregates('b)) { val query = relation .groupBy('a)('a, agg) @@ -73,7 +76,6 @@ class RemoveRedundantAggregatesSuite extends PlanTest { } test("Remove redundant aggregate with different grouping") { - val relation = LocalRelation('a.int, 'b.int) val query = relation .groupBy('a, 'b)('a) .groupBy('a)('a) @@ -86,7 +88,6 @@ class RemoveRedundantAggregatesSuite extends PlanTest { } test("Remove redundant aggregate with aliases") { - val relation = LocalRelation('a.int, 'b.int) for (agg <- aggregates('b)) { val query = relation .groupBy('a + 'b)(('a + 'b) as 'c, agg) @@ -101,7 +102,6 @@ class RemoveRedundantAggregatesSuite extends PlanTest { } test("Remove redundant aggregate with non-deterministic upper") { - val relation = LocalRelation('a.int, 'b.int) val query = relation .groupBy('a)('a) .groupBy('a)('a, rand(0) as 'c) @@ -114,7 +114,6 @@ class RemoveRedundantAggregatesSuite extends PlanTest { } test("Remove redundant aggregate with non-deterministic lower") { - val relation = LocalRelation('a.int, 'b.int) val query = relation .groupBy('a, 'c)('a, rand(0) as 'c) .groupBy('a, 'c)('a, 'c) @@ -127,7 +126,6 @@ class RemoveRedundantAggregatesSuite extends PlanTest { } test("Keep non-redundant aggregate - upper has duplicate sensitive agg expression") { - val relation = LocalRelation('a.int, 'b.int) for (agg <- aggregates('b)) { val query = relation .groupBy('a, 'b)('a, 'b) @@ -140,7 +138,6 @@ class RemoveRedundantAggregatesSuite extends PlanTest { } test("Remove redundant aggregate - upper has duplicate agnostic agg expression") { - val relation = LocalRelation('a.int, 'b.int) val query = relation .groupBy('a, 'b)('a, 'b) // The max and countDistinct does not change if there are duplicate values @@ -154,7 +151,6 @@ class RemoveRedundantAggregatesSuite extends PlanTest { } test("Keep non-redundant aggregate - upper references agg expression") { - val relation = LocalRelation('a.int, 'b.int) for (agg <- aggregates('b)) { val query = relation .groupBy('a)('a, agg as 'c) @@ -166,7 +162,6 @@ class RemoveRedundantAggregatesSuite extends PlanTest { } test("Keep non-redundant aggregate - upper references non-deterministic non-grouping") { - val relation = LocalRelation('a.int, 'b.int) val query = relation .groupBy('a)('a, ('a + rand(0)) as 'c) .groupBy('a, 'c)('a, 'c) @@ -174,4 +169,120 @@ class RemoveRedundantAggregatesSuite extends PlanTest { val optimized = Optimize.execute(query) comparePlans(optimized, query) } + + test("SPARK-36194: Remove aggregation from left semi/anti join if aggregation the same") { + Seq(LeftSemi, LeftAnti).foreach { joinType => + val originalQuery = x.groupBy('a, 'b)('a, 'b) + .join(y, joinType, Some("x.a".attr === "y.a".attr && "x.b".attr === "y.b".attr)) + .groupBy("x.a".attr, "x.b".attr)("x.a".attr, "x.b".attr) + val correctAnswer = x.groupBy('a, 'b)('a, 'b) + .join(y, joinType, Some("x.a".attr === "y.a".attr && "x.b".attr === "y.b".attr)) + .select("x.a".attr, "x.b".attr) + + val optimized = Optimize.execute(originalQuery.analyze) + comparePlans(optimized, correctAnswer.analyze) + } + } + + test("SPARK-36194: Remove aggregation from left semi/anti join with alias") { + Seq(LeftSemi, LeftAnti).foreach { joinType => + val originalQuery = x.groupBy('a, 'b)('a, 'b.as("d")) + .join(y, joinType, Some("x.a".attr === "y.a".attr && "d".attr === "y.b".attr)) + .groupBy("x.a".attr, "d".attr)("x.a".attr, "d".attr) + val correctAnswer = x.groupBy('a, 'b)('a, 'b.as("d")) + .join(y, joinType, Some("x.a".attr === "y.a".attr && "d".attr === "y.b".attr)) + .select("x.a".attr, "d".attr) + + val optimized = Optimize.execute(originalQuery.analyze) + comparePlans(optimized, correctAnswer.analyze) + } + } + + test("SPARK-36194: Remove aggregation from left semi/anti join if it is the sub aggregateExprs") { + Seq(LeftSemi, LeftAnti).foreach { joinType => + val originalQuery = x.groupBy('a, 'b)('a, 'b) + .join(y, joinType, Some("x.a".attr === "y.a".attr && "x.b".attr === "y.b".attr)) + .groupBy("x.a".attr, "x.b".attr)("x.a".attr) + val correctAnswer = x.groupBy('a, 'b)('a, 'b) + .join(y, joinType, Some("x.a".attr === "y.a".attr && "x.b".attr === "y.b".attr)) + .select("x.a".attr) + + val optimized = Optimize.execute(originalQuery.analyze) + comparePlans(optimized, correctAnswer.analyze) + } + } + + test("SPARK-36194: Transform down to remove more aggregates") { + Seq(LeftSemi, LeftAnti).foreach { joinType => + val originalQuery = x.groupBy('a, 'b)('a, 'b) + .join(y, joinType, Some("x.a".attr === "y.a".attr && "x.b".attr === "y.b".attr)) + .groupBy("x.a".attr, "x.b".attr)("x.a".attr, "x.b".attr) + .join(y, joinType, Some("x.a".attr === "y.a".attr && "x.b".attr === "y.b".attr)) + .groupBy("x.a".attr, "x.b".attr)("x.a".attr) + val correctAnswer = x.groupBy('a, 'b)('a, 'b) + .join(y, joinType, Some("x.a".attr === "y.a".attr && "x.b".attr === "y.b".attr)) + .select("x.a".attr, "x.b".attr) + .join(y, joinType, Some("x.a".attr === "y.a".attr && "x.b".attr === "y.b".attr)) + .select("x.a".attr) + + val optimized = Optimize.execute(originalQuery.analyze) + comparePlans(optimized, correctAnswer.analyze) + } + } + + test("SPARK-36194: Child distinct keys is the subset of required keys") { + val originalQuery = relation + .groupBy('a)('a, count('b).as("cnt")) + .groupBy('a, 'cnt)('a, 'cnt) + .analyze + val correctAnswer = relation + .groupBy('a)('a, count('b).as("cnt")) + .select('a, 'cnt) + .analyze + val optimized = Optimize.execute(originalQuery) + comparePlans(optimized, correctAnswer) + } + + test("SPARK-36194: Child distinct keys are subsets and aggregateExpressions are foldable") { + val originalQuery = x.groupBy('a, 'b)('a, 'b) + .join(y, LeftSemi, Some("x.a".attr === "y.a".attr && "x.b".attr === "y.b".attr)) + .groupBy("x.a".attr, "x.b".attr)(TrueLiteral) + .analyze + val correctAnswer = x.groupBy('a, 'b)('a, 'b) + .join(y, LeftSemi, Some("x.a".attr === "y.a".attr && "x.b".attr === "y.b".attr)) + .select(TrueLiteral) + .analyze + val optimized = Optimize.execute(originalQuery) + comparePlans(optimized, correctAnswer) + } + + test("SPARK-36194: Negative case: child distinct keys is not the subset of required keys") { + Seq(LeftSemi, LeftAnti).foreach { joinType => + val originalQuery1 = x.groupBy('a, 'b)('a, 'b) + .join(y, joinType, Some("x.a".attr === "y.a".attr && "x.b".attr === "y.b".attr)) + .groupBy("x.a".attr)("x.a".attr) + .analyze + comparePlans(Optimize.execute(originalQuery1), originalQuery1) + + val originalQuery2 = x.groupBy('a, 'b)('a, 'b) + .join(y, joinType, Some("x.a".attr === "y.a".attr && "x.b".attr === "y.b".attr)) + .groupBy("x.a".attr)(count("x.b".attr)) + .analyze + comparePlans(Optimize.execute(originalQuery2), originalQuery2) + } + } + + test("SPARK-36194: Negative case: child distinct keys is empty") { + val originalQuery = Distinct(x.groupBy('a, 'b)('a, TrueLiteral)).analyze + comparePlans(Optimize.execute(originalQuery), originalQuery) + } + + test("SPARK-36194: Negative case: Remove aggregation from contains non-deterministic") { + val query = relation + .groupBy('a)('a, (count('b) + rand(0)).as("cnt")) + .groupBy('a, 'cnt)('a, 'cnt) + .analyze + val optimized = Optimize.execute(query) + comparePlans(optimized, query) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/logical/DistinctKeyVisitorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/logical/DistinctKeyVisitorSuite.scala new file mode 100644 index 0000000000000..0526331b20091 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/logical/DistinctKeyVisitorSuite.scala @@ -0,0 +1,172 @@ +/* + * 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.plans.logical + +import scala.collection.mutable +import scala.reflect.runtime.universe.TypeTag + +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, ExpressionSet, UnspecifiedFrame} +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.types.IntegerType + +class DistinctKeyVisitorSuite extends PlanTest { + + private val a = AttributeReference("a", IntegerType)() + private val b = AttributeReference("b", IntegerType)() + private val c = AttributeReference("c", IntegerType)() + private val d = a.as("aliased_a") + private val e = b.as("aliased_b") + private val f = Alias(a + 1, (a + 1).toString)() + private val x = AttributeReference("x", IntegerType)() + private val y = AttributeReference("y", IntegerType)() + private val z = AttributeReference("z", IntegerType)() + + + private val t1 = LocalRelation(a, b, c).as("t1") + private val t2 = LocalRelation(x, y, z).as("t2") + + private def checkDistinctAttributes(plan: LogicalPlan, distinctKeys: Set[ExpressionSet]) = { + assert(plan.analyze.distinctKeys === distinctKeys) + } + + implicit private def productEncoder[T <: Product : TypeTag] = ExpressionEncoder[T]() + + test("Aggregate's distinct attributes") { + checkDistinctAttributes(t1.groupBy('a, 'b)('a, 'b, 1), Set(ExpressionSet(Seq(a, b)))) + checkDistinctAttributes(t1.groupBy('a)('a), Set(ExpressionSet(Seq(a)))) + checkDistinctAttributes(t1.groupBy('a, 'b)('a, 'b), Set(ExpressionSet(Seq(a, b)))) + checkDistinctAttributes(t1.groupBy('a, 'b, 1)('a, 'b), Set(ExpressionSet(Seq(a, b)))) + checkDistinctAttributes(t1.groupBy('a, 'b)('a, 'b, 1), Set(ExpressionSet(Seq(a, b)))) + checkDistinctAttributes(t1.groupBy('a, 'b, 1)('a, 'b, 1), Set(ExpressionSet(Seq(a, b)))) + checkDistinctAttributes(t1.groupBy('a, 'b)('a, 'a), Set.empty) + checkDistinctAttributes(t1.groupBy('a, 'b)('a), Set.empty) + checkDistinctAttributes(t1.groupBy('a)('a, max('b)), Set(ExpressionSet(Seq(a)))) + checkDistinctAttributes(t1.groupBy('a, 'b)('a, 'b, d, e), + Set(ExpressionSet(Seq(a, b)), ExpressionSet(Seq(d.toAttribute, e.toAttribute)))) + checkDistinctAttributes(t1.groupBy()(sum('c)), Set.empty) + checkDistinctAttributes(t1.groupBy('a)('a, 'a % 10, d, sum('b)), + Set(ExpressionSet(Seq(a)), ExpressionSet(Seq(d.toAttribute)))) + checkDistinctAttributes(t1.groupBy(f.child, 'b)(f, 'b, sum('c)), + Set(ExpressionSet(Seq(f.toAttribute, b)))) + } + + test("Distinct's distinct attributes") { + checkDistinctAttributes(Distinct(t1), Set(ExpressionSet(Seq(a, b, c)))) + checkDistinctAttributes(Distinct(t1.select('a, 'c)), Set(ExpressionSet(Seq(a, c)))) + } + + test("Except's distinct attributes") { + checkDistinctAttributes(Except(t1, t2, false), Set(ExpressionSet(Seq(a, b, c)))) + checkDistinctAttributes(Except(t1, t2, true), Set.empty) + } + + test("Filter's distinct attributes") { + checkDistinctAttributes(Filter('a > 1, t1), Set.empty) + checkDistinctAttributes(Filter('a > 1, Distinct(t1)), Set(ExpressionSet(Seq(a, b, c)))) + } + + test("Limit's distinct attributes") { + checkDistinctAttributes(Distinct(t1).limit(10), Set(ExpressionSet(Seq(a, b, c)))) + checkDistinctAttributes(LocalLimit(10, Distinct(t1)), Set(ExpressionSet(Seq(a, b, c)))) + checkDistinctAttributes(t1.limit(1), Set(ExpressionSet(Seq(a, b, c)))) + } + + test("Intersect's distinct attributes") { + checkDistinctAttributes(Intersect(t1, t2, false), Set(ExpressionSet(Seq(a, b, c)))) + checkDistinctAttributes(Intersect(t1, t2, true), Set.empty) + } + + test("Join's distinct attributes") { + Seq(LeftSemi, LeftAnti).foreach { joinType => + checkDistinctAttributes( + Distinct(t1).join(t2, joinType, Some('a === 'x)), Set(ExpressionSet(Seq(a, b, c)))) + } + + checkDistinctAttributes( + Distinct(t1).join(Distinct(t2), Inner, Some('a === 'x && 'b === 'y && 'c === 'z)), + Set(ExpressionSet(Seq(a, b, c)), ExpressionSet(Seq(x, y, z)))) + + checkDistinctAttributes(t1.join(t2, LeftSemi, Some('a === 'x)), + Set.empty) + checkDistinctAttributes( + Distinct(t1).join(Distinct(t2), Inner, Some('a === 'x && 'b === 'y)), + Set.empty) + checkDistinctAttributes( + Distinct(t1).join(Distinct(t2), Inner, Some('a === 'x && 'b === 'y && 'c % 5 === 'z % 5)), + Set.empty) + Seq(LeftOuter, Cross, RightOuter).foreach { joinType => + checkDistinctAttributes( + Distinct(t1).join(Distinct(t2), joinType, Some('a === 'x && 'b === 'y && 'c === 'z)), + Set.empty) + } + } + + test("Project's distinct attributes") { + checkDistinctAttributes(t1.select('a, 'b), Set.empty) + checkDistinctAttributes(Distinct(t1).select('a), Set.empty) + checkDistinctAttributes(Distinct(t1).select('a, 'b, d, e), Set.empty) + checkDistinctAttributes(Distinct(t1).select('a, 'b, 'c, 1), Set(ExpressionSet(Seq(a, b, c)))) + checkDistinctAttributes(Distinct(t1).select('a, 'b, c, d), + Set(ExpressionSet(Seq(a, b, c)), ExpressionSet(Seq(b, c, d.toAttribute)))) + checkDistinctAttributes(t1.groupBy('a, 'b)('a, 'b, d).select('a, 'b, e), + Set(ExpressionSet(Seq(a, b)), ExpressionSet(Seq(a, e.toAttribute)))) + } + + test("Repartition's distinct attributes") { + checkDistinctAttributes(t1.repartition(8), Set.empty) + checkDistinctAttributes(Distinct(t1).repartition(8), Set(ExpressionSet(Seq(a, b, c)))) + checkDistinctAttributes(RepartitionByExpression(Seq(a), Distinct(t1), None), + Set(ExpressionSet(Seq(a, b, c)))) + } + + test("Sample's distinct attributes") { + checkDistinctAttributes(t1.sample(0, 0.2, false, 1), Set.empty) + checkDistinctAttributes(Distinct(t1).sample(0, 0.2, false, 1), Set(ExpressionSet(Seq(a, b, c)))) + } + + test("Window's distinct attributes") { + val winExpr = windowExpr(count('b), windowSpec('a :: Nil, 'b.asc :: Nil, UnspecifiedFrame)) + + checkDistinctAttributes( + Distinct(t1).select('a, 'b, 'c, winExpr.as('window)), Set(ExpressionSet(Seq(a, b, c)))) + checkDistinctAttributes( + Distinct(t1).select('a, 'b, winExpr.as('window)), Set()) + } + + test("Tail's distinct attributes") { + checkDistinctAttributes(Tail(10, Distinct(t1)), Set(ExpressionSet(Seq(a, b, c)))) + } + + test("Sort's distinct attributes") { + checkDistinctAttributes(t1.sortBy('a.asc), Set.empty) + checkDistinctAttributes(Distinct(t1).sortBy('a.asc), Set(ExpressionSet(Seq(a, b, c)))) + } + + test("RebalancePartitions's distinct attributes") { + checkDistinctAttributes(RebalancePartitions(Seq(a), Distinct(t1)), + Set(ExpressionSet(Seq(a, b, c)))) + } + + test("WithCTE's distinct attributes") { + checkDistinctAttributes(WithCTE(Distinct(t1), mutable.ArrayBuffer.empty[CTERelationDef].toSeq), + Set(ExpressionSet(Seq(a, b, c)))) + } +} 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 e3eac82fee26b..4105a94131dda 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 @@ -1,130 +1,127 @@ == Physical Plan == -TakeOrderedAndProject (126) -+- * HashAggregate (125) - +- Exchange (124) - +- * HashAggregate (123) - +- * Expand (122) - +- Union (121) - :- * Project (82) - : +- * Filter (81) - : +- * HashAggregate (80) - : +- Exchange (79) - : +- * HashAggregate (78) - : +- * Project (77) - : +- * BroadcastHashJoin Inner BuildRight (76) - : :- * Project (66) - : : +- * BroadcastHashJoin Inner BuildRight (65) - : : :- * SortMergeJoin LeftSemi (63) +TakeOrderedAndProject (123) ++- * HashAggregate (122) + +- Exchange (121) + +- * HashAggregate (120) + +- * Expand (119) + +- Union (118) + :- * Project (79) + : +- * Filter (78) + : +- * HashAggregate (77) + : +- Exchange (76) + : +- * HashAggregate (75) + : +- * Project (74) + : +- * BroadcastHashJoin Inner BuildRight (73) + : :- * Project (63) + : : +- * BroadcastHashJoin Inner BuildRight (62) + : : :- * SortMergeJoin LeftSemi (60) : : : :- * Sort (5) : : : : +- Exchange (4) : : : : +- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.store_sales (1) - : : : +- * Sort (62) - : : : +- Exchange (61) - : : : +- * Project (60) - : : : +- * BroadcastHashJoin Inner BuildRight (59) + : : : +- * Sort (59) + : : : +- Exchange (58) + : : : +- * Project (57) + : : : +- * BroadcastHashJoin Inner BuildRight (56) : : : :- * Filter (8) : : : : +- * ColumnarToRow (7) : : : : +- Scan parquet default.item (6) - : : : +- BroadcastExchange (58) - : : : +- * HashAggregate (57) - : : : +- Exchange (56) - : : : +- * HashAggregate (55) - : : : +- * SortMergeJoin LeftSemi (54) - : : : :- * Sort (42) - : : : : +- Exchange (41) - : : : : +- * HashAggregate (40) - : : : : +- Exchange (39) - : : : : +- * HashAggregate (38) - : : : : +- * Project (37) - : : : : +- * BroadcastHashJoin Inner BuildRight (36) - : : : : :- * Project (14) - : : : : : +- * BroadcastHashJoin Inner BuildRight (13) - : : : : : :- * Filter (11) - : : : : : : +- * ColumnarToRow (10) - : : : : : : +- Scan parquet default.store_sales (9) - : : : : : +- ReusedExchange (12) - : : : : +- BroadcastExchange (35) - : : : : +- * SortMergeJoin LeftSemi (34) - : : : : :- * Sort (19) - : : : : : +- Exchange (18) - : : : : : +- * Filter (17) - : : : : : +- * ColumnarToRow (16) - : : : : : +- Scan parquet default.item (15) - : : : : +- * Sort (33) - : : : : +- Exchange (32) - : : : : +- * Project (31) - : : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : : :- * Project (25) - : : : : : +- * BroadcastHashJoin Inner BuildRight (24) - : : : : : :- * Filter (22) - : : : : : : +- * ColumnarToRow (21) - : : : : : : +- Scan parquet default.catalog_sales (20) - : : : : : +- ReusedExchange (23) - : : : : +- BroadcastExchange (29) - : : : : +- * Filter (28) - : : : : +- * ColumnarToRow (27) - : : : : +- Scan parquet default.item (26) - : : : +- * Sort (53) - : : : +- Exchange (52) - : : : +- * Project (51) - : : : +- * BroadcastHashJoin Inner BuildRight (50) - : : : :- * Project (48) - : : : : +- * BroadcastHashJoin Inner BuildRight (47) - : : : : :- * Filter (45) - : : : : : +- * ColumnarToRow (44) - : : : : : +- Scan parquet default.web_sales (43) - : : : : +- ReusedExchange (46) - : : : +- ReusedExchange (49) - : : +- ReusedExchange (64) - : +- BroadcastExchange (75) - : +- * SortMergeJoin LeftSemi (74) - : :- * Sort (71) - : : +- Exchange (70) - : : +- * Filter (69) - : : +- * ColumnarToRow (68) - : : +- Scan parquet default.item (67) - : +- * Sort (73) - : +- ReusedExchange (72) - :- * Project (101) - : +- * Filter (100) - : +- * HashAggregate (99) - : +- Exchange (98) - : +- * HashAggregate (97) - : +- * Project (96) - : +- * BroadcastHashJoin Inner BuildRight (95) - : :- * Project (93) - : : +- * BroadcastHashJoin Inner BuildRight (92) - : : :- * SortMergeJoin LeftSemi (90) - : : : :- * Sort (87) - : : : : +- Exchange (86) - : : : : +- * Filter (85) - : : : : +- * ColumnarToRow (84) - : : : : +- Scan parquet default.catalog_sales (83) - : : : +- * Sort (89) - : : : +- ReusedExchange (88) - : : +- ReusedExchange (91) - : +- ReusedExchange (94) - +- * Project (120) - +- * Filter (119) - +- * HashAggregate (118) - +- Exchange (117) - +- * HashAggregate (116) - +- * Project (115) - +- * BroadcastHashJoin Inner BuildRight (114) - :- * Project (112) - : +- * BroadcastHashJoin Inner BuildRight (111) - : :- * SortMergeJoin LeftSemi (109) - : : :- * Sort (106) - : : : +- Exchange (105) - : : : +- * Filter (104) - : : : +- * ColumnarToRow (103) - : : : +- Scan parquet default.web_sales (102) - : : +- * Sort (108) - : : +- ReusedExchange (107) - : +- ReusedExchange (110) - +- ReusedExchange (113) + : : : +- BroadcastExchange (55) + : : : +- * SortMergeJoin LeftSemi (54) + : : : :- * Sort (42) + : : : : +- Exchange (41) + : : : : +- * HashAggregate (40) + : : : : +- Exchange (39) + : : : : +- * HashAggregate (38) + : : : : +- * Project (37) + : : : : +- * BroadcastHashJoin Inner BuildRight (36) + : : : : :- * Project (14) + : : : : : +- * BroadcastHashJoin Inner BuildRight (13) + : : : : : :- * Filter (11) + : : : : : : +- * ColumnarToRow (10) + : : : : : : +- Scan parquet default.store_sales (9) + : : : : : +- ReusedExchange (12) + : : : : +- BroadcastExchange (35) + : : : : +- * SortMergeJoin LeftSemi (34) + : : : : :- * Sort (19) + : : : : : +- Exchange (18) + : : : : : +- * Filter (17) + : : : : : +- * ColumnarToRow (16) + : : : : : +- Scan parquet default.item (15) + : : : : +- * Sort (33) + : : : : +- Exchange (32) + : : : : +- * Project (31) + : : : : +- * BroadcastHashJoin Inner BuildRight (30) + : : : : :- * Project (25) + : : : : : +- * BroadcastHashJoin Inner BuildRight (24) + : : : : : :- * Filter (22) + : : : : : : +- * ColumnarToRow (21) + : : : : : : +- Scan parquet default.catalog_sales (20) + : : : : : +- ReusedExchange (23) + : : : : +- BroadcastExchange (29) + : : : : +- * Filter (28) + : : : : +- * ColumnarToRow (27) + : : : : +- Scan parquet default.item (26) + : : : +- * Sort (53) + : : : +- Exchange (52) + : : : +- * Project (51) + : : : +- * BroadcastHashJoin Inner BuildRight (50) + : : : :- * Project (48) + : : : : +- * BroadcastHashJoin Inner BuildRight (47) + : : : : :- * Filter (45) + : : : : : +- * ColumnarToRow (44) + : : : : : +- Scan parquet default.web_sales (43) + : : : : +- ReusedExchange (46) + : : : +- ReusedExchange (49) + : : +- ReusedExchange (61) + : +- BroadcastExchange (72) + : +- * SortMergeJoin LeftSemi (71) + : :- * Sort (68) + : : +- Exchange (67) + : : +- * Filter (66) + : : +- * ColumnarToRow (65) + : : +- Scan parquet default.item (64) + : +- * Sort (70) + : +- ReusedExchange (69) + :- * Project (98) + : +- * Filter (97) + : +- * HashAggregate (96) + : +- Exchange (95) + : +- * HashAggregate (94) + : +- * Project (93) + : +- * BroadcastHashJoin Inner BuildRight (92) + : :- * Project (90) + : : +- * BroadcastHashJoin Inner BuildRight (89) + : : :- * SortMergeJoin LeftSemi (87) + : : : :- * Sort (84) + : : : : +- Exchange (83) + : : : : +- * Filter (82) + : : : : +- * ColumnarToRow (81) + : : : : +- Scan parquet default.catalog_sales (80) + : : : +- * Sort (86) + : : : +- ReusedExchange (85) + : : +- ReusedExchange (88) + : +- ReusedExchange (91) + +- * Project (117) + +- * Filter (116) + +- * HashAggregate (115) + +- Exchange (114) + +- * HashAggregate (113) + +- * Project (112) + +- * BroadcastHashJoin Inner BuildRight (111) + :- * Project (109) + : +- * BroadcastHashJoin Inner BuildRight (108) + : :- * SortMergeJoin LeftSemi (106) + : : :- * Sort (103) + : : : +- Exchange (102) + : : : +- * Filter (101) + : : : +- * ColumnarToRow (100) + : : : +- Scan parquet default.web_sales (99) + : : +- * Sort (105) + : : +- ReusedExchange (104) + : +- ReusedExchange (107) + +- ReusedExchange (110) (1) Scan parquet default.store_sales @@ -157,10 +154,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)) @@ -179,7 +176,7 @@ Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] Condition : isnotnull(ss_item_sk#11) -(12) ReusedExchange [Reuses operator id: 155] +(12) ReusedExchange [Reuses operator id: 152] Output [1]: [d_date_sk#14] (13) BroadcastHashJoin [codegen id : 11] @@ -228,7 +225,7 @@ Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] Condition : isnotnull(cs_item_sk#20) -(23) ReusedExchange [Reuses operator id: 155] +(23) ReusedExchange [Reuses operator id: 152] Output [1]: [d_date_sk#22] (24) BroadcastHashJoin [codegen id : 8] @@ -334,7 +331,7 @@ Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] Condition : isnotnull(ws_item_sk#35) -(46) ReusedExchange [Reuses operator id: 155] +(46) ReusedExchange [Reuses operator id: 152] Output [1]: [d_date_sk#37] (47) BroadcastHashJoin [codegen id : 16] @@ -371,519 +368,501 @@ Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id 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 -(55) HashAggregate [codegen id : 18] +(55) BroadcastExchange 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#30, class_id#31, category_id#32] - -(56) 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] - -(57) HashAggregate [codegen id : 19] -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#30, class_id#31, category_id#32] - -(58) 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] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#43] -(59) BroadcastHashJoin [codegen id : 20] +(56) 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] Join condition: None -(60) Project [codegen id : 20] -Output [1]: [i_item_sk#7 AS ss_item_sk#45] +(57) Project [codegen id : 19] +Output [1]: [i_item_sk#7 AS ss_item_sk#44] 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] -(61) Exchange -Input [1]: [ss_item_sk#45] -Arguments: hashpartitioning(ss_item_sk#45, 5), ENSURE_REQUIREMENTS, [id=#46] +(58) Exchange +Input [1]: [ss_item_sk#44] +Arguments: hashpartitioning(ss_item_sk#44, 5), ENSURE_REQUIREMENTS, [id=#45] -(62) Sort [codegen id : 21] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +(59) Sort [codegen id : 20] +Input [1]: [ss_item_sk#44] +Arguments: [ss_item_sk#44 ASC NULLS FIRST], false, 0 -(63) SortMergeJoin [codegen id : 45] +(60) SortMergeJoin [codegen id : 43] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#45] +Right keys [1]: [ss_item_sk#44] Join condition: None -(64) ReusedExchange [Reuses operator id: 150] -Output [1]: [d_date_sk#47] +(61) ReusedExchange [Reuses operator id: 147] +Output [1]: [d_date_sk#46] -(65) BroadcastHashJoin [codegen id : 45] +(62) BroadcastHashJoin [codegen id : 43] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#47] +Right keys [1]: [d_date_sk#46] Join condition: None -(66) Project [codegen id : 45] +(63) Project [codegen id : 43] 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#47] +Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#46] -(67) Scan parquet default.item -Output [4]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51] +(64) Scan parquet default.item +Output [4]: [i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(68) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51] +(65) ColumnarToRow [codegen id : 22] +Input [4]: [i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] -(69) Filter [codegen id : 23] -Input [4]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51] -Condition : isnotnull(i_item_sk#48) +(66) Filter [codegen id : 22] +Input [4]: [i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] +Condition : isnotnull(i_item_sk#47) -(70) Exchange -Input [4]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51] -Arguments: hashpartitioning(i_item_sk#48, 5), ENSURE_REQUIREMENTS, [id=#52] +(67) Exchange +Input [4]: [i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] +Arguments: hashpartitioning(i_item_sk#47, 5), ENSURE_REQUIREMENTS, [id=#51] -(71) Sort [codegen id : 24] -Input [4]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51] -Arguments: [i_item_sk#48 ASC NULLS FIRST], false, 0 +(68) Sort [codegen id : 23] +Input [4]: [i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] +Arguments: [i_item_sk#47 ASC NULLS FIRST], false, 0 -(72) ReusedExchange [Reuses operator id: 61] -Output [1]: [ss_item_sk#45] +(69) ReusedExchange [Reuses operator id: 58] +Output [1]: [ss_item_sk#44] -(73) Sort [codegen id : 43] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +(70) Sort [codegen id : 41] +Input [1]: [ss_item_sk#44] +Arguments: [ss_item_sk#44 ASC NULLS FIRST], false, 0 -(74) SortMergeJoin [codegen id : 44] -Left keys [1]: [i_item_sk#48] -Right keys [1]: [ss_item_sk#45] +(71) SortMergeJoin [codegen id : 42] +Left keys [1]: [i_item_sk#47] +Right keys [1]: [ss_item_sk#44] Join condition: None -(75) BroadcastExchange -Input [4]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#53] +(72) BroadcastExchange +Input [4]: [i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#52] -(76) BroadcastHashJoin [codegen id : 45] +(73) BroadcastHashJoin [codegen id : 43] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#48] +Right keys [1]: [i_item_sk#47] Join condition: None -(77) Project [codegen id : 45] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#49, i_class_id#50, i_category_id#51] -Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51] +(74) Project [codegen id : 43] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#48, i_class_id#49, i_category_id#50] +Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] -(78) HashAggregate [codegen id : 45] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#49, i_class_id#50, i_category_id#51] -Keys [3]: [i_brand_id#49, i_class_id#50, i_category_id#51] +(75) HashAggregate [codegen id : 43] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#48, i_class_id#49, i_category_id#50] +Keys [3]: [i_brand_id#48, i_class_id#49, i_category_id#50] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)] -Aggregate Attributes [3]: [sum#54, isEmpty#55, count#56] -Results [6]: [i_brand_id#49, i_class_id#50, i_category_id#51, sum#57, isEmpty#58, count#59] +Aggregate Attributes [3]: [sum#53, isEmpty#54, count#55] +Results [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum#56, isEmpty#57, count#58] -(79) Exchange -Input [6]: [i_brand_id#49, i_class_id#50, i_category_id#51, sum#57, isEmpty#58, count#59] -Arguments: hashpartitioning(i_brand_id#49, i_class_id#50, i_category_id#51, 5), ENSURE_REQUIREMENTS, [id=#60] +(76) Exchange +Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum#56, isEmpty#57, count#58] +Arguments: hashpartitioning(i_brand_id#48, i_class_id#49, i_category_id#50, 5), ENSURE_REQUIREMENTS, [id=#59] -(80) HashAggregate [codegen id : 46] -Input [6]: [i_brand_id#49, i_class_id#50, i_category_id#51, sum#57, isEmpty#58, count#59] -Keys [3]: [i_brand_id#49, i_class_id#50, i_category_id#51] +(77) HashAggregate [codegen id : 44] +Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum#56, isEmpty#57, count#58] +Keys [3]: [i_brand_id#48, i_class_id#49, i_category_id#50] Functions [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2))), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#61, count(1)#62] -Results [5]: [i_brand_id#49, i_class_id#50, i_category_id#51, sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#61 AS sales#63, count(1)#62 AS number_sales#64] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#60, count(1)#61] +Results [5]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#60 AS sales#62, count(1)#61 AS number_sales#63] -(81) Filter [codegen id : 46] -Input [5]: [i_brand_id#49, i_class_id#50, i_category_id#51, sales#63, number_sales#64] -Condition : (isnotnull(sales#63) AND (cast(sales#63 as decimal(32,6)) > cast(Subquery scalar-subquery#65, [id=#66] as decimal(32,6)))) +(78) Filter [codegen id : 44] +Input [5]: [i_brand_id#48, i_class_id#49, i_category_id#50, sales#62, number_sales#63] +Condition : (isnotnull(sales#62) AND (cast(sales#62 as decimal(32,6)) > cast(Subquery scalar-subquery#64, [id=#65] as decimal(32,6)))) -(82) Project [codegen id : 46] -Output [6]: [sales#63, number_sales#64, store AS channel#67, i_brand_id#49, i_class_id#50, i_category_id#51] -Input [5]: [i_brand_id#49, i_class_id#50, i_category_id#51, sales#63, number_sales#64] +(79) Project [codegen id : 44] +Output [6]: [sales#62, number_sales#63, store AS channel#66, i_brand_id#48, i_class_id#49, i_category_id#50] +Input [5]: [i_brand_id#48, i_class_id#49, i_category_id#50, sales#62, number_sales#63] -(83) Scan parquet default.catalog_sales -Output [4]: [cs_item_sk#68, cs_quantity#69, cs_list_price#70, cs_sold_date_sk#71] +(80) Scan parquet default.catalog_sales +Output [4]: [cs_item_sk#67, cs_quantity#68, cs_list_price#69, cs_sold_date_sk#70] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#71), dynamicpruningexpression(cs_sold_date_sk#71 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#70), dynamicpruningexpression(cs_sold_date_sk#70 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(84) ColumnarToRow [codegen id : 47] -Input [4]: [cs_item_sk#68, cs_quantity#69, cs_list_price#70, cs_sold_date_sk#71] +(81) ColumnarToRow [codegen id : 45] +Input [4]: [cs_item_sk#67, cs_quantity#68, cs_list_price#69, cs_sold_date_sk#70] -(85) Filter [codegen id : 47] -Input [4]: [cs_item_sk#68, cs_quantity#69, cs_list_price#70, cs_sold_date_sk#71] -Condition : isnotnull(cs_item_sk#68) +(82) Filter [codegen id : 45] +Input [4]: [cs_item_sk#67, cs_quantity#68, cs_list_price#69, cs_sold_date_sk#70] +Condition : isnotnull(cs_item_sk#67) -(86) Exchange -Input [4]: [cs_item_sk#68, cs_quantity#69, cs_list_price#70, cs_sold_date_sk#71] -Arguments: hashpartitioning(cs_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#72] +(83) Exchange +Input [4]: [cs_item_sk#67, cs_quantity#68, cs_list_price#69, cs_sold_date_sk#70] +Arguments: hashpartitioning(cs_item_sk#67, 5), ENSURE_REQUIREMENTS, [id=#71] -(87) Sort [codegen id : 48] -Input [4]: [cs_item_sk#68, cs_quantity#69, cs_list_price#70, cs_sold_date_sk#71] -Arguments: [cs_item_sk#68 ASC NULLS FIRST], false, 0 +(84) Sort [codegen id : 46] +Input [4]: [cs_item_sk#67, cs_quantity#68, cs_list_price#69, cs_sold_date_sk#70] +Arguments: [cs_item_sk#67 ASC NULLS FIRST], false, 0 -(88) ReusedExchange [Reuses operator id: 61] -Output [1]: [ss_item_sk#45] +(85) ReusedExchange [Reuses operator id: 58] +Output [1]: [ss_item_sk#44] -(89) Sort [codegen id : 67] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +(86) Sort [codegen id : 64] +Input [1]: [ss_item_sk#44] +Arguments: [ss_item_sk#44 ASC NULLS FIRST], false, 0 -(90) SortMergeJoin [codegen id : 91] -Left keys [1]: [cs_item_sk#68] -Right keys [1]: [ss_item_sk#45] +(87) SortMergeJoin [codegen id : 87] +Left keys [1]: [cs_item_sk#67] +Right keys [1]: [ss_item_sk#44] Join condition: None -(91) ReusedExchange [Reuses operator id: 150] -Output [1]: [d_date_sk#73] +(88) ReusedExchange [Reuses operator id: 147] +Output [1]: [d_date_sk#72] -(92) BroadcastHashJoin [codegen id : 91] -Left keys [1]: [cs_sold_date_sk#71] -Right keys [1]: [d_date_sk#73] +(89) BroadcastHashJoin [codegen id : 87] +Left keys [1]: [cs_sold_date_sk#70] +Right keys [1]: [d_date_sk#72] Join condition: None -(93) Project [codegen id : 91] -Output [3]: [cs_item_sk#68, cs_quantity#69, cs_list_price#70] -Input [5]: [cs_item_sk#68, cs_quantity#69, cs_list_price#70, cs_sold_date_sk#71, d_date_sk#73] +(90) Project [codegen id : 87] +Output [3]: [cs_item_sk#67, cs_quantity#68, cs_list_price#69] +Input [5]: [cs_item_sk#67, cs_quantity#68, cs_list_price#69, cs_sold_date_sk#70, d_date_sk#72] -(94) ReusedExchange [Reuses operator id: 75] -Output [4]: [i_item_sk#74, i_brand_id#75, i_class_id#76, i_category_id#77] +(91) ReusedExchange [Reuses operator id: 72] +Output [4]: [i_item_sk#73, i_brand_id#74, i_class_id#75, i_category_id#76] -(95) BroadcastHashJoin [codegen id : 91] -Left keys [1]: [cs_item_sk#68] -Right keys [1]: [i_item_sk#74] +(92) BroadcastHashJoin [codegen id : 87] +Left keys [1]: [cs_item_sk#67] +Right keys [1]: [i_item_sk#73] Join condition: None -(96) Project [codegen id : 91] -Output [5]: [cs_quantity#69, cs_list_price#70, i_brand_id#75, i_class_id#76, i_category_id#77] -Input [7]: [cs_item_sk#68, cs_quantity#69, cs_list_price#70, i_item_sk#74, i_brand_id#75, i_class_id#76, i_category_id#77] - -(97) HashAggregate [codegen id : 91] -Input [5]: [cs_quantity#69, cs_list_price#70, i_brand_id#75, i_class_id#76, i_category_id#77] -Keys [3]: [i_brand_id#75, i_class_id#76, i_category_id#77] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cs_quantity#69 as decimal(12,2))) * promote_precision(cast(cs_list_price#70 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)] -Aggregate Attributes [3]: [sum#78, isEmpty#79, count#80] -Results [6]: [i_brand_id#75, i_class_id#76, i_category_id#77, sum#81, isEmpty#82, count#83] - -(98) Exchange -Input [6]: [i_brand_id#75, i_class_id#76, i_category_id#77, sum#81, isEmpty#82, count#83] -Arguments: hashpartitioning(i_brand_id#75, i_class_id#76, i_category_id#77, 5), ENSURE_REQUIREMENTS, [id=#84] - -(99) HashAggregate [codegen id : 92] -Input [6]: [i_brand_id#75, i_class_id#76, i_category_id#77, sum#81, isEmpty#82, count#83] -Keys [3]: [i_brand_id#75, i_class_id#76, i_category_id#77] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cs_quantity#69 as decimal(12,2))) * promote_precision(cast(cs_list_price#70 as decimal(12,2)))), DecimalType(18,2))), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cs_quantity#69 as decimal(12,2))) * promote_precision(cast(cs_list_price#70 as decimal(12,2)))), DecimalType(18,2)))#85, count(1)#86] -Results [5]: [i_brand_id#75, i_class_id#76, i_category_id#77, sum(CheckOverflow((promote_precision(cast(cs_quantity#69 as decimal(12,2))) * promote_precision(cast(cs_list_price#70 as decimal(12,2)))), DecimalType(18,2)))#85 AS sales#87, count(1)#86 AS number_sales#88] - -(100) Filter [codegen id : 92] -Input [5]: [i_brand_id#75, i_class_id#76, i_category_id#77, sales#87, number_sales#88] -Condition : (isnotnull(sales#87) AND (cast(sales#87 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#65, [id=#66] as decimal(32,6)))) - -(101) Project [codegen id : 92] -Output [6]: [sales#87, number_sales#88, catalog AS channel#89, i_brand_id#75, i_class_id#76, i_category_id#77] -Input [5]: [i_brand_id#75, i_class_id#76, i_category_id#77, sales#87, number_sales#88] - -(102) Scan parquet default.web_sales -Output [4]: [ws_item_sk#90, ws_quantity#91, ws_list_price#92, ws_sold_date_sk#93] +(93) Project [codegen id : 87] +Output [5]: [cs_quantity#68, cs_list_price#69, i_brand_id#74, i_class_id#75, i_category_id#76] +Input [7]: [cs_item_sk#67, cs_quantity#68, cs_list_price#69, i_item_sk#73, i_brand_id#74, i_class_id#75, i_category_id#76] + +(94) HashAggregate [codegen id : 87] +Input [5]: [cs_quantity#68, cs_list_price#69, 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(cs_quantity#68 as decimal(12,2))) * promote_precision(cast(cs_list_price#69 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)] +Aggregate Attributes [3]: [sum#77, isEmpty#78, count#79] +Results [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#80, isEmpty#81, count#82] + +(95) Exchange +Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#80, isEmpty#81, count#82] +Arguments: hashpartitioning(i_brand_id#74, i_class_id#75, i_category_id#76, 5), ENSURE_REQUIREMENTS, [id=#83] + +(96) HashAggregate [codegen id : 88] +Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#80, isEmpty#81, count#82] +Keys [3]: [i_brand_id#74, i_class_id#75, i_category_id#76] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cs_quantity#68 as decimal(12,2))) * promote_precision(cast(cs_list_price#69 as decimal(12,2)))), DecimalType(18,2))), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cs_quantity#68 as decimal(12,2))) * promote_precision(cast(cs_list_price#69 as decimal(12,2)))), DecimalType(18,2)))#84, count(1)#85] +Results [5]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum(CheckOverflow((promote_precision(cast(cs_quantity#68 as decimal(12,2))) * promote_precision(cast(cs_list_price#69 as decimal(12,2)))), DecimalType(18,2)))#84 AS sales#86, count(1)#85 AS number_sales#87] + +(97) Filter [codegen id : 88] +Input [5]: [i_brand_id#74, i_class_id#75, i_category_id#76, sales#86, number_sales#87] +Condition : (isnotnull(sales#86) AND (cast(sales#86 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#64, [id=#65] as decimal(32,6)))) + +(98) Project [codegen id : 88] +Output [6]: [sales#86, number_sales#87, catalog AS channel#88, i_brand_id#74, i_class_id#75, i_category_id#76] +Input [5]: [i_brand_id#74, i_class_id#75, i_category_id#76, sales#86, number_sales#87] + +(99) Scan parquet default.web_sales +Output [4]: [ws_item_sk#89, ws_quantity#90, ws_list_price#91, ws_sold_date_sk#92] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#93), dynamicpruningexpression(ws_sold_date_sk#93 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#92), dynamicpruningexpression(ws_sold_date_sk#92 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(103) ColumnarToRow [codegen id : 93] -Input [4]: [ws_item_sk#90, ws_quantity#91, ws_list_price#92, ws_sold_date_sk#93] +(100) ColumnarToRow [codegen id : 89] +Input [4]: [ws_item_sk#89, ws_quantity#90, ws_list_price#91, ws_sold_date_sk#92] -(104) Filter [codegen id : 93] -Input [4]: [ws_item_sk#90, ws_quantity#91, ws_list_price#92, ws_sold_date_sk#93] -Condition : isnotnull(ws_item_sk#90) +(101) Filter [codegen id : 89] +Input [4]: [ws_item_sk#89, ws_quantity#90, ws_list_price#91, ws_sold_date_sk#92] +Condition : isnotnull(ws_item_sk#89) -(105) Exchange -Input [4]: [ws_item_sk#90, ws_quantity#91, ws_list_price#92, ws_sold_date_sk#93] -Arguments: hashpartitioning(ws_item_sk#90, 5), ENSURE_REQUIREMENTS, [id=#94] +(102) Exchange +Input [4]: [ws_item_sk#89, ws_quantity#90, ws_list_price#91, ws_sold_date_sk#92] +Arguments: hashpartitioning(ws_item_sk#89, 5), ENSURE_REQUIREMENTS, [id=#93] -(106) Sort [codegen id : 94] -Input [4]: [ws_item_sk#90, ws_quantity#91, ws_list_price#92, ws_sold_date_sk#93] -Arguments: [ws_item_sk#90 ASC NULLS FIRST], false, 0 +(103) Sort [codegen id : 90] +Input [4]: [ws_item_sk#89, ws_quantity#90, ws_list_price#91, ws_sold_date_sk#92] +Arguments: [ws_item_sk#89 ASC NULLS FIRST], false, 0 -(107) ReusedExchange [Reuses operator id: 61] -Output [1]: [ss_item_sk#45] +(104) ReusedExchange [Reuses operator id: 58] +Output [1]: [ss_item_sk#44] -(108) Sort [codegen id : 113] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +(105) Sort [codegen id : 108] +Input [1]: [ss_item_sk#44] +Arguments: [ss_item_sk#44 ASC NULLS FIRST], false, 0 -(109) SortMergeJoin [codegen id : 137] -Left keys [1]: [ws_item_sk#90] -Right keys [1]: [ss_item_sk#45] +(106) SortMergeJoin [codegen id : 131] +Left keys [1]: [ws_item_sk#89] +Right keys [1]: [ss_item_sk#44] Join condition: None -(110) ReusedExchange [Reuses operator id: 150] -Output [1]: [d_date_sk#95] +(107) ReusedExchange [Reuses operator id: 147] +Output [1]: [d_date_sk#94] -(111) BroadcastHashJoin [codegen id : 137] -Left keys [1]: [ws_sold_date_sk#93] -Right keys [1]: [d_date_sk#95] +(108) BroadcastHashJoin [codegen id : 131] +Left keys [1]: [ws_sold_date_sk#92] +Right keys [1]: [d_date_sk#94] Join condition: None -(112) Project [codegen id : 137] -Output [3]: [ws_item_sk#90, ws_quantity#91, ws_list_price#92] -Input [5]: [ws_item_sk#90, ws_quantity#91, ws_list_price#92, ws_sold_date_sk#93, d_date_sk#95] +(109) Project [codegen id : 131] +Output [3]: [ws_item_sk#89, ws_quantity#90, ws_list_price#91] +Input [5]: [ws_item_sk#89, ws_quantity#90, ws_list_price#91, ws_sold_date_sk#92, d_date_sk#94] -(113) ReusedExchange [Reuses operator id: 75] -Output [4]: [i_item_sk#96, i_brand_id#97, i_class_id#98, i_category_id#99] +(110) ReusedExchange [Reuses operator id: 72] +Output [4]: [i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98] -(114) BroadcastHashJoin [codegen id : 137] -Left keys [1]: [ws_item_sk#90] -Right keys [1]: [i_item_sk#96] +(111) BroadcastHashJoin [codegen id : 131] +Left keys [1]: [ws_item_sk#89] +Right keys [1]: [i_item_sk#95] Join condition: None -(115) Project [codegen id : 137] -Output [5]: [ws_quantity#91, ws_list_price#92, i_brand_id#97, i_class_id#98, i_category_id#99] -Input [7]: [ws_item_sk#90, ws_quantity#91, ws_list_price#92, i_item_sk#96, i_brand_id#97, i_class_id#98, i_category_id#99] - -(116) HashAggregate [codegen id : 137] -Input [5]: [ws_quantity#91, ws_list_price#92, i_brand_id#97, i_class_id#98, i_category_id#99] -Keys [3]: [i_brand_id#97, i_class_id#98, i_category_id#99] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(ws_quantity#91 as decimal(12,2))) * promote_precision(cast(ws_list_price#92 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)] -Aggregate Attributes [3]: [sum#100, isEmpty#101, count#102] -Results [6]: [i_brand_id#97, i_class_id#98, i_category_id#99, sum#103, isEmpty#104, count#105] - -(117) Exchange -Input [6]: [i_brand_id#97, i_class_id#98, i_category_id#99, sum#103, isEmpty#104, count#105] -Arguments: hashpartitioning(i_brand_id#97, i_class_id#98, i_category_id#99, 5), ENSURE_REQUIREMENTS, [id=#106] - -(118) HashAggregate [codegen id : 138] -Input [6]: [i_brand_id#97, i_class_id#98, i_category_id#99, sum#103, isEmpty#104, count#105] -Keys [3]: [i_brand_id#97, i_class_id#98, i_category_id#99] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(ws_quantity#91 as decimal(12,2))) * promote_precision(cast(ws_list_price#92 as decimal(12,2)))), DecimalType(18,2))), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ws_quantity#91 as decimal(12,2))) * promote_precision(cast(ws_list_price#92 as decimal(12,2)))), DecimalType(18,2)))#107, count(1)#108] -Results [5]: [i_brand_id#97, i_class_id#98, i_category_id#99, sum(CheckOverflow((promote_precision(cast(ws_quantity#91 as decimal(12,2))) * promote_precision(cast(ws_list_price#92 as decimal(12,2)))), DecimalType(18,2)))#107 AS sales#109, count(1)#108 AS number_sales#110] - -(119) Filter [codegen id : 138] -Input [5]: [i_brand_id#97, i_class_id#98, i_category_id#99, sales#109, number_sales#110] -Condition : (isnotnull(sales#109) AND (cast(sales#109 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#65, [id=#66] as decimal(32,6)))) - -(120) Project [codegen id : 138] -Output [6]: [sales#109, number_sales#110, web AS channel#111, i_brand_id#97, i_class_id#98, i_category_id#99] -Input [5]: [i_brand_id#97, i_class_id#98, i_category_id#99, sales#109, number_sales#110] - -(121) Union - -(122) Expand [codegen id : 139] -Input [6]: [sales#63, number_sales#64, channel#67, i_brand_id#49, i_class_id#50, i_category_id#51] -Arguments: [[sales#63, number_sales#64, channel#67, i_brand_id#49, i_class_id#50, i_category_id#51, 0], [sales#63, number_sales#64, channel#67, i_brand_id#49, i_class_id#50, null, 1], [sales#63, number_sales#64, channel#67, i_brand_id#49, null, null, 3], [sales#63, number_sales#64, channel#67, null, null, null, 7], [sales#63, number_sales#64, null, null, null, null, 15]], [sales#63, number_sales#64, channel#112, i_brand_id#113, i_class_id#114, i_category_id#115, spark_grouping_id#116] - -(123) HashAggregate [codegen id : 139] -Input [7]: [sales#63, number_sales#64, channel#112, i_brand_id#113, i_class_id#114, i_category_id#115, spark_grouping_id#116] -Keys [5]: [channel#112, i_brand_id#113, i_class_id#114, i_category_id#115, spark_grouping_id#116] -Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] -Aggregate Attributes [3]: [sum#117, isEmpty#118, sum#119] -Results [8]: [channel#112, i_brand_id#113, i_class_id#114, i_category_id#115, spark_grouping_id#116, sum#120, isEmpty#121, sum#122] - -(124) Exchange -Input [8]: [channel#112, i_brand_id#113, i_class_id#114, i_category_id#115, spark_grouping_id#116, sum#120, isEmpty#121, sum#122] -Arguments: hashpartitioning(channel#112, i_brand_id#113, i_class_id#114, i_category_id#115, spark_grouping_id#116, 5), ENSURE_REQUIREMENTS, [id=#123] - -(125) HashAggregate [codegen id : 140] -Input [8]: [channel#112, i_brand_id#113, i_class_id#114, i_category_id#115, spark_grouping_id#116, sum#120, isEmpty#121, sum#122] -Keys [5]: [channel#112, i_brand_id#113, i_class_id#114, i_category_id#115, spark_grouping_id#116] -Functions [2]: [sum(sales#63), sum(number_sales#64)] -Aggregate Attributes [2]: [sum(sales#63)#124, sum(number_sales#64)#125] -Results [6]: [channel#112, i_brand_id#113, i_class_id#114, i_category_id#115, sum(sales#63)#124 AS sum(sales)#126, sum(number_sales#64)#125 AS sum(number_sales)#127] - -(126) TakeOrderedAndProject -Input [6]: [channel#112, i_brand_id#113, i_class_id#114, i_category_id#115, sum(sales)#126, sum(number_sales)#127] -Arguments: 100, [channel#112 ASC NULLS FIRST, i_brand_id#113 ASC NULLS FIRST, i_class_id#114 ASC NULLS FIRST, i_category_id#115 ASC NULLS FIRST], [channel#112, i_brand_id#113, i_class_id#114, i_category_id#115, sum(sales)#126, sum(number_sales)#127] +(112) Project [codegen id : 131] +Output [5]: [ws_quantity#90, ws_list_price#91, i_brand_id#96, i_class_id#97, i_category_id#98] +Input [7]: [ws_item_sk#89, ws_quantity#90, ws_list_price#91, i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98] + +(113) HashAggregate [codegen id : 131] +Input [5]: [ws_quantity#90, ws_list_price#91, 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(ws_quantity#90 as decimal(12,2))) * promote_precision(cast(ws_list_price#91 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)] +Aggregate Attributes [3]: [sum#99, isEmpty#100, count#101] +Results [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#102, isEmpty#103, count#104] + +(114) Exchange +Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#102, isEmpty#103, count#104] +Arguments: hashpartitioning(i_brand_id#96, i_class_id#97, i_category_id#98, 5), ENSURE_REQUIREMENTS, [id=#105] + +(115) HashAggregate [codegen id : 132] +Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#102, isEmpty#103, count#104] +Keys [3]: [i_brand_id#96, i_class_id#97, i_category_id#98] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(ws_quantity#90 as decimal(12,2))) * promote_precision(cast(ws_list_price#91 as decimal(12,2)))), DecimalType(18,2))), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ws_quantity#90 as decimal(12,2))) * promote_precision(cast(ws_list_price#91 as decimal(12,2)))), DecimalType(18,2)))#106, count(1)#107] +Results [5]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum(CheckOverflow((promote_precision(cast(ws_quantity#90 as decimal(12,2))) * promote_precision(cast(ws_list_price#91 as decimal(12,2)))), DecimalType(18,2)))#106 AS sales#108, count(1)#107 AS number_sales#109] + +(116) Filter [codegen id : 132] +Input [5]: [i_brand_id#96, i_class_id#97, i_category_id#98, sales#108, number_sales#109] +Condition : (isnotnull(sales#108) AND (cast(sales#108 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#64, [id=#65] as decimal(32,6)))) + +(117) Project [codegen id : 132] +Output [6]: [sales#108, number_sales#109, web AS channel#110, i_brand_id#96, i_class_id#97, i_category_id#98] +Input [5]: [i_brand_id#96, i_class_id#97, i_category_id#98, sales#108, number_sales#109] + +(118) Union + +(119) Expand [codegen id : 133] +Input [6]: [sales#62, number_sales#63, channel#66, i_brand_id#48, i_class_id#49, i_category_id#50] +Arguments: [[sales#62, number_sales#63, channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, 0], [sales#62, number_sales#63, channel#66, i_brand_id#48, i_class_id#49, null, 1], [sales#62, number_sales#63, channel#66, i_brand_id#48, null, null, 3], [sales#62, number_sales#63, channel#66, null, null, null, 7], [sales#62, number_sales#63, null, null, null, null, 15]], [sales#62, number_sales#63, channel#111, i_brand_id#112, i_class_id#113, i_category_id#114, spark_grouping_id#115] + +(120) HashAggregate [codegen id : 133] +Input [7]: [sales#62, number_sales#63, channel#111, i_brand_id#112, i_class_id#113, i_category_id#114, spark_grouping_id#115] +Keys [5]: [channel#111, i_brand_id#112, i_class_id#113, i_category_id#114, spark_grouping_id#115] +Functions [2]: [partial_sum(sales#62), partial_sum(number_sales#63)] +Aggregate Attributes [3]: [sum#116, isEmpty#117, sum#118] +Results [8]: [channel#111, i_brand_id#112, i_class_id#113, i_category_id#114, spark_grouping_id#115, sum#119, isEmpty#120, sum#121] + +(121) Exchange +Input [8]: [channel#111, i_brand_id#112, i_class_id#113, i_category_id#114, spark_grouping_id#115, sum#119, isEmpty#120, sum#121] +Arguments: hashpartitioning(channel#111, i_brand_id#112, i_class_id#113, i_category_id#114, spark_grouping_id#115, 5), ENSURE_REQUIREMENTS, [id=#122] + +(122) HashAggregate [codegen id : 134] +Input [8]: [channel#111, i_brand_id#112, i_class_id#113, i_category_id#114, spark_grouping_id#115, sum#119, isEmpty#120, sum#121] +Keys [5]: [channel#111, i_brand_id#112, i_class_id#113, i_category_id#114, spark_grouping_id#115] +Functions [2]: [sum(sales#62), sum(number_sales#63)] +Aggregate Attributes [2]: [sum(sales#62)#123, sum(number_sales#63)#124] +Results [6]: [channel#111, i_brand_id#112, i_class_id#113, i_category_id#114, sum(sales#62)#123 AS sum(sales)#125, sum(number_sales#63)#124 AS sum(number_sales)#126] + +(123) TakeOrderedAndProject +Input [6]: [channel#111, i_brand_id#112, i_class_id#113, i_category_id#114, sum(sales)#125, sum(number_sales)#126] +Arguments: 100, [channel#111 ASC NULLS FIRST, i_brand_id#112 ASC NULLS FIRST, i_class_id#113 ASC NULLS FIRST, i_category_id#114 ASC NULLS FIRST], [channel#111, i_brand_id#112, i_class_id#113, i_category_id#114, sum(sales)#125, sum(number_sales)#126] ===== Subqueries ===== -Subquery:1 Hosting operator id = 81 Hosting Expression = Subquery scalar-subquery#65, [id=#66] -* HashAggregate (145) -+- Exchange (144) - +- * HashAggregate (143) - +- Union (142) - :- * Project (131) - : +- * BroadcastHashJoin Inner BuildRight (130) - : :- * ColumnarToRow (128) - : : +- Scan parquet default.store_sales (127) - : +- ReusedExchange (129) - :- * Project (136) - : +- * BroadcastHashJoin Inner BuildRight (135) - : :- * ColumnarToRow (133) - : : +- Scan parquet default.catalog_sales (132) - : +- ReusedExchange (134) - +- * Project (141) - +- * BroadcastHashJoin Inner BuildRight (140) - :- * ColumnarToRow (138) - : +- Scan parquet default.web_sales (137) - +- ReusedExchange (139) - - -(127) Scan parquet default.store_sales -Output [3]: [ss_quantity#128, ss_list_price#129, ss_sold_date_sk#130] +Subquery:1 Hosting operator id = 78 Hosting Expression = Subquery scalar-subquery#64, [id=#65] +* HashAggregate (142) ++- Exchange (141) + +- * HashAggregate (140) + +- Union (139) + :- * Project (128) + : +- * BroadcastHashJoin Inner BuildRight (127) + : :- * ColumnarToRow (125) + : : +- Scan parquet default.store_sales (124) + : +- ReusedExchange (126) + :- * Project (133) + : +- * BroadcastHashJoin Inner BuildRight (132) + : :- * ColumnarToRow (130) + : : +- Scan parquet default.catalog_sales (129) + : +- ReusedExchange (131) + +- * Project (138) + +- * BroadcastHashJoin Inner BuildRight (137) + :- * ColumnarToRow (135) + : +- Scan parquet default.web_sales (134) + +- ReusedExchange (136) + + +(124) Scan parquet default.store_sales +Output [3]: [ss_quantity#127, ss_list_price#128, ss_sold_date_sk#129] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#130), dynamicpruningexpression(ss_sold_date_sk#130 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ss_sold_date_sk#129), dynamicpruningexpression(ss_sold_date_sk#129 IN dynamicpruning#13)] ReadSchema: struct -(128) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#128, ss_list_price#129, ss_sold_date_sk#130] +(125) ColumnarToRow [codegen id : 2] +Input [3]: [ss_quantity#127, ss_list_price#128, ss_sold_date_sk#129] -(129) ReusedExchange [Reuses operator id: 155] -Output [1]: [d_date_sk#131] +(126) ReusedExchange [Reuses operator id: 152] +Output [1]: [d_date_sk#130] -(130) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#130] -Right keys [1]: [d_date_sk#131] +(127) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#129] +Right keys [1]: [d_date_sk#130] Join condition: None -(131) Project [codegen id : 2] -Output [2]: [ss_quantity#128 AS quantity#132, ss_list_price#129 AS list_price#133] -Input [4]: [ss_quantity#128, ss_list_price#129, ss_sold_date_sk#130, d_date_sk#131] +(128) Project [codegen id : 2] +Output [2]: [ss_quantity#127 AS quantity#131, ss_list_price#128 AS list_price#132] +Input [4]: [ss_quantity#127, ss_list_price#128, ss_sold_date_sk#129, d_date_sk#130] -(132) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#134, cs_list_price#135, cs_sold_date_sk#136] +(129) Scan parquet default.catalog_sales +Output [3]: [cs_quantity#133, cs_list_price#134, cs_sold_date_sk#135] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#136), dynamicpruningexpression(cs_sold_date_sk#136 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(cs_sold_date_sk#135), dynamicpruningexpression(cs_sold_date_sk#135 IN dynamicpruning#13)] ReadSchema: struct -(133) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#134, cs_list_price#135, cs_sold_date_sk#136] +(130) ColumnarToRow [codegen id : 4] +Input [3]: [cs_quantity#133, cs_list_price#134, cs_sold_date_sk#135] -(134) ReusedExchange [Reuses operator id: 155] -Output [1]: [d_date_sk#137] +(131) ReusedExchange [Reuses operator id: 152] +Output [1]: [d_date_sk#136] -(135) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#136] -Right keys [1]: [d_date_sk#137] +(132) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_sold_date_sk#135] +Right keys [1]: [d_date_sk#136] Join condition: None -(136) Project [codegen id : 4] -Output [2]: [cs_quantity#134 AS quantity#138, cs_list_price#135 AS list_price#139] -Input [4]: [cs_quantity#134, cs_list_price#135, cs_sold_date_sk#136, d_date_sk#137] +(133) Project [codegen id : 4] +Output [2]: [cs_quantity#133 AS quantity#137, cs_list_price#134 AS list_price#138] +Input [4]: [cs_quantity#133, cs_list_price#134, cs_sold_date_sk#135, d_date_sk#136] -(137) Scan parquet default.web_sales -Output [3]: [ws_quantity#140, ws_list_price#141, ws_sold_date_sk#142] +(134) Scan parquet default.web_sales +Output [3]: [ws_quantity#139, ws_list_price#140, ws_sold_date_sk#141] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#142), dynamicpruningexpression(ws_sold_date_sk#142 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ws_sold_date_sk#141), dynamicpruningexpression(ws_sold_date_sk#141 IN dynamicpruning#13)] ReadSchema: struct -(138) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#140, ws_list_price#141, ws_sold_date_sk#142] +(135) ColumnarToRow [codegen id : 6] +Input [3]: [ws_quantity#139, ws_list_price#140, ws_sold_date_sk#141] -(139) ReusedExchange [Reuses operator id: 155] -Output [1]: [d_date_sk#143] +(136) ReusedExchange [Reuses operator id: 152] +Output [1]: [d_date_sk#142] -(140) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#142] -Right keys [1]: [d_date_sk#143] +(137) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#141] +Right keys [1]: [d_date_sk#142] Join condition: None -(141) Project [codegen id : 6] -Output [2]: [ws_quantity#140 AS quantity#144, ws_list_price#141 AS list_price#145] -Input [4]: [ws_quantity#140, ws_list_price#141, ws_sold_date_sk#142, d_date_sk#143] +(138) Project [codegen id : 6] +Output [2]: [ws_quantity#139 AS quantity#143, ws_list_price#140 AS list_price#144] +Input [4]: [ws_quantity#139, ws_list_price#140, ws_sold_date_sk#141, d_date_sk#142] -(142) Union +(139) Union -(143) HashAggregate [codegen id : 7] -Input [2]: [quantity#132, list_price#133] +(140) HashAggregate [codegen id : 7] +Input [2]: [quantity#131, list_price#132] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(quantity#132 as decimal(12,2))) * promote_precision(cast(list_price#133 as decimal(12,2)))), DecimalType(18,2)))] -Aggregate Attributes [2]: [sum#146, count#147] -Results [2]: [sum#148, count#149] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(quantity#131 as decimal(12,2))) * promote_precision(cast(list_price#132 as decimal(12,2)))), DecimalType(18,2)))] +Aggregate Attributes [2]: [sum#145, count#146] +Results [2]: [sum#147, count#148] -(144) Exchange -Input [2]: [sum#148, count#149] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#150] +(141) Exchange +Input [2]: [sum#147, count#148] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#149] -(145) HashAggregate [codegen id : 8] -Input [2]: [sum#148, count#149] +(142) HashAggregate [codegen id : 8] +Input [2]: [sum#147, count#148] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(quantity#132 as decimal(12,2))) * promote_precision(cast(list_price#133 as decimal(12,2)))), DecimalType(18,2)))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(quantity#132 as decimal(12,2))) * promote_precision(cast(list_price#133 as decimal(12,2)))), DecimalType(18,2)))#151] -Results [1]: [avg(CheckOverflow((promote_precision(cast(quantity#132 as decimal(12,2))) * promote_precision(cast(list_price#133 as decimal(12,2)))), DecimalType(18,2)))#151 AS average_sales#152] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(quantity#131 as decimal(12,2))) * promote_precision(cast(list_price#132 as decimal(12,2)))), DecimalType(18,2)))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(quantity#131 as decimal(12,2))) * promote_precision(cast(list_price#132 as decimal(12,2)))), DecimalType(18,2)))#150] +Results [1]: [avg(CheckOverflow((promote_precision(cast(quantity#131 as decimal(12,2))) * promote_precision(cast(list_price#132 as decimal(12,2)))), DecimalType(18,2)))#150 AS average_sales#151] -Subquery:2 Hosting operator id = 127 Hosting Expression = ss_sold_date_sk#130 IN dynamicpruning#13 +Subquery:2 Hosting operator id = 124 Hosting Expression = ss_sold_date_sk#129 IN dynamicpruning#13 -Subquery:3 Hosting operator id = 132 Hosting Expression = cs_sold_date_sk#136 IN dynamicpruning#13 +Subquery:3 Hosting operator id = 129 Hosting Expression = cs_sold_date_sk#135 IN dynamicpruning#13 -Subquery:4 Hosting operator id = 137 Hosting Expression = ws_sold_date_sk#142 IN dynamicpruning#13 +Subquery:4 Hosting operator id = 134 Hosting Expression = ws_sold_date_sk#141 IN dynamicpruning#13 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (150) -+- * Project (149) - +- * Filter (148) - +- * ColumnarToRow (147) - +- Scan parquet default.date_dim (146) +BroadcastExchange (147) ++- * Project (146) + +- * Filter (145) + +- * ColumnarToRow (144) + +- Scan parquet default.date_dim (143) -(146) Scan parquet default.date_dim -Output [3]: [d_date_sk#47, d_year#153, d_moy#154] +(143) Scan parquet default.date_dim +Output [3]: [d_date_sk#46, d_year#152, d_moy#153] 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 -(147) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#47, d_year#153, d_moy#154] +(144) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#46, d_year#152, d_moy#153] -(148) Filter [codegen id : 1] -Input [3]: [d_date_sk#47, d_year#153, d_moy#154] -Condition : ((((isnotnull(d_year#153) AND isnotnull(d_moy#154)) AND (d_year#153 = 2001)) AND (d_moy#154 = 11)) AND isnotnull(d_date_sk#47)) +(145) Filter [codegen id : 1] +Input [3]: [d_date_sk#46, d_year#152, d_moy#153] +Condition : ((((isnotnull(d_year#152) AND isnotnull(d_moy#153)) AND (d_year#152 = 2001)) AND (d_moy#153 = 11)) AND isnotnull(d_date_sk#46)) -(149) Project [codegen id : 1] -Output [1]: [d_date_sk#47] -Input [3]: [d_date_sk#47, d_year#153, d_moy#154] +(146) Project [codegen id : 1] +Output [1]: [d_date_sk#46] +Input [3]: [d_date_sk#46, d_year#152, d_moy#153] -(150) BroadcastExchange -Input [1]: [d_date_sk#47] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#155] +(147) BroadcastExchange +Input [1]: [d_date_sk#46] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#154] Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (155) -+- * Project (154) - +- * Filter (153) - +- * ColumnarToRow (152) - +- Scan parquet default.date_dim (151) +BroadcastExchange (152) ++- * Project (151) + +- * Filter (150) + +- * ColumnarToRow (149) + +- Scan parquet default.date_dim (148) -(151) Scan parquet default.date_dim -Output [2]: [d_date_sk#14, d_year#156] +(148) Scan parquet default.date_dim +Output [2]: [d_date_sk#14, d_year#155] 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 -(152) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#14, d_year#156] +(149) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#14, d_year#155] -(153) Filter [codegen id : 1] -Input [2]: [d_date_sk#14, d_year#156] -Condition : (((isnotnull(d_year#156) AND (d_year#156 >= 1999)) AND (d_year#156 <= 2001)) AND isnotnull(d_date_sk#14)) +(150) Filter [codegen id : 1] +Input [2]: [d_date_sk#14, d_year#155] +Condition : (((isnotnull(d_year#155) AND (d_year#155 >= 1999)) AND (d_year#155 <= 2001)) AND isnotnull(d_date_sk#14)) -(154) Project [codegen id : 1] +(151) Project [codegen id : 1] Output [1]: [d_date_sk#14] -Input [2]: [d_date_sk#14, d_year#156] +Input [2]: [d_date_sk#14, d_year#155] -(155) BroadcastExchange +(152) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#157] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#156] Subquery:7 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#13 Subquery:8 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#13 -Subquery:9 Hosting operator id = 100 Hosting Expression = ReusedSubquery Subquery scalar-subquery#65, [id=#66] +Subquery:9 Hosting operator id = 97 Hosting Expression = ReusedSubquery Subquery scalar-subquery#64, [id=#65] -Subquery:10 Hosting operator id = 83 Hosting Expression = cs_sold_date_sk#71 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 80 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 119 Hosting Expression = ReusedSubquery Subquery scalar-subquery#65, [id=#66] +Subquery:11 Hosting operator id = 116 Hosting Expression = ReusedSubquery Subquery scalar-subquery#64, [id=#65] -Subquery:12 Hosting operator id = 102 Hosting Expression = ws_sold_date_sk#93 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 99 Hosting Expression = ws_sold_date_sk#92 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt index 5984e5165f78d..f445a370581af 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,21 +1,21 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] - WholeStageCodegen (140) + WholeStageCodegen (134) 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 (139) + WholeStageCodegen (133) 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 (46) + WholeStageCodegen (44) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sales] Subquery #3 WholeStageCodegen (8) HashAggregate [sum,count] [avg(CheckOverflow((promote_precision(cast(quantity as decimal(12,2))) * promote_precision(cast(list_price as decimal(12,2)))), DecimalType(18,2))),average_sales,sum,count] InputAdapter - Exchange #18 + Exchange #17 WholeStageCodegen (7) HashAggregate [quantity,list_price] [sum,count,sum,count] InputAdapter @@ -28,7 +28,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Scan parquet default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #10 + ReusedExchange [d_date_sk] #9 WholeStageCodegen (4) Project [cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] @@ -37,7 +37,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Scan parquet default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #10 + ReusedExchange [d_date_sk] #9 WholeStageCodegen (6) Project [ws_quantity,ws_list_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] @@ -46,11 +46,11 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Scan parquet default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #10 + ReusedExchange [d_date_sk] #9 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(ss_quantity as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2))),count(1),sales,number_sales,sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (45) + WholeStageCodegen (43) 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 InputAdapter Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - 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,128 +89,123 @@ 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) - HashAggregate [brand_id,class_id,category_id] + WholeStageCodegen (18) + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] InputAdapter - Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (18) - HashAggregate [brand_id,class_id,category_id] - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - InputAdapter - WholeStageCodegen (13) - Sort [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #8 - WholeStageCodegen (12) - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #9 - 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 - BroadcastExchange #10 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] + WholeStageCodegen (13) + Sort [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #7 + WholeStageCodegen (12) + HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #8 + WholeStageCodegen (11) + HashAggregate [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #9 + WholeStageCodegen (1) + 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] #9 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (10) + SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (5) + Sort [i_brand_id,i_class_id,i_category_id] InputAdapter - ReusedExchange [d_date_sk] #10 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (10) - SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - InputAdapter - WholeStageCodegen (5) - Sort [i_brand_id,i_class_id,i_category_id] + 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 - 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] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (9) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #12 + WholeStageCodegen (8) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #9 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (7) + Filter [i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - WholeStageCodegen (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.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - WholeStageCodegen (17) - Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (17) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #14 + WholeStageCodegen (16) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #9 InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #15 - WholeStageCodegen (16) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #10 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #16 - WholeStageCodegen (44) + BroadcastExchange #15 + WholeStageCodegen (42) SortMergeJoin [i_item_sk,ss_item_sk] InputAdapter - WholeStageCodegen (24) + WholeStageCodegen (23) Sort [i_item_sk] InputAdapter - Exchange [i_item_sk] #17 - WholeStageCodegen (23) + Exchange [i_item_sk] #16 + WholeStageCodegen (22) Filter [i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter - WholeStageCodegen (43) + WholeStageCodegen (41) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #5 - WholeStageCodegen (92) + WholeStageCodegen (88) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sales] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cs_quantity as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2))),count(1),sales,number_sales,sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #19 - WholeStageCodegen (91) + Exchange [i_brand_id,i_class_id,i_category_id] #18 + WholeStageCodegen (87) 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] @@ -218,33 +213,33 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [cs_sold_date_sk,d_date_sk] SortMergeJoin [cs_item_sk,ss_item_sk] InputAdapter - WholeStageCodegen (48) + WholeStageCodegen (46) Sort [cs_item_sk] InputAdapter - Exchange [cs_item_sk] #20 - WholeStageCodegen (47) + Exchange [cs_item_sk] #19 + WholeStageCodegen (45) Filter [cs_item_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter - WholeStageCodegen (67) + WholeStageCodegen (64) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #5 InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #16 - WholeStageCodegen (138) + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15 + WholeStageCodegen (132) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sales] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(ws_quantity as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2))),count(1),sales,number_sales,sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #21 - WholeStageCodegen (137) + Exchange [i_brand_id,i_class_id,i_category_id] #20 + WholeStageCodegen (131) 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,22 +247,22 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [ws_sold_date_sk,d_date_sk] SortMergeJoin [ws_item_sk,ss_item_sk] InputAdapter - WholeStageCodegen (94) + WholeStageCodegen (90) Sort [ws_item_sk] InputAdapter - Exchange [ws_item_sk] #22 - WholeStageCodegen (93) + Exchange [ws_item_sk] #21 + WholeStageCodegen (89) Filter [ws_item_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter - WholeStageCodegen (113) + WholeStageCodegen (108) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #5 InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #16 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt index b263d0f642e45..300cfd7ccbb21 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 @@ -1,111 +1,109 @@ == Physical Plan == -TakeOrderedAndProject (107) -+- * HashAggregate (106) - +- Exchange (105) - +- * HashAggregate (104) - +- * Expand (103) - +- Union (102) - :- * Project (69) - : +- * Filter (68) - : +- * HashAggregate (67) - : +- Exchange (66) - : +- * HashAggregate (65) - : +- * Project (64) - : +- * BroadcastHashJoin Inner BuildRight (63) - : :- * Project (61) - : : +- * BroadcastHashJoin Inner BuildRight (60) - : : :- * BroadcastHashJoin LeftSemi BuildRight (53) +TakeOrderedAndProject (105) ++- * HashAggregate (104) + +- Exchange (103) + +- * HashAggregate (102) + +- * Expand (101) + +- Union (100) + :- * Project (67) + : +- * Filter (66) + : +- * HashAggregate (65) + : +- Exchange (64) + : +- * HashAggregate (63) + : +- * Project (62) + : +- * BroadcastHashJoin Inner BuildRight (61) + : :- * Project (59) + : : +- * BroadcastHashJoin Inner BuildRight (58) + : : :- * BroadcastHashJoin LeftSemi BuildRight (51) : : : :- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (52) - : : : +- * Project (51) - : : : +- * BroadcastHashJoin Inner BuildRight (50) + : : : +- BroadcastExchange (50) + : : : +- * Project (49) + : : : +- * BroadcastHashJoin Inner BuildRight (48) : : : :- * Filter (6) : : : : +- * ColumnarToRow (5) : : : : +- Scan parquet default.item (4) - : : : +- BroadcastExchange (49) - : : : +- * HashAggregate (48) - : : : +- * HashAggregate (47) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) - : : : :- * HashAggregate (35) - : : : : +- Exchange (34) - : : : : +- * HashAggregate (33) - : : : : +- * Project (32) - : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * Filter (9) - : : : : : : +- * ColumnarToRow (8) - : : : : : : +- Scan parquet default.store_sales (7) - : : : : : +- BroadcastExchange (27) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet default.item (10) - : : : : : +- BroadcastExchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * 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) - : : : : : +- ReusedExchange (22) - : : : : +- ReusedExchange (30) - : : : +- BroadcastExchange (45) - : : : +- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * Project (41) - : : : : +- * BroadcastHashJoin Inner BuildRight (40) - : : : : :- * Filter (38) - : : : : : +- * ColumnarToRow (37) - : : : : : +- Scan parquet default.web_sales (36) - : : : : +- ReusedExchange (39) - : : : +- ReusedExchange (42) - : : +- BroadcastExchange (59) - : : +- * BroadcastHashJoin LeftSemi BuildRight (58) - : : :- * Filter (56) - : : : +- * ColumnarToRow (55) - : : : +- Scan parquet default.item (54) - : : +- ReusedExchange (57) - : +- ReusedExchange (62) - :- * Project (85) - : +- * Filter (84) - : +- * HashAggregate (83) - : +- Exchange (82) - : +- * HashAggregate (81) - : +- * Project (80) - : +- * BroadcastHashJoin Inner BuildRight (79) - : :- * Project (77) - : : +- * BroadcastHashJoin Inner BuildRight (76) - : : :- * BroadcastHashJoin LeftSemi BuildRight (74) - : : : :- * Filter (72) - : : : : +- * ColumnarToRow (71) - : : : : +- Scan parquet default.catalog_sales (70) - : : : +- ReusedExchange (73) - : : +- ReusedExchange (75) - : +- ReusedExchange (78) - +- * Project (101) - +- * Filter (100) - +- * HashAggregate (99) - +- Exchange (98) - +- * HashAggregate (97) - +- * Project (96) - +- * BroadcastHashJoin Inner BuildRight (95) - :- * Project (93) - : +- * BroadcastHashJoin Inner BuildRight (92) - : :- * BroadcastHashJoin LeftSemi BuildRight (90) - : : :- * Filter (88) - : : : +- * ColumnarToRow (87) - : : : +- Scan parquet default.web_sales (86) - : : +- ReusedExchange (89) - : +- ReusedExchange (91) - +- ReusedExchange (94) + : : : +- BroadcastExchange (47) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) + : : : :- * HashAggregate (35) + : : : : +- Exchange (34) + : : : : +- * HashAggregate (33) + : : : : +- * Project (32) + : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : :- * Project (29) + : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : :- * Filter (9) + : : : : : : +- * ColumnarToRow (8) + : : : : : : +- Scan parquet default.store_sales (7) + : : : : : +- BroadcastExchange (27) + : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) + : : : : : :- * Filter (12) + : : : : : : +- * ColumnarToRow (11) + : : : : : : +- Scan parquet default.item (10) + : : : : : +- BroadcastExchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * 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) + : : : : : +- ReusedExchange (22) + : : : : +- ReusedExchange (30) + : : : +- BroadcastExchange (45) + : : : +- * Project (44) + : : : +- * BroadcastHashJoin Inner BuildRight (43) + : : : :- * Project (41) + : : : : +- * BroadcastHashJoin Inner BuildRight (40) + : : : : :- * Filter (38) + : : : : : +- * ColumnarToRow (37) + : : : : : +- Scan parquet default.web_sales (36) + : : : : +- ReusedExchange (39) + : : : +- ReusedExchange (42) + : : +- BroadcastExchange (57) + : : +- * BroadcastHashJoin LeftSemi BuildRight (56) + : : :- * Filter (54) + : : : +- * ColumnarToRow (53) + : : : +- Scan parquet default.item (52) + : : +- ReusedExchange (55) + : +- ReusedExchange (60) + :- * Project (83) + : +- * Filter (82) + : +- * HashAggregate (81) + : +- Exchange (80) + : +- * HashAggregate (79) + : +- * Project (78) + : +- * BroadcastHashJoin Inner BuildRight (77) + : :- * Project (75) + : : +- * BroadcastHashJoin Inner BuildRight (74) + : : :- * BroadcastHashJoin LeftSemi BuildRight (72) + : : : :- * Filter (70) + : : : : +- * ColumnarToRow (69) + : : : : +- Scan parquet default.catalog_sales (68) + : : : +- ReusedExchange (71) + : : +- ReusedExchange (73) + : +- ReusedExchange (76) + +- * Project (99) + +- * Filter (98) + +- * HashAggregate (97) + +- Exchange (96) + +- * HashAggregate (95) + +- * Project (94) + +- * BroadcastHashJoin Inner BuildRight (93) + :- * Project (91) + : +- * BroadcastHashJoin Inner BuildRight (90) + : :- * BroadcastHashJoin LeftSemi BuildRight (88) + : : :- * Filter (86) + : : : +- * ColumnarToRow (85) + : : : +- Scan parquet default.web_sales (84) + : : +- ReusedExchange (87) + : +- ReusedExchange (89) + +- ReusedExchange (92) (1) Scan parquet default.store_sales @@ -208,7 +206,7 @@ Join condition: None 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) ReusedExchange [Reuses operator id: 136] +(22) ReusedExchange [Reuses operator id: 134] Output [1]: [d_date_sk#24] (23) BroadcastHashJoin [codegen id : 3] @@ -242,7 +240,7 @@ Join condition: None 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] -(30) ReusedExchange [Reuses operator id: 136] +(30) ReusedExchange [Reuses operator id: 134] Output [1]: [d_date_sk#27] (31) BroadcastHashJoin [codegen id : 6] @@ -299,7 +297,7 @@ Join condition: None 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] -(42) ReusedExchange [Reuses operator id: 136] +(42) ReusedExchange [Reuses operator id: 134] Output [1]: [d_date_sk#38] (43) BroadcastHashJoin [codegen id : 9] @@ -320,116 +318,102 @@ Left keys [6]: [coalesce(brand_id#28, 0), isnull(brand_id#28), coalesce(class_id 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 -(47) 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] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] - -(48) 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] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] - -(49) BroadcastExchange +(47) 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] -(50) BroadcastHashJoin [codegen id : 11] +(48) 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] Join condition: None -(51) Project [codegen id : 11] +(49) 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] -(52) BroadcastExchange +(50) BroadcastExchange Input [1]: [ss_item_sk#41] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] -(53) BroadcastHashJoin [codegen id : 25] +(51) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#41] Join condition: None -(54) Scan parquet default.item +(52) Scan parquet default.item Output [4]: [i_item_sk#43, i_brand_id#44, i_class_id#45, i_category_id#46] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(55) ColumnarToRow [codegen id : 23] +(53) ColumnarToRow [codegen id : 23] Input [4]: [i_item_sk#43, i_brand_id#44, i_class_id#45, i_category_id#46] -(56) Filter [codegen id : 23] +(54) Filter [codegen id : 23] Input [4]: [i_item_sk#43, i_brand_id#44, i_class_id#45, i_category_id#46] Condition : isnotnull(i_item_sk#43) -(57) ReusedExchange [Reuses operator id: 52] +(55) ReusedExchange [Reuses operator id: 50] Output [1]: [ss_item_sk#41] -(58) BroadcastHashJoin [codegen id : 23] +(56) BroadcastHashJoin [codegen id : 23] Left keys [1]: [i_item_sk#43] Right keys [1]: [ss_item_sk#41] Join condition: None -(59) BroadcastExchange +(57) BroadcastExchange Input [4]: [i_item_sk#43, i_brand_id#44, i_class_id#45, i_category_id#46] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#47] -(60) BroadcastHashJoin [codegen id : 25] +(58) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#43] Join condition: None -(61) Project [codegen id : 25] +(59) Project [codegen id : 25] Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#44, i_class_id#45, i_category_id#46] Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#43, i_brand_id#44, i_class_id#45, i_category_id#46] -(62) ReusedExchange [Reuses operator id: 131] +(60) ReusedExchange [Reuses operator id: 129] Output [1]: [d_date_sk#48] -(63) BroadcastHashJoin [codegen id : 25] +(61) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#48] Join condition: None -(64) Project [codegen id : 25] +(62) Project [codegen id : 25] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#44, i_class_id#45, i_category_id#46] Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#44, i_class_id#45, i_category_id#46, d_date_sk#48] -(65) HashAggregate [codegen id : 25] +(63) HashAggregate [codegen id : 25] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#44, i_class_id#45, i_category_id#46] Keys [3]: [i_brand_id#44, i_class_id#45, i_category_id#46] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)] Aggregate Attributes [3]: [sum#49, isEmpty#50, count#51] Results [6]: [i_brand_id#44, i_class_id#45, i_category_id#46, sum#52, isEmpty#53, count#54] -(66) Exchange +(64) Exchange Input [6]: [i_brand_id#44, i_class_id#45, i_category_id#46, sum#52, isEmpty#53, count#54] Arguments: hashpartitioning(i_brand_id#44, i_class_id#45, i_category_id#46, 5), ENSURE_REQUIREMENTS, [id=#55] -(67) HashAggregate [codegen id : 26] +(65) HashAggregate [codegen id : 26] Input [6]: [i_brand_id#44, i_class_id#45, i_category_id#46, sum#52, isEmpty#53, count#54] Keys [3]: [i_brand_id#44, i_class_id#45, i_category_id#46] Functions [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2))), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#56, count(1)#57] Results [5]: [i_brand_id#44, i_class_id#45, i_category_id#46, sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#56 AS sales#58, count(1)#57 AS number_sales#59] -(68) Filter [codegen id : 26] +(66) Filter [codegen id : 26] Input [5]: [i_brand_id#44, i_class_id#45, i_category_id#46, sales#58, number_sales#59] Condition : (isnotnull(sales#58) AND (cast(sales#58 as decimal(32,6)) > cast(Subquery scalar-subquery#60, [id=#61] as decimal(32,6)))) -(69) Project [codegen id : 26] +(67) Project [codegen id : 26] Output [6]: [sales#58, number_sales#59, store AS channel#62, i_brand_id#44, i_class_id#45, i_category_id#46] Input [5]: [i_brand_id#44, i_class_id#45, i_category_id#46, sales#58, number_sales#59] -(70) Scan parquet default.catalog_sales +(68) Scan parquet default.catalog_sales Output [4]: [cs_item_sk#63, cs_quantity#64, cs_list_price#65, cs_sold_date_sk#66] Batched: true Location: InMemoryFileIndex [] @@ -437,72 +421,72 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#66), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(71) ColumnarToRow [codegen id : 51] +(69) ColumnarToRow [codegen id : 51] Input [4]: [cs_item_sk#63, cs_quantity#64, cs_list_price#65, cs_sold_date_sk#66] -(72) Filter [codegen id : 51] +(70) Filter [codegen id : 51] Input [4]: [cs_item_sk#63, cs_quantity#64, cs_list_price#65, cs_sold_date_sk#66] Condition : isnotnull(cs_item_sk#63) -(73) ReusedExchange [Reuses operator id: 52] +(71) ReusedExchange [Reuses operator id: 50] Output [1]: [ss_item_sk#41] -(74) BroadcastHashJoin [codegen id : 51] +(72) BroadcastHashJoin [codegen id : 51] Left keys [1]: [cs_item_sk#63] Right keys [1]: [ss_item_sk#41] Join condition: None -(75) ReusedExchange [Reuses operator id: 59] +(73) ReusedExchange [Reuses operator id: 57] Output [4]: [i_item_sk#67, i_brand_id#68, i_class_id#69, i_category_id#70] -(76) BroadcastHashJoin [codegen id : 51] +(74) BroadcastHashJoin [codegen id : 51] Left keys [1]: [cs_item_sk#63] Right keys [1]: [i_item_sk#67] Join condition: None -(77) Project [codegen id : 51] +(75) Project [codegen id : 51] Output [6]: [cs_quantity#64, cs_list_price#65, cs_sold_date_sk#66, i_brand_id#68, i_class_id#69, i_category_id#70] Input [8]: [cs_item_sk#63, cs_quantity#64, cs_list_price#65, cs_sold_date_sk#66, i_item_sk#67, i_brand_id#68, i_class_id#69, i_category_id#70] -(78) ReusedExchange [Reuses operator id: 131] +(76) ReusedExchange [Reuses operator id: 129] Output [1]: [d_date_sk#71] -(79) BroadcastHashJoin [codegen id : 51] +(77) BroadcastHashJoin [codegen id : 51] Left keys [1]: [cs_sold_date_sk#66] Right keys [1]: [d_date_sk#71] Join condition: None -(80) Project [codegen id : 51] +(78) Project [codegen id : 51] Output [5]: [cs_quantity#64, cs_list_price#65, i_brand_id#68, i_class_id#69, i_category_id#70] Input [7]: [cs_quantity#64, cs_list_price#65, cs_sold_date_sk#66, i_brand_id#68, i_class_id#69, i_category_id#70, d_date_sk#71] -(81) HashAggregate [codegen id : 51] +(79) HashAggregate [codegen id : 51] Input [5]: [cs_quantity#64, cs_list_price#65, i_brand_id#68, i_class_id#69, i_category_id#70] Keys [3]: [i_brand_id#68, i_class_id#69, i_category_id#70] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cs_quantity#64 as decimal(12,2))) * promote_precision(cast(cs_list_price#65 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)] Aggregate Attributes [3]: [sum#72, isEmpty#73, count#74] Results [6]: [i_brand_id#68, i_class_id#69, i_category_id#70, sum#75, isEmpty#76, count#77] -(82) Exchange +(80) Exchange Input [6]: [i_brand_id#68, i_class_id#69, i_category_id#70, sum#75, isEmpty#76, count#77] Arguments: hashpartitioning(i_brand_id#68, i_class_id#69, i_category_id#70, 5), ENSURE_REQUIREMENTS, [id=#78] -(83) HashAggregate [codegen id : 52] +(81) HashAggregate [codegen id : 52] Input [6]: [i_brand_id#68, i_class_id#69, i_category_id#70, sum#75, isEmpty#76, count#77] Keys [3]: [i_brand_id#68, i_class_id#69, i_category_id#70] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cs_quantity#64 as decimal(12,2))) * promote_precision(cast(cs_list_price#65 as decimal(12,2)))), DecimalType(18,2))), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cs_quantity#64 as decimal(12,2))) * promote_precision(cast(cs_list_price#65 as decimal(12,2)))), DecimalType(18,2)))#79, count(1)#80] Results [5]: [i_brand_id#68, i_class_id#69, i_category_id#70, sum(CheckOverflow((promote_precision(cast(cs_quantity#64 as decimal(12,2))) * promote_precision(cast(cs_list_price#65 as decimal(12,2)))), DecimalType(18,2)))#79 AS sales#81, count(1)#80 AS number_sales#82] -(84) Filter [codegen id : 52] +(82) Filter [codegen id : 52] Input [5]: [i_brand_id#68, i_class_id#69, i_category_id#70, sales#81, number_sales#82] Condition : (isnotnull(sales#81) AND (cast(sales#81 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#60, [id=#61] as decimal(32,6)))) -(85) Project [codegen id : 52] +(83) Project [codegen id : 52] Output [6]: [sales#81, number_sales#82, catalog AS channel#83, i_brand_id#68, i_class_id#69, i_category_id#70] Input [5]: [i_brand_id#68, i_class_id#69, i_category_id#70, sales#81, number_sales#82] -(86) Scan parquet default.web_sales +(84) Scan parquet default.web_sales Output [4]: [ws_item_sk#84, ws_quantity#85, ws_list_price#86, ws_sold_date_sk#87] Batched: true Location: InMemoryFileIndex [] @@ -510,272 +494,272 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#87), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(87) ColumnarToRow [codegen id : 77] +(85) ColumnarToRow [codegen id : 77] Input [4]: [ws_item_sk#84, ws_quantity#85, ws_list_price#86, ws_sold_date_sk#87] -(88) Filter [codegen id : 77] +(86) Filter [codegen id : 77] Input [4]: [ws_item_sk#84, ws_quantity#85, ws_list_price#86, ws_sold_date_sk#87] Condition : isnotnull(ws_item_sk#84) -(89) ReusedExchange [Reuses operator id: 52] +(87) ReusedExchange [Reuses operator id: 50] Output [1]: [ss_item_sk#41] -(90) BroadcastHashJoin [codegen id : 77] +(88) BroadcastHashJoin [codegen id : 77] Left keys [1]: [ws_item_sk#84] Right keys [1]: [ss_item_sk#41] Join condition: None -(91) ReusedExchange [Reuses operator id: 59] +(89) ReusedExchange [Reuses operator id: 57] Output [4]: [i_item_sk#88, i_brand_id#89, i_class_id#90, i_category_id#91] -(92) BroadcastHashJoin [codegen id : 77] +(90) BroadcastHashJoin [codegen id : 77] Left keys [1]: [ws_item_sk#84] Right keys [1]: [i_item_sk#88] Join condition: None -(93) Project [codegen id : 77] +(91) Project [codegen id : 77] Output [6]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#87, i_brand_id#89, i_class_id#90, i_category_id#91] Input [8]: [ws_item_sk#84, ws_quantity#85, ws_list_price#86, ws_sold_date_sk#87, i_item_sk#88, i_brand_id#89, i_class_id#90, i_category_id#91] -(94) ReusedExchange [Reuses operator id: 131] +(92) ReusedExchange [Reuses operator id: 129] Output [1]: [d_date_sk#92] -(95) BroadcastHashJoin [codegen id : 77] +(93) BroadcastHashJoin [codegen id : 77] Left keys [1]: [ws_sold_date_sk#87] Right keys [1]: [d_date_sk#92] Join condition: None -(96) Project [codegen id : 77] +(94) Project [codegen id : 77] Output [5]: [ws_quantity#85, ws_list_price#86, i_brand_id#89, i_class_id#90, i_category_id#91] Input [7]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#87, i_brand_id#89, i_class_id#90, i_category_id#91, d_date_sk#92] -(97) HashAggregate [codegen id : 77] +(95) HashAggregate [codegen id : 77] Input [5]: [ws_quantity#85, ws_list_price#86, i_brand_id#89, i_class_id#90, i_category_id#91] Keys [3]: [i_brand_id#89, i_class_id#90, i_category_id#91] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(ws_quantity#85 as decimal(12,2))) * promote_precision(cast(ws_list_price#86 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)] Aggregate Attributes [3]: [sum#93, isEmpty#94, count#95] Results [6]: [i_brand_id#89, i_class_id#90, i_category_id#91, sum#96, isEmpty#97, count#98] -(98) Exchange +(96) Exchange Input [6]: [i_brand_id#89, i_class_id#90, i_category_id#91, sum#96, isEmpty#97, count#98] Arguments: hashpartitioning(i_brand_id#89, i_class_id#90, i_category_id#91, 5), ENSURE_REQUIREMENTS, [id=#99] -(99) HashAggregate [codegen id : 78] +(97) HashAggregate [codegen id : 78] Input [6]: [i_brand_id#89, i_class_id#90, i_category_id#91, sum#96, isEmpty#97, count#98] Keys [3]: [i_brand_id#89, i_class_id#90, i_category_id#91] Functions [2]: [sum(CheckOverflow((promote_precision(cast(ws_quantity#85 as decimal(12,2))) * promote_precision(cast(ws_list_price#86 as decimal(12,2)))), DecimalType(18,2))), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ws_quantity#85 as decimal(12,2))) * promote_precision(cast(ws_list_price#86 as decimal(12,2)))), DecimalType(18,2)))#100, count(1)#101] Results [5]: [i_brand_id#89, i_class_id#90, i_category_id#91, sum(CheckOverflow((promote_precision(cast(ws_quantity#85 as decimal(12,2))) * promote_precision(cast(ws_list_price#86 as decimal(12,2)))), DecimalType(18,2)))#100 AS sales#102, count(1)#101 AS number_sales#103] -(100) Filter [codegen id : 78] +(98) Filter [codegen id : 78] Input [5]: [i_brand_id#89, i_class_id#90, i_category_id#91, sales#102, number_sales#103] Condition : (isnotnull(sales#102) AND (cast(sales#102 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#60, [id=#61] as decimal(32,6)))) -(101) Project [codegen id : 78] +(99) Project [codegen id : 78] Output [6]: [sales#102, number_sales#103, web AS channel#104, i_brand_id#89, i_class_id#90, i_category_id#91] Input [5]: [i_brand_id#89, i_class_id#90, i_category_id#91, sales#102, number_sales#103] -(102) Union +(100) Union -(103) Expand [codegen id : 79] +(101) Expand [codegen id : 79] Input [6]: [sales#58, number_sales#59, channel#62, i_brand_id#44, i_class_id#45, i_category_id#46] Arguments: [[sales#58, number_sales#59, channel#62, i_brand_id#44, i_class_id#45, i_category_id#46, 0], [sales#58, number_sales#59, channel#62, i_brand_id#44, i_class_id#45, null, 1], [sales#58, number_sales#59, channel#62, i_brand_id#44, null, null, 3], [sales#58, number_sales#59, channel#62, null, null, null, 7], [sales#58, number_sales#59, null, null, null, null, 15]], [sales#58, number_sales#59, channel#105, i_brand_id#106, i_class_id#107, i_category_id#108, spark_grouping_id#109] -(104) HashAggregate [codegen id : 79] +(102) HashAggregate [codegen id : 79] Input [7]: [sales#58, number_sales#59, channel#105, i_brand_id#106, i_class_id#107, i_category_id#108, spark_grouping_id#109] Keys [5]: [channel#105, i_brand_id#106, i_class_id#107, i_category_id#108, spark_grouping_id#109] Functions [2]: [partial_sum(sales#58), partial_sum(number_sales#59)] Aggregate Attributes [3]: [sum#110, isEmpty#111, sum#112] Results [8]: [channel#105, i_brand_id#106, i_class_id#107, i_category_id#108, spark_grouping_id#109, sum#113, isEmpty#114, sum#115] -(105) Exchange +(103) Exchange Input [8]: [channel#105, i_brand_id#106, i_class_id#107, i_category_id#108, spark_grouping_id#109, sum#113, isEmpty#114, sum#115] Arguments: hashpartitioning(channel#105, i_brand_id#106, i_class_id#107, i_category_id#108, spark_grouping_id#109, 5), ENSURE_REQUIREMENTS, [id=#116] -(106) HashAggregate [codegen id : 80] +(104) HashAggregate [codegen id : 80] Input [8]: [channel#105, i_brand_id#106, i_class_id#107, i_category_id#108, spark_grouping_id#109, sum#113, isEmpty#114, sum#115] Keys [5]: [channel#105, i_brand_id#106, i_class_id#107, i_category_id#108, spark_grouping_id#109] Functions [2]: [sum(sales#58), sum(number_sales#59)] Aggregate Attributes [2]: [sum(sales#58)#117, sum(number_sales#59)#118] Results [6]: [channel#105, i_brand_id#106, i_class_id#107, i_category_id#108, sum(sales#58)#117 AS sum(sales)#119, sum(number_sales#59)#118 AS sum(number_sales)#120] -(107) TakeOrderedAndProject +(105) TakeOrderedAndProject Input [6]: [channel#105, i_brand_id#106, i_class_id#107, i_category_id#108, sum(sales)#119, sum(number_sales)#120] Arguments: 100, [channel#105 ASC NULLS FIRST, i_brand_id#106 ASC NULLS FIRST, i_class_id#107 ASC NULLS FIRST, i_category_id#108 ASC NULLS FIRST], [channel#105, i_brand_id#106, i_class_id#107, i_category_id#108, sum(sales)#119, sum(number_sales)#120] ===== Subqueries ===== -Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#60, [id=#61] -* HashAggregate (126) -+- Exchange (125) - +- * HashAggregate (124) - +- Union (123) - :- * Project (112) - : +- * BroadcastHashJoin Inner BuildRight (111) - : :- * ColumnarToRow (109) - : : +- Scan parquet default.store_sales (108) - : +- ReusedExchange (110) - :- * Project (117) - : +- * BroadcastHashJoin Inner BuildRight (116) - : :- * ColumnarToRow (114) - : : +- Scan parquet default.catalog_sales (113) - : +- ReusedExchange (115) - +- * Project (122) - +- * BroadcastHashJoin Inner BuildRight (121) - :- * ColumnarToRow (119) - : +- Scan parquet default.web_sales (118) - +- ReusedExchange (120) - - -(108) Scan parquet default.store_sales +Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#60, [id=#61] +* HashAggregate (124) ++- Exchange (123) + +- * HashAggregate (122) + +- Union (121) + :- * Project (110) + : +- * BroadcastHashJoin Inner BuildRight (109) + : :- * ColumnarToRow (107) + : : +- Scan parquet default.store_sales (106) + : +- ReusedExchange (108) + :- * Project (115) + : +- * BroadcastHashJoin Inner BuildRight (114) + : :- * ColumnarToRow (112) + : : +- Scan parquet default.catalog_sales (111) + : +- ReusedExchange (113) + +- * Project (120) + +- * BroadcastHashJoin Inner BuildRight (119) + :- * ColumnarToRow (117) + : +- Scan parquet default.web_sales (116) + +- ReusedExchange (118) + + +(106) Scan parquet default.store_sales Output [3]: [ss_quantity#121, ss_list_price#122, ss_sold_date_sk#123] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#123), dynamicpruningexpression(ss_sold_date_sk#123 IN dynamicpruning#12)] ReadSchema: struct -(109) ColumnarToRow [codegen id : 2] +(107) ColumnarToRow [codegen id : 2] Input [3]: [ss_quantity#121, ss_list_price#122, ss_sold_date_sk#123] -(110) ReusedExchange [Reuses operator id: 136] +(108) ReusedExchange [Reuses operator id: 134] Output [1]: [d_date_sk#124] -(111) BroadcastHashJoin [codegen id : 2] +(109) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#123] Right keys [1]: [d_date_sk#124] Join condition: None -(112) Project [codegen id : 2] +(110) Project [codegen id : 2] Output [2]: [ss_quantity#121 AS quantity#125, ss_list_price#122 AS list_price#126] Input [4]: [ss_quantity#121, ss_list_price#122, ss_sold_date_sk#123, d_date_sk#124] -(113) Scan parquet default.catalog_sales +(111) Scan parquet default.catalog_sales Output [3]: [cs_quantity#127, cs_list_price#128, cs_sold_date_sk#129] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#129), dynamicpruningexpression(cs_sold_date_sk#129 IN dynamicpruning#12)] ReadSchema: struct -(114) ColumnarToRow [codegen id : 4] +(112) ColumnarToRow [codegen id : 4] Input [3]: [cs_quantity#127, cs_list_price#128, cs_sold_date_sk#129] -(115) ReusedExchange [Reuses operator id: 136] +(113) ReusedExchange [Reuses operator id: 134] Output [1]: [d_date_sk#130] -(116) BroadcastHashJoin [codegen id : 4] +(114) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#129] Right keys [1]: [d_date_sk#130] Join condition: None -(117) Project [codegen id : 4] +(115) Project [codegen id : 4] Output [2]: [cs_quantity#127 AS quantity#131, cs_list_price#128 AS list_price#132] Input [4]: [cs_quantity#127, cs_list_price#128, cs_sold_date_sk#129, d_date_sk#130] -(118) Scan parquet default.web_sales +(116) Scan parquet default.web_sales Output [3]: [ws_quantity#133, ws_list_price#134, ws_sold_date_sk#135] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#135), dynamicpruningexpression(ws_sold_date_sk#135 IN dynamicpruning#12)] ReadSchema: struct -(119) ColumnarToRow [codegen id : 6] +(117) ColumnarToRow [codegen id : 6] Input [3]: [ws_quantity#133, ws_list_price#134, ws_sold_date_sk#135] -(120) ReusedExchange [Reuses operator id: 136] +(118) ReusedExchange [Reuses operator id: 134] Output [1]: [d_date_sk#136] -(121) BroadcastHashJoin [codegen id : 6] +(119) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#135] Right keys [1]: [d_date_sk#136] Join condition: None -(122) Project [codegen id : 6] +(120) Project [codegen id : 6] Output [2]: [ws_quantity#133 AS quantity#137, ws_list_price#134 AS list_price#138] Input [4]: [ws_quantity#133, ws_list_price#134, ws_sold_date_sk#135, d_date_sk#136] -(123) Union +(121) Union -(124) HashAggregate [codegen id : 7] +(122) HashAggregate [codegen id : 7] Input [2]: [quantity#125, list_price#126] Keys: [] Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(quantity#125 as decimal(12,2))) * promote_precision(cast(list_price#126 as decimal(12,2)))), DecimalType(18,2)))] Aggregate Attributes [2]: [sum#139, count#140] Results [2]: [sum#141, count#142] -(125) Exchange +(123) Exchange Input [2]: [sum#141, count#142] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#143] -(126) HashAggregate [codegen id : 8] +(124) HashAggregate [codegen id : 8] Input [2]: [sum#141, count#142] Keys: [] Functions [1]: [avg(CheckOverflow((promote_precision(cast(quantity#125 as decimal(12,2))) * promote_precision(cast(list_price#126 as decimal(12,2)))), DecimalType(18,2)))] Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(quantity#125 as decimal(12,2))) * promote_precision(cast(list_price#126 as decimal(12,2)))), DecimalType(18,2)))#144] Results [1]: [avg(CheckOverflow((promote_precision(cast(quantity#125 as decimal(12,2))) * promote_precision(cast(list_price#126 as decimal(12,2)))), DecimalType(18,2)))#144 AS average_sales#145] -Subquery:2 Hosting operator id = 108 Hosting Expression = ss_sold_date_sk#123 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#123 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 113 Hosting Expression = cs_sold_date_sk#129 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 111 Hosting Expression = cs_sold_date_sk#129 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 118 Hosting Expression = ws_sold_date_sk#135 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 116 Hosting Expression = ws_sold_date_sk#135 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (131) -+- * Project (130) - +- * Filter (129) - +- * ColumnarToRow (128) - +- Scan parquet default.date_dim (127) +BroadcastExchange (129) ++- * Project (128) + +- * Filter (127) + +- * ColumnarToRow (126) + +- Scan parquet default.date_dim (125) -(127) Scan parquet default.date_dim +(125) Scan parquet default.date_dim Output [3]: [d_date_sk#48, d_year#146, d_moy#147] 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 -(128) ColumnarToRow [codegen id : 1] +(126) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#48, d_year#146, d_moy#147] -(129) Filter [codegen id : 1] +(127) Filter [codegen id : 1] Input [3]: [d_date_sk#48, d_year#146, d_moy#147] Condition : ((((isnotnull(d_year#146) AND isnotnull(d_moy#147)) AND (d_year#146 = 2001)) AND (d_moy#147 = 11)) AND isnotnull(d_date_sk#48)) -(130) Project [codegen id : 1] +(128) Project [codegen id : 1] Output [1]: [d_date_sk#48] Input [3]: [d_date_sk#48, d_year#146, d_moy#147] -(131) BroadcastExchange +(129) BroadcastExchange Input [1]: [d_date_sk#48] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#148] Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (136) -+- * Project (135) - +- * Filter (134) - +- * ColumnarToRow (133) - +- Scan parquet default.date_dim (132) +BroadcastExchange (134) ++- * Project (133) + +- * Filter (132) + +- * ColumnarToRow (131) + +- Scan parquet default.date_dim (130) -(132) Scan parquet default.date_dim +(130) Scan parquet default.date_dim Output [2]: [d_date_sk#27, d_year#149] 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 -(133) ColumnarToRow [codegen id : 1] +(131) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#27, d_year#149] -(134) Filter [codegen id : 1] +(132) Filter [codegen id : 1] Input [2]: [d_date_sk#27, d_year#149] Condition : (((isnotnull(d_year#149) AND (d_year#149 >= 1999)) AND (d_year#149 <= 2001)) AND isnotnull(d_date_sk#27)) -(135) Project [codegen id : 1] +(133) Project [codegen id : 1] Output [1]: [d_date_sk#27] Input [2]: [d_date_sk#27, d_year#149] -(136) BroadcastExchange +(134) BroadcastExchange Input [1]: [d_date_sk#27] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#150] @@ -783,12 +767,12 @@ Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN d Subquery:8 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 84 Hosting Expression = ReusedSubquery Subquery scalar-subquery#60, [id=#61] +Subquery:9 Hosting operator id = 82 Hosting Expression = ReusedSubquery Subquery scalar-subquery#60, [id=#61] -Subquery:10 Hosting operator id = 70 Hosting Expression = cs_sold_date_sk#66 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 68 Hosting Expression = cs_sold_date_sk#66 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 100 Hosting Expression = ReusedSubquery Subquery scalar-subquery#60, [id=#61] +Subquery:11 Hosting operator id = 98 Hosting Expression = ReusedSubquery Subquery scalar-subquery#60, [id=#61] -Subquery:12 Hosting operator id = 86 Hosting Expression = ws_sold_date_sk#87 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 84 Hosting Expression = ws_sold_date_sk#87 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 653e3e6564e41..b8125b2af8e92 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 @@ -81,77 +81,75 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter BroadcastExchange #5 WholeStageCodegen (10) - HashAggregate [brand_id,class_id,category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] HashAggregate [brand_id,class_id,category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #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 - BroadcastExchange #7 - WholeStageCodegen (1) - 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 (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 #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 #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 - ReusedExchange [d_date_sk] #7 - InputAdapter - ReusedExchange [d_date_sk] #7 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (9) + InputAdapter + 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 [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] + 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.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + 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 #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 #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 + ReusedExchange [d_date_sk] #7 InputAdapter ReusedExchange [d_date_sk] #7 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + InputAdapter + ReusedExchange [d_date_sk] #7 InputAdapter BroadcastExchange #12 WholeStageCodegen (23) 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 78133a44c9a69..3f0acc0ea73be 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt @@ -1,106 +1,103 @@ == Physical Plan == -TakeOrderedAndProject (102) -+- * BroadcastHashJoin Inner BuildRight (101) - :- * Filter (81) - : +- * HashAggregate (80) - : +- Exchange (79) - : +- * HashAggregate (78) - : +- * Project (77) - : +- * BroadcastHashJoin Inner BuildRight (76) - : :- * Project (66) - : : +- * BroadcastHashJoin Inner BuildRight (65) - : : :- * SortMergeJoin LeftSemi (63) +TakeOrderedAndProject (99) ++- * BroadcastHashJoin Inner BuildRight (98) + :- * Filter (78) + : +- * HashAggregate (77) + : +- Exchange (76) + : +- * HashAggregate (75) + : +- * Project (74) + : +- * BroadcastHashJoin Inner BuildRight (73) + : :- * Project (63) + : : +- * BroadcastHashJoin Inner BuildRight (62) + : : :- * SortMergeJoin LeftSemi (60) : : : :- * Sort (5) : : : : +- Exchange (4) : : : : +- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.store_sales (1) - : : : +- * Sort (62) - : : : +- Exchange (61) - : : : +- * Project (60) - : : : +- * BroadcastHashJoin Inner BuildRight (59) + : : : +- * Sort (59) + : : : +- Exchange (58) + : : : +- * Project (57) + : : : +- * BroadcastHashJoin Inner BuildRight (56) : : : :- * Filter (8) : : : : +- * ColumnarToRow (7) : : : : +- Scan parquet default.item (6) - : : : +- BroadcastExchange (58) - : : : +- * HashAggregate (57) - : : : +- Exchange (56) - : : : +- * HashAggregate (55) - : : : +- * SortMergeJoin LeftSemi (54) - : : : :- * Sort (42) - : : : : +- Exchange (41) - : : : : +- * HashAggregate (40) - : : : : +- Exchange (39) - : : : : +- * HashAggregate (38) - : : : : +- * Project (37) - : : : : +- * BroadcastHashJoin Inner BuildRight (36) - : : : : :- * Project (14) - : : : : : +- * BroadcastHashJoin Inner BuildRight (13) - : : : : : :- * Filter (11) - : : : : : : +- * ColumnarToRow (10) - : : : : : : +- Scan parquet default.store_sales (9) - : : : : : +- ReusedExchange (12) - : : : : +- BroadcastExchange (35) - : : : : +- * SortMergeJoin LeftSemi (34) - : : : : :- * Sort (19) - : : : : : +- Exchange (18) - : : : : : +- * Filter (17) - : : : : : +- * ColumnarToRow (16) - : : : : : +- Scan parquet default.item (15) - : : : : +- * Sort (33) - : : : : +- Exchange (32) - : : : : +- * Project (31) - : : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : : :- * Project (25) - : : : : : +- * BroadcastHashJoin Inner BuildRight (24) - : : : : : :- * Filter (22) - : : : : : : +- * ColumnarToRow (21) - : : : : : : +- Scan parquet default.catalog_sales (20) - : : : : : +- ReusedExchange (23) - : : : : +- BroadcastExchange (29) - : : : : +- * Filter (28) - : : : : +- * ColumnarToRow (27) - : : : : +- Scan parquet default.item (26) - : : : +- * Sort (53) - : : : +- Exchange (52) - : : : +- * Project (51) - : : : +- * BroadcastHashJoin Inner BuildRight (50) - : : : :- * Project (48) - : : : : +- * BroadcastHashJoin Inner BuildRight (47) - : : : : :- * Filter (45) - : : : : : +- * ColumnarToRow (44) - : : : : : +- Scan parquet default.web_sales (43) - : : : : +- ReusedExchange (46) - : : : +- ReusedExchange (49) - : : +- ReusedExchange (64) - : +- BroadcastExchange (75) - : +- * SortMergeJoin LeftSemi (74) - : :- * Sort (71) - : : +- Exchange (70) - : : +- * Filter (69) - : : +- * ColumnarToRow (68) - : : +- Scan parquet default.item (67) - : +- * Sort (73) - : +- ReusedExchange (72) - +- BroadcastExchange (100) - +- * Filter (99) - +- * HashAggregate (98) - +- Exchange (97) - +- * HashAggregate (96) - +- * Project (95) - +- * BroadcastHashJoin Inner BuildRight (94) - :- * Project (92) - : +- * BroadcastHashJoin Inner BuildRight (91) - : :- * SortMergeJoin LeftSemi (89) - : : :- * Sort (86) - : : : +- Exchange (85) - : : : +- * Filter (84) - : : : +- * ColumnarToRow (83) - : : : +- Scan parquet default.store_sales (82) - : : +- * Sort (88) - : : +- ReusedExchange (87) - : +- ReusedExchange (90) - +- ReusedExchange (93) + : : : +- BroadcastExchange (55) + : : : +- * SortMergeJoin LeftSemi (54) + : : : :- * Sort (42) + : : : : +- Exchange (41) + : : : : +- * HashAggregate (40) + : : : : +- Exchange (39) + : : : : +- * HashAggregate (38) + : : : : +- * Project (37) + : : : : +- * BroadcastHashJoin Inner BuildRight (36) + : : : : :- * Project (14) + : : : : : +- * BroadcastHashJoin Inner BuildRight (13) + : : : : : :- * Filter (11) + : : : : : : +- * ColumnarToRow (10) + : : : : : : +- Scan parquet default.store_sales (9) + : : : : : +- ReusedExchange (12) + : : : : +- BroadcastExchange (35) + : : : : +- * SortMergeJoin LeftSemi (34) + : : : : :- * Sort (19) + : : : : : +- Exchange (18) + : : : : : +- * Filter (17) + : : : : : +- * ColumnarToRow (16) + : : : : : +- Scan parquet default.item (15) + : : : : +- * Sort (33) + : : : : +- Exchange (32) + : : : : +- * Project (31) + : : : : +- * BroadcastHashJoin Inner BuildRight (30) + : : : : :- * Project (25) + : : : : : +- * BroadcastHashJoin Inner BuildRight (24) + : : : : : :- * Filter (22) + : : : : : : +- * ColumnarToRow (21) + : : : : : : +- Scan parquet default.catalog_sales (20) + : : : : : +- ReusedExchange (23) + : : : : +- BroadcastExchange (29) + : : : : +- * Filter (28) + : : : : +- * ColumnarToRow (27) + : : : : +- Scan parquet default.item (26) + : : : +- * Sort (53) + : : : +- Exchange (52) + : : : +- * Project (51) + : : : +- * BroadcastHashJoin Inner BuildRight (50) + : : : :- * Project (48) + : : : : +- * BroadcastHashJoin Inner BuildRight (47) + : : : : :- * Filter (45) + : : : : : +- * ColumnarToRow (44) + : : : : : +- Scan parquet default.web_sales (43) + : : : : +- ReusedExchange (46) + : : : +- ReusedExchange (49) + : : +- ReusedExchange (61) + : +- BroadcastExchange (72) + : +- * SortMergeJoin LeftSemi (71) + : :- * Sort (68) + : : +- Exchange (67) + : : +- * Filter (66) + : : +- * ColumnarToRow (65) + : : +- Scan parquet default.item (64) + : +- * Sort (70) + : +- ReusedExchange (69) + +- BroadcastExchange (97) + +- * Filter (96) + +- * HashAggregate (95) + +- Exchange (94) + +- * HashAggregate (93) + +- * Project (92) + +- * BroadcastHashJoin Inner BuildRight (91) + :- * Project (89) + : +- * BroadcastHashJoin Inner BuildRight (88) + : :- * SortMergeJoin LeftSemi (86) + : : :- * Sort (83) + : : : +- Exchange (82) + : : : +- * Filter (81) + : : : +- * ColumnarToRow (80) + : : : +- Scan parquet default.store_sales (79) + : : +- * Sort (85) + : : +- ReusedExchange (84) + : +- ReusedExchange (87) + +- ReusedExchange (90) (1) Scan parquet default.store_sales @@ -133,10 +130,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)) @@ -155,7 +152,7 @@ Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] Condition : isnotnull(ss_item_sk#11) -(12) ReusedExchange [Reuses operator id: 135] +(12) ReusedExchange [Reuses operator id: 132] Output [1]: [d_date_sk#14] (13) BroadcastHashJoin [codegen id : 11] @@ -204,7 +201,7 @@ Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] Condition : isnotnull(cs_item_sk#20) -(23) ReusedExchange [Reuses operator id: 135] +(23) ReusedExchange [Reuses operator id: 132] Output [1]: [d_date_sk#22] (24) BroadcastHashJoin [codegen id : 8] @@ -310,7 +307,7 @@ Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] Condition : isnotnull(ws_item_sk#35) -(46) ReusedExchange [Reuses operator id: 135] +(46) ReusedExchange [Reuses operator id: 132] Output [1]: [d_date_sk#37] (47) BroadcastHashJoin [codegen id : 16] @@ -347,485 +344,467 @@ Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id 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 -(55) HashAggregate [codegen id : 18] +(55) BroadcastExchange 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#30, class_id#31, category_id#32] - -(56) 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] - -(57) HashAggregate [codegen id : 19] -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#30, class_id#31, category_id#32] - -(58) 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] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#43] -(59) BroadcastHashJoin [codegen id : 20] +(56) 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] Join condition: None -(60) Project [codegen id : 20] -Output [1]: [i_item_sk#7 AS ss_item_sk#45] +(57) Project [codegen id : 19] +Output [1]: [i_item_sk#7 AS ss_item_sk#44] 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] -(61) Exchange -Input [1]: [ss_item_sk#45] -Arguments: hashpartitioning(ss_item_sk#45, 5), ENSURE_REQUIREMENTS, [id=#46] +(58) Exchange +Input [1]: [ss_item_sk#44] +Arguments: hashpartitioning(ss_item_sk#44, 5), ENSURE_REQUIREMENTS, [id=#45] -(62) Sort [codegen id : 21] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +(59) Sort [codegen id : 20] +Input [1]: [ss_item_sk#44] +Arguments: [ss_item_sk#44 ASC NULLS FIRST], false, 0 -(63) SortMergeJoin [codegen id : 45] +(60) SortMergeJoin [codegen id : 43] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#45] +Right keys [1]: [ss_item_sk#44] Join condition: None -(64) ReusedExchange [Reuses operator id: 126] -Output [1]: [d_date_sk#47] +(61) ReusedExchange [Reuses operator id: 123] +Output [1]: [d_date_sk#46] -(65) BroadcastHashJoin [codegen id : 45] +(62) BroadcastHashJoin [codegen id : 43] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#47] +Right keys [1]: [d_date_sk#46] Join condition: None -(66) Project [codegen id : 45] +(63) Project [codegen id : 43] 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#47] +Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#46] -(67) Scan parquet default.item -Output [4]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51] +(64) Scan parquet default.item +Output [4]: [i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] 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 -(68) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51] +(65) ColumnarToRow [codegen id : 22] +Input [4]: [i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] -(69) Filter [codegen id : 23] -Input [4]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51] -Condition : (((isnotnull(i_item_sk#48) AND isnotnull(i_brand_id#49)) AND isnotnull(i_class_id#50)) AND isnotnull(i_category_id#51)) +(66) Filter [codegen id : 22] +Input [4]: [i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] +Condition : (((isnotnull(i_item_sk#47) AND isnotnull(i_brand_id#48)) AND isnotnull(i_class_id#49)) AND isnotnull(i_category_id#50)) -(70) Exchange -Input [4]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51] -Arguments: hashpartitioning(i_item_sk#48, 5), ENSURE_REQUIREMENTS, [id=#52] +(67) Exchange +Input [4]: [i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] +Arguments: hashpartitioning(i_item_sk#47, 5), ENSURE_REQUIREMENTS, [id=#51] -(71) Sort [codegen id : 24] -Input [4]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51] -Arguments: [i_item_sk#48 ASC NULLS FIRST], false, 0 +(68) Sort [codegen id : 23] +Input [4]: [i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] +Arguments: [i_item_sk#47 ASC NULLS FIRST], false, 0 -(72) ReusedExchange [Reuses operator id: 61] -Output [1]: [ss_item_sk#45] +(69) ReusedExchange [Reuses operator id: 58] +Output [1]: [ss_item_sk#44] -(73) Sort [codegen id : 43] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +(70) Sort [codegen id : 41] +Input [1]: [ss_item_sk#44] +Arguments: [ss_item_sk#44 ASC NULLS FIRST], false, 0 -(74) SortMergeJoin [codegen id : 44] -Left keys [1]: [i_item_sk#48] -Right keys [1]: [ss_item_sk#45] +(71) SortMergeJoin [codegen id : 42] +Left keys [1]: [i_item_sk#47] +Right keys [1]: [ss_item_sk#44] Join condition: None -(75) BroadcastExchange -Input [4]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#53] +(72) BroadcastExchange +Input [4]: [i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#52] -(76) BroadcastHashJoin [codegen id : 45] +(73) BroadcastHashJoin [codegen id : 43] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#48] +Right keys [1]: [i_item_sk#47] Join condition: None -(77) Project [codegen id : 45] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#49, i_class_id#50, i_category_id#51] -Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51] +(74) Project [codegen id : 43] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#48, i_class_id#49, i_category_id#50] +Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] -(78) HashAggregate [codegen id : 45] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#49, i_class_id#50, i_category_id#51] -Keys [3]: [i_brand_id#49, i_class_id#50, i_category_id#51] +(75) HashAggregate [codegen id : 43] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#48, i_class_id#49, i_category_id#50] +Keys [3]: [i_brand_id#48, i_class_id#49, i_category_id#50] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)] -Aggregate Attributes [3]: [sum#54, isEmpty#55, count#56] -Results [6]: [i_brand_id#49, i_class_id#50, i_category_id#51, sum#57, isEmpty#58, count#59] +Aggregate Attributes [3]: [sum#53, isEmpty#54, count#55] +Results [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum#56, isEmpty#57, count#58] -(79) Exchange -Input [6]: [i_brand_id#49, i_class_id#50, i_category_id#51, sum#57, isEmpty#58, count#59] -Arguments: hashpartitioning(i_brand_id#49, i_class_id#50, i_category_id#51, 5), ENSURE_REQUIREMENTS, [id=#60] +(76) Exchange +Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum#56, isEmpty#57, count#58] +Arguments: hashpartitioning(i_brand_id#48, i_class_id#49, i_category_id#50, 5), ENSURE_REQUIREMENTS, [id=#59] -(80) HashAggregate [codegen id : 92] -Input [6]: [i_brand_id#49, i_class_id#50, i_category_id#51, sum#57, isEmpty#58, count#59] -Keys [3]: [i_brand_id#49, i_class_id#50, i_category_id#51] +(77) HashAggregate [codegen id : 88] +Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum#56, isEmpty#57, count#58] +Keys [3]: [i_brand_id#48, i_class_id#49, i_category_id#50] Functions [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2))), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#61, count(1)#62] -Results [6]: [store AS channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#61 AS sales#64, count(1)#62 AS number_sales#65] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#60, count(1)#61] +Results [6]: [store AS channel#62, i_brand_id#48, i_class_id#49, i_category_id#50, sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#60 AS sales#63, count(1)#61 AS number_sales#64] -(81) Filter [codegen id : 92] -Input [6]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sales#64, number_sales#65] -Condition : (isnotnull(sales#64) AND (cast(sales#64 as decimal(32,6)) > cast(Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +(78) Filter [codegen id : 88] +Input [6]: [channel#62, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64] +Condition : (isnotnull(sales#63) AND (cast(sales#63 as decimal(32,6)) > cast(Subquery scalar-subquery#65, [id=#66] as decimal(32,6)))) -(82) Scan parquet default.store_sales -Output [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] +(79) Scan parquet default.store_sales +Output [4]: [ss_item_sk#67, ss_quantity#68, ss_list_price#69, ss_sold_date_sk#70] 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#70), dynamicpruningexpression(ss_sold_date_sk#70 IN dynamicpruning#71)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(83) ColumnarToRow [codegen id : 46] -Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] +(80) ColumnarToRow [codegen id : 44] +Input [4]: [ss_item_sk#67, ss_quantity#68, ss_list_price#69, ss_sold_date_sk#70] -(84) Filter [codegen id : 46] -Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] -Condition : isnotnull(ss_item_sk#68) +(81) Filter [codegen id : 44] +Input [4]: [ss_item_sk#67, ss_quantity#68, ss_list_price#69, ss_sold_date_sk#70] +Condition : isnotnull(ss_item_sk#67) -(85) 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] +(82) Exchange +Input [4]: [ss_item_sk#67, ss_quantity#68, ss_list_price#69, ss_sold_date_sk#70] +Arguments: hashpartitioning(ss_item_sk#67, 5), ENSURE_REQUIREMENTS, [id=#72] -(86) Sort [codegen id : 47] -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 +(83) Sort [codegen id : 45] +Input [4]: [ss_item_sk#67, ss_quantity#68, ss_list_price#69, ss_sold_date_sk#70] +Arguments: [ss_item_sk#67 ASC NULLS FIRST], false, 0 -(87) ReusedExchange [Reuses operator id: 61] -Output [1]: [ss_item_sk#45] +(84) ReusedExchange [Reuses operator id: 58] +Output [1]: [ss_item_sk#44] -(88) Sort [codegen id : 66] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +(85) Sort [codegen id : 63] +Input [1]: [ss_item_sk#44] +Arguments: [ss_item_sk#44 ASC NULLS FIRST], false, 0 -(89) SortMergeJoin [codegen id : 90] -Left keys [1]: [ss_item_sk#68] -Right keys [1]: [ss_item_sk#45] +(86) SortMergeJoin [codegen id : 86] +Left keys [1]: [ss_item_sk#67] +Right keys [1]: [ss_item_sk#44] Join condition: None -(90) ReusedExchange [Reuses operator id: 140] -Output [1]: [d_date_sk#74] +(87) ReusedExchange [Reuses operator id: 137] +Output [1]: [d_date_sk#73] -(91) BroadcastHashJoin [codegen id : 90] -Left keys [1]: [ss_sold_date_sk#71] -Right keys [1]: [d_date_sk#74] +(88) BroadcastHashJoin [codegen id : 86] +Left keys [1]: [ss_sold_date_sk#70] +Right keys [1]: [d_date_sk#73] Join condition: None -(92) Project [codegen id : 90] -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] +(89) Project [codegen id : 86] +Output [3]: [ss_item_sk#67, ss_quantity#68, ss_list_price#69] +Input [5]: [ss_item_sk#67, ss_quantity#68, ss_list_price#69, ss_sold_date_sk#70, d_date_sk#73] -(93) ReusedExchange [Reuses operator id: 75] -Output [4]: [i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] +(90) ReusedExchange [Reuses operator id: 72] +Output [4]: [i_item_sk#74, i_brand_id#75, i_class_id#76, i_category_id#77] -(94) BroadcastHashJoin [codegen id : 90] -Left keys [1]: [ss_item_sk#68] -Right keys [1]: [i_item_sk#75] +(91) BroadcastHashJoin [codegen id : 86] +Left keys [1]: [ss_item_sk#67] +Right keys [1]: [i_item_sk#74] Join condition: None -(95) Project [codegen id : 90] -Output [5]: [ss_quantity#69, ss_list_price#70, i_brand_id#76, i_class_id#77, i_category_id#78] -Input [7]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] - -(96) HashAggregate [codegen id : 90] -Input [5]: [ss_quantity#69, ss_list_price#70, 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(ss_quantity#69 as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)] -Aggregate Attributes [3]: [sum#79, isEmpty#80, count#81] -Results [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#82, isEmpty#83, count#84] - -(97) Exchange -Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#82, isEmpty#83, count#84] -Arguments: hashpartitioning(i_brand_id#76, i_class_id#77, i_category_id#78, 5), ENSURE_REQUIREMENTS, [id=#85] - -(98) HashAggregate [codegen id : 91] -Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#82, isEmpty#83, count#84] -Keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#69 as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2))), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#69 as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2)))#86, count(1)#87] -Results [6]: [store AS channel#88, i_brand_id#76, i_class_id#77, i_category_id#78, sum(CheckOverflow((promote_precision(cast(ss_quantity#69 as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2)))#86 AS sales#89, count(1)#87 AS number_sales#90] - -(99) Filter [codegen id : 91] -Input [6]: [channel#88, i_brand_id#76, i_class_id#77, i_category_id#78, sales#89, number_sales#90] -Condition : (isnotnull(sales#89) AND (cast(sales#89 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) - -(100) BroadcastExchange -Input [6]: [channel#88, i_brand_id#76, i_class_id#77, i_category_id#78, sales#89, number_sales#90] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#91] - -(101) BroadcastHashJoin [codegen id : 92] -Left keys [3]: [i_brand_id#49, i_class_id#50, i_category_id#51] -Right keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] +(92) Project [codegen id : 86] +Output [5]: [ss_quantity#68, ss_list_price#69, i_brand_id#75, i_class_id#76, i_category_id#77] +Input [7]: [ss_item_sk#67, ss_quantity#68, ss_list_price#69, i_item_sk#74, i_brand_id#75, i_class_id#76, i_category_id#77] + +(93) HashAggregate [codegen id : 86] +Input [5]: [ss_quantity#68, ss_list_price#69, i_brand_id#75, i_class_id#76, i_category_id#77] +Keys [3]: [i_brand_id#75, i_class_id#76, i_category_id#77] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(ss_quantity#68 as decimal(12,2))) * promote_precision(cast(ss_list_price#69 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)] +Aggregate Attributes [3]: [sum#78, isEmpty#79, count#80] +Results [6]: [i_brand_id#75, i_class_id#76, i_category_id#77, sum#81, isEmpty#82, count#83] + +(94) Exchange +Input [6]: [i_brand_id#75, i_class_id#76, i_category_id#77, sum#81, isEmpty#82, count#83] +Arguments: hashpartitioning(i_brand_id#75, i_class_id#76, i_category_id#77, 5), ENSURE_REQUIREMENTS, [id=#84] + +(95) HashAggregate [codegen id : 87] +Input [6]: [i_brand_id#75, i_class_id#76, i_category_id#77, sum#81, isEmpty#82, count#83] +Keys [3]: [i_brand_id#75, i_class_id#76, i_category_id#77] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#68 as decimal(12,2))) * promote_precision(cast(ss_list_price#69 as decimal(12,2)))), DecimalType(18,2))), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#68 as decimal(12,2))) * promote_precision(cast(ss_list_price#69 as decimal(12,2)))), DecimalType(18,2)))#85, count(1)#86] +Results [6]: [store AS channel#87, i_brand_id#75, i_class_id#76, i_category_id#77, sum(CheckOverflow((promote_precision(cast(ss_quantity#68 as decimal(12,2))) * promote_precision(cast(ss_list_price#69 as decimal(12,2)))), DecimalType(18,2)))#85 AS sales#88, count(1)#86 AS number_sales#89] + +(96) Filter [codegen id : 87] +Input [6]: [channel#87, i_brand_id#75, i_class_id#76, i_category_id#77, sales#88, number_sales#89] +Condition : (isnotnull(sales#88) AND (cast(sales#88 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#65, [id=#66] as decimal(32,6)))) + +(97) BroadcastExchange +Input [6]: [channel#87, i_brand_id#75, i_class_id#76, i_category_id#77, sales#88, number_sales#89] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#90] + +(98) BroadcastHashJoin [codegen id : 88] +Left keys [3]: [i_brand_id#48, i_class_id#49, i_category_id#50] +Right keys [3]: [i_brand_id#75, i_class_id#76, i_category_id#77] Join condition: None -(102) TakeOrderedAndProject -Input [12]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sales#64, number_sales#65, channel#88, i_brand_id#76, i_class_id#77, i_category_id#78, sales#89, number_sales#90] -Arguments: 100, [i_brand_id#49 ASC NULLS FIRST, i_class_id#50 ASC NULLS FIRST, i_category_id#51 ASC NULLS FIRST], [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sales#64, number_sales#65, channel#88, i_brand_id#76, i_class_id#77, i_category_id#78, sales#89, number_sales#90] +(99) TakeOrderedAndProject +Input [12]: [channel#62, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64, channel#87, i_brand_id#75, i_class_id#76, i_category_id#77, sales#88, number_sales#89] +Arguments: 100, [i_brand_id#48 ASC NULLS FIRST, i_class_id#49 ASC NULLS FIRST, i_category_id#50 ASC NULLS FIRST], [channel#62, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64, channel#87, i_brand_id#75, i_class_id#76, i_category_id#77, sales#88, number_sales#89] ===== Subqueries ===== -Subquery:1 Hosting operator id = 81 Hosting Expression = Subquery scalar-subquery#66, [id=#67] -* HashAggregate (121) -+- Exchange (120) - +- * HashAggregate (119) - +- Union (118) - :- * Project (107) - : +- * BroadcastHashJoin Inner BuildRight (106) - : :- * ColumnarToRow (104) - : : +- Scan parquet default.store_sales (103) - : +- ReusedExchange (105) - :- * Project (112) - : +- * BroadcastHashJoin Inner BuildRight (111) - : :- * ColumnarToRow (109) - : : +- Scan parquet default.catalog_sales (108) - : +- ReusedExchange (110) - +- * Project (117) - +- * BroadcastHashJoin Inner BuildRight (116) - :- * ColumnarToRow (114) - : +- Scan parquet default.web_sales (113) - +- ReusedExchange (115) - - -(103) Scan parquet default.store_sales -Output [3]: [ss_quantity#92, ss_list_price#93, ss_sold_date_sk#94] +Subquery:1 Hosting operator id = 78 Hosting Expression = Subquery scalar-subquery#65, [id=#66] +* HashAggregate (118) ++- Exchange (117) + +- * HashAggregate (116) + +- Union (115) + :- * Project (104) + : +- * BroadcastHashJoin Inner BuildRight (103) + : :- * ColumnarToRow (101) + : : +- Scan parquet default.store_sales (100) + : +- ReusedExchange (102) + :- * Project (109) + : +- * BroadcastHashJoin Inner BuildRight (108) + : :- * ColumnarToRow (106) + : : +- Scan parquet default.catalog_sales (105) + : +- ReusedExchange (107) + +- * Project (114) + +- * BroadcastHashJoin Inner BuildRight (113) + :- * ColumnarToRow (111) + : +- Scan parquet default.web_sales (110) + +- ReusedExchange (112) + + +(100) Scan parquet default.store_sales +Output [3]: [ss_quantity#91, ss_list_price#92, ss_sold_date_sk#93] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#94), dynamicpruningexpression(ss_sold_date_sk#94 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ss_sold_date_sk#93), dynamicpruningexpression(ss_sold_date_sk#93 IN dynamicpruning#13)] ReadSchema: struct -(104) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#92, ss_list_price#93, ss_sold_date_sk#94] +(101) ColumnarToRow [codegen id : 2] +Input [3]: [ss_quantity#91, ss_list_price#92, ss_sold_date_sk#93] -(105) ReusedExchange [Reuses operator id: 135] -Output [1]: [d_date_sk#95] +(102) ReusedExchange [Reuses operator id: 132] +Output [1]: [d_date_sk#94] -(106) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#94] -Right keys [1]: [d_date_sk#95] +(103) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#93] +Right keys [1]: [d_date_sk#94] Join condition: None -(107) Project [codegen id : 2] -Output [2]: [ss_quantity#92 AS quantity#96, ss_list_price#93 AS list_price#97] -Input [4]: [ss_quantity#92, ss_list_price#93, ss_sold_date_sk#94, d_date_sk#95] +(104) Project [codegen id : 2] +Output [2]: [ss_quantity#91 AS quantity#95, ss_list_price#92 AS list_price#96] +Input [4]: [ss_quantity#91, ss_list_price#92, ss_sold_date_sk#93, d_date_sk#94] -(108) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#98, cs_list_price#99, cs_sold_date_sk#100] +(105) Scan parquet default.catalog_sales +Output [3]: [cs_quantity#97, cs_list_price#98, cs_sold_date_sk#99] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#100), dynamicpruningexpression(cs_sold_date_sk#100 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(cs_sold_date_sk#99), dynamicpruningexpression(cs_sold_date_sk#99 IN dynamicpruning#13)] ReadSchema: struct -(109) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#98, cs_list_price#99, cs_sold_date_sk#100] +(106) ColumnarToRow [codegen id : 4] +Input [3]: [cs_quantity#97, cs_list_price#98, cs_sold_date_sk#99] -(110) ReusedExchange [Reuses operator id: 135] -Output [1]: [d_date_sk#101] +(107) ReusedExchange [Reuses operator id: 132] +Output [1]: [d_date_sk#100] -(111) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#100] -Right keys [1]: [d_date_sk#101] +(108) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_sold_date_sk#99] +Right keys [1]: [d_date_sk#100] Join condition: None -(112) Project [codegen id : 4] -Output [2]: [cs_quantity#98 AS quantity#102, cs_list_price#99 AS list_price#103] -Input [4]: [cs_quantity#98, cs_list_price#99, cs_sold_date_sk#100, d_date_sk#101] +(109) Project [codegen id : 4] +Output [2]: [cs_quantity#97 AS quantity#101, cs_list_price#98 AS list_price#102] +Input [4]: [cs_quantity#97, cs_list_price#98, cs_sold_date_sk#99, d_date_sk#100] -(113) Scan parquet default.web_sales -Output [3]: [ws_quantity#104, ws_list_price#105, ws_sold_date_sk#106] +(110) Scan parquet default.web_sales +Output [3]: [ws_quantity#103, ws_list_price#104, ws_sold_date_sk#105] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#106), dynamicpruningexpression(ws_sold_date_sk#106 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ws_sold_date_sk#105), dynamicpruningexpression(ws_sold_date_sk#105 IN dynamicpruning#13)] ReadSchema: struct -(114) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#104, ws_list_price#105, ws_sold_date_sk#106] +(111) ColumnarToRow [codegen id : 6] +Input [3]: [ws_quantity#103, ws_list_price#104, ws_sold_date_sk#105] -(115) ReusedExchange [Reuses operator id: 135] -Output [1]: [d_date_sk#107] +(112) ReusedExchange [Reuses operator id: 132] +Output [1]: [d_date_sk#106] -(116) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#106] -Right keys [1]: [d_date_sk#107] +(113) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#105] +Right keys [1]: [d_date_sk#106] Join condition: None -(117) Project [codegen id : 6] -Output [2]: [ws_quantity#104 AS quantity#108, ws_list_price#105 AS list_price#109] -Input [4]: [ws_quantity#104, ws_list_price#105, ws_sold_date_sk#106, d_date_sk#107] +(114) Project [codegen id : 6] +Output [2]: [ws_quantity#103 AS quantity#107, ws_list_price#104 AS list_price#108] +Input [4]: [ws_quantity#103, ws_list_price#104, ws_sold_date_sk#105, d_date_sk#106] -(118) Union +(115) Union -(119) HashAggregate [codegen id : 7] -Input [2]: [quantity#96, list_price#97] +(116) HashAggregate [codegen id : 7] +Input [2]: [quantity#95, list_price#96] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(quantity#96 as decimal(12,2))) * promote_precision(cast(list_price#97 as decimal(12,2)))), DecimalType(18,2)))] -Aggregate Attributes [2]: [sum#110, count#111] -Results [2]: [sum#112, count#113] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(quantity#95 as decimal(12,2))) * promote_precision(cast(list_price#96 as decimal(12,2)))), DecimalType(18,2)))] +Aggregate Attributes [2]: [sum#109, count#110] +Results [2]: [sum#111, count#112] -(120) Exchange -Input [2]: [sum#112, count#113] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#114] +(117) Exchange +Input [2]: [sum#111, count#112] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#113] -(121) HashAggregate [codegen id : 8] -Input [2]: [sum#112, count#113] +(118) HashAggregate [codegen id : 8] +Input [2]: [sum#111, count#112] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(quantity#96 as decimal(12,2))) * promote_precision(cast(list_price#97 as decimal(12,2)))), DecimalType(18,2)))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(quantity#96 as decimal(12,2))) * promote_precision(cast(list_price#97 as decimal(12,2)))), DecimalType(18,2)))#115] -Results [1]: [avg(CheckOverflow((promote_precision(cast(quantity#96 as decimal(12,2))) * promote_precision(cast(list_price#97 as decimal(12,2)))), DecimalType(18,2)))#115 AS average_sales#116] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(quantity#95 as decimal(12,2))) * promote_precision(cast(list_price#96 as decimal(12,2)))), DecimalType(18,2)))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(quantity#95 as decimal(12,2))) * promote_precision(cast(list_price#96 as decimal(12,2)))), DecimalType(18,2)))#114] +Results [1]: [avg(CheckOverflow((promote_precision(cast(quantity#95 as decimal(12,2))) * promote_precision(cast(list_price#96 as decimal(12,2)))), DecimalType(18,2)))#114 AS average_sales#115] -Subquery:2 Hosting operator id = 103 Hosting Expression = ss_sold_date_sk#94 IN dynamicpruning#13 +Subquery:2 Hosting operator id = 100 Hosting Expression = ss_sold_date_sk#93 IN dynamicpruning#13 -Subquery:3 Hosting operator id = 108 Hosting Expression = cs_sold_date_sk#100 IN dynamicpruning#13 +Subquery:3 Hosting operator id = 105 Hosting Expression = cs_sold_date_sk#99 IN dynamicpruning#13 -Subquery:4 Hosting operator id = 113 Hosting Expression = ws_sold_date_sk#106 IN dynamicpruning#13 +Subquery:4 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#105 IN dynamicpruning#13 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (126) -+- * Project (125) - +- * Filter (124) - +- * ColumnarToRow (123) - +- Scan parquet default.date_dim (122) +BroadcastExchange (123) ++- * Project (122) + +- * Filter (121) + +- * ColumnarToRow (120) + +- Scan parquet default.date_dim (119) -(122) Scan parquet default.date_dim -Output [2]: [d_date_sk#47, d_week_seq#117] +(119) Scan parquet default.date_dim +Output [2]: [d_date_sk#46, d_week_seq#116] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(123) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#47, d_week_seq#117] +(120) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#46, d_week_seq#116] -(124) Filter [codegen id : 1] -Input [2]: [d_date_sk#47, d_week_seq#117] -Condition : ((isnotnull(d_week_seq#117) AND (d_week_seq#117 = Subquery scalar-subquery#118, [id=#119])) AND isnotnull(d_date_sk#47)) +(121) Filter [codegen id : 1] +Input [2]: [d_date_sk#46, d_week_seq#116] +Condition : ((isnotnull(d_week_seq#116) AND (d_week_seq#116 = Subquery scalar-subquery#117, [id=#118])) AND isnotnull(d_date_sk#46)) -(125) Project [codegen id : 1] -Output [1]: [d_date_sk#47] -Input [2]: [d_date_sk#47, d_week_seq#117] +(122) Project [codegen id : 1] +Output [1]: [d_date_sk#46] +Input [2]: [d_date_sk#46, d_week_seq#116] -(126) BroadcastExchange -Input [1]: [d_date_sk#47] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#120] +(123) BroadcastExchange +Input [1]: [d_date_sk#46] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#119] -Subquery:6 Hosting operator id = 124 Hosting Expression = Subquery scalar-subquery#118, [id=#119] -* Project (130) -+- * Filter (129) - +- * ColumnarToRow (128) - +- Scan parquet default.date_dim (127) +Subquery:6 Hosting operator id = 121 Hosting Expression = Subquery scalar-subquery#117, [id=#118] +* Project (127) ++- * Filter (126) + +- * ColumnarToRow (125) + +- Scan parquet default.date_dim (124) -(127) Scan parquet default.date_dim -Output [4]: [d_week_seq#121, d_year#122, d_moy#123, d_dom#124] +(124) Scan parquet default.date_dim +Output [4]: [d_week_seq#120, d_year#121, d_moy#122, d_dom#123] 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#121, d_year#122, d_moy#123, d_dom#124] +(125) ColumnarToRow [codegen id : 1] +Input [4]: [d_week_seq#120, d_year#121, d_moy#122, d_dom#123] -(129) Filter [codegen id : 1] -Input [4]: [d_week_seq#121, d_year#122, d_moy#123, d_dom#124] -Condition : (((((isnotnull(d_year#122) AND isnotnull(d_moy#123)) AND isnotnull(d_dom#124)) AND (d_year#122 = 2000)) AND (d_moy#123 = 12)) AND (d_dom#124 = 11)) +(126) Filter [codegen id : 1] +Input [4]: [d_week_seq#120, d_year#121, d_moy#122, d_dom#123] +Condition : (((((isnotnull(d_year#121) AND isnotnull(d_moy#122)) AND isnotnull(d_dom#123)) AND (d_year#121 = 2000)) AND (d_moy#122 = 12)) AND (d_dom#123 = 11)) -(130) Project [codegen id : 1] -Output [1]: [d_week_seq#121] -Input [4]: [d_week_seq#121, d_year#122, d_moy#123, d_dom#124] +(127) Project [codegen id : 1] +Output [1]: [d_week_seq#120] +Input [4]: [d_week_seq#120, d_year#121, d_moy#122, d_dom#123] Subquery:7 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (135) -+- * Project (134) - +- * Filter (133) - +- * ColumnarToRow (132) - +- Scan parquet default.date_dim (131) +BroadcastExchange (132) ++- * Project (131) + +- * Filter (130) + +- * ColumnarToRow (129) + +- Scan parquet default.date_dim (128) -(131) Scan parquet default.date_dim -Output [2]: [d_date_sk#14, d_year#125] +(128) Scan parquet default.date_dim +Output [2]: [d_date_sk#14, d_year#124] 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 -(132) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#14, d_year#125] +(129) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#14, d_year#124] -(133) Filter [codegen id : 1] -Input [2]: [d_date_sk#14, d_year#125] -Condition : (((isnotnull(d_year#125) AND (d_year#125 >= 1999)) AND (d_year#125 <= 2001)) AND isnotnull(d_date_sk#14)) +(130) Filter [codegen id : 1] +Input [2]: [d_date_sk#14, d_year#124] +Condition : (((isnotnull(d_year#124) AND (d_year#124 >= 1999)) AND (d_year#124 <= 2001)) AND isnotnull(d_date_sk#14)) -(134) Project [codegen id : 1] +(131) Project [codegen id : 1] Output [1]: [d_date_sk#14] -Input [2]: [d_date_sk#14, d_year#125] +Input [2]: [d_date_sk#14, d_year#124] -(135) BroadcastExchange +(132) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#126] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#125] Subquery:8 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#13 Subquery:9 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#13 -Subquery:10 Hosting operator id = 99 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#65, [id=#66] -Subquery:11 Hosting operator id = 82 Hosting Expression = ss_sold_date_sk#71 IN dynamicpruning#72 -BroadcastExchange (140) -+- * Project (139) - +- * Filter (138) - +- * ColumnarToRow (137) - +- Scan parquet default.date_dim (136) +Subquery:11 Hosting operator id = 79 Hosting Expression = ss_sold_date_sk#70 IN dynamicpruning#71 +BroadcastExchange (137) ++- * Project (136) + +- * Filter (135) + +- * ColumnarToRow (134) + +- Scan parquet default.date_dim (133) -(136) Scan parquet default.date_dim -Output [2]: [d_date_sk#74, d_week_seq#127] +(133) Scan parquet default.date_dim +Output [2]: [d_date_sk#73, d_week_seq#126] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(137) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#74, d_week_seq#127] +(134) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#73, d_week_seq#126] -(138) Filter [codegen id : 1] -Input [2]: [d_date_sk#74, d_week_seq#127] -Condition : ((isnotnull(d_week_seq#127) AND (d_week_seq#127 = Subquery scalar-subquery#128, [id=#129])) AND isnotnull(d_date_sk#74)) +(135) Filter [codegen id : 1] +Input [2]: [d_date_sk#73, d_week_seq#126] +Condition : ((isnotnull(d_week_seq#126) AND (d_week_seq#126 = Subquery scalar-subquery#127, [id=#128])) AND isnotnull(d_date_sk#73)) -(139) Project [codegen id : 1] -Output [1]: [d_date_sk#74] -Input [2]: [d_date_sk#74, d_week_seq#127] +(136) Project [codegen id : 1] +Output [1]: [d_date_sk#73] +Input [2]: [d_date_sk#73, d_week_seq#126] -(140) BroadcastExchange -Input [1]: [d_date_sk#74] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#130] +(137) BroadcastExchange +Input [1]: [d_date_sk#73] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#129] -Subquery:12 Hosting operator id = 138 Hosting Expression = Subquery scalar-subquery#128, [id=#129] -* Project (144) -+- * Filter (143) - +- * ColumnarToRow (142) - +- Scan parquet default.date_dim (141) +Subquery:12 Hosting operator id = 135 Hosting Expression = Subquery scalar-subquery#127, [id=#128] +* Project (141) ++- * Filter (140) + +- * ColumnarToRow (139) + +- Scan parquet default.date_dim (138) -(141) Scan parquet default.date_dim -Output [4]: [d_week_seq#131, d_year#132, d_moy#133, d_dom#134] +(138) Scan parquet default.date_dim +Output [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct -(142) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#131, d_year#132, d_moy#133, d_dom#134] +(139) ColumnarToRow [codegen id : 1] +Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] -(143) Filter [codegen id : 1] -Input [4]: [d_week_seq#131, d_year#132, d_moy#133, d_dom#134] -Condition : (((((isnotnull(d_year#132) AND isnotnull(d_moy#133)) AND isnotnull(d_dom#134)) AND (d_year#132 = 1999)) AND (d_moy#133 = 12)) AND (d_dom#134 = 11)) +(140) 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)) -(144) Project [codegen id : 1] -Output [1]: [d_week_seq#131] -Input [4]: [d_week_seq#131, d_year#132, d_moy#133, d_dom#134] +(141) 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] 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 e7d3f84db0c72..82e338515f431 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,12 +1,12 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (92) + WholeStageCodegen (88) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [sales] Subquery #4 WholeStageCodegen (8) HashAggregate [sum,count] [avg(CheckOverflow((promote_precision(cast(quantity as decimal(12,2))) * promote_precision(cast(list_price as decimal(12,2)))), DecimalType(18,2))),average_sales,sum,count] InputAdapter - Exchange #17 + Exchange #16 WholeStageCodegen (7) HashAggregate [quantity,list_price] [sum,count,sum,count] InputAdapter @@ -19,7 +19,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #3 InputAdapter - ReusedExchange [d_date_sk] #9 + ReusedExchange [d_date_sk] #8 WholeStageCodegen (4) Project [cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] @@ -28,7 +28,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 InputAdapter - ReusedExchange [d_date_sk] #9 + ReusedExchange [d_date_sk] #8 WholeStageCodegen (6) Project [ws_quantity,ws_list_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] @@ -37,11 +37,11 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 InputAdapter - ReusedExchange [d_date_sk] #9 + ReusedExchange [d_date_sk] #8 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(ss_quantity as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2))),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (45) + WholeStageCodegen (43) 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] @@ -74,11 +74,11 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter Scan parquet default.date_dim [d_date_sk,d_week_seq] InputAdapter - 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] @@ -87,129 +87,124 @@ 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) - HashAggregate [brand_id,class_id,category_id] + WholeStageCodegen (18) + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] InputAdapter - Exchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (18) - HashAggregate [brand_id,class_id,category_id] - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - InputAdapter - WholeStageCodegen (13) - Sort [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (12) - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #8 - WholeStageCodegen (11) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #9 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] + WholeStageCodegen (13) + Sort [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #6 + WholeStageCodegen (12) + HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #7 + 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] #3 + BroadcastExchange #8 + WholeStageCodegen (1) + 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 #9 + WholeStageCodegen (10) + SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (5) + Sort [i_brand_id,i_class_id,i_category_id] InputAdapter - ReusedExchange [d_date_sk] #9 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (10) - SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - InputAdapter - WholeStageCodegen (5) - Sort [i_brand_id,i_class_id,i_category_id] + Exchange [i_brand_id,i_class_id,i_category_id] #10 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow 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] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (9) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #11 + 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] #3 + InputAdapter + ReusedExchange [d_date_sk] #8 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (7) + Filter [i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - WholeStageCodegen (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] #3 - InputAdapter - ReusedExchange [d_date_sk] #9 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (7) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - WholeStageCodegen (17) - Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (17) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #13 + WholeStageCodegen (16) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #8 InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (16) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #9 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #15 - WholeStageCodegen (44) + BroadcastExchange #14 + WholeStageCodegen (42) SortMergeJoin [i_item_sk,ss_item_sk] InputAdapter - WholeStageCodegen (24) + WholeStageCodegen (23) Sort [i_item_sk] InputAdapter - Exchange [i_item_sk] #16 - WholeStageCodegen (23) + Exchange [i_item_sk] #15 + 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] InputAdapter - WholeStageCodegen (43) + WholeStageCodegen (41) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #4 InputAdapter - BroadcastExchange #18 - WholeStageCodegen (91) + BroadcastExchange #17 + WholeStageCodegen (87) Filter [sales] ReusedSubquery [average_sales] #4 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(ss_quantity as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2))),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #19 - WholeStageCodegen (90) + Exchange [i_brand_id,i_class_id,i_category_id] #18 + WholeStageCodegen (86) 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] @@ -217,17 +212,17 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_sold_date_sk,d_date_sk] SortMergeJoin [ss_item_sk,ss_item_sk] InputAdapter - WholeStageCodegen (47) + WholeStageCodegen (45) Sort [ss_item_sk] InputAdapter - Exchange [ss_item_sk] #20 - WholeStageCodegen (46) + Exchange [ss_item_sk] #19 + WholeStageCodegen (44) 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] #5 - BroadcastExchange #21 + BroadcastExchange #20 WholeStageCodegen (1) Project [d_date_sk] Filter [d_week_seq,d_date_sk] @@ -242,11 +237,11 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter Scan parquet default.date_dim [d_date_sk,d_week_seq] InputAdapter - WholeStageCodegen (66) + WholeStageCodegen (63) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #4 InputAdapter - ReusedExchange [d_date_sk] #21 + ReusedExchange [d_date_sk] #20 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt index b0fe619430132..69be776d2ac28 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt @@ -1,90 +1,88 @@ == Physical Plan == -TakeOrderedAndProject (86) -+- * BroadcastHashJoin Inner BuildRight (85) - :- * Filter (68) - : +- * HashAggregate (67) - : +- Exchange (66) - : +- * HashAggregate (65) - : +- * Project (64) - : +- * BroadcastHashJoin Inner BuildRight (63) - : :- * Project (61) - : : +- * BroadcastHashJoin Inner BuildRight (60) - : : :- * BroadcastHashJoin LeftSemi BuildRight (53) +TakeOrderedAndProject (84) ++- * BroadcastHashJoin Inner BuildRight (83) + :- * Filter (66) + : +- * HashAggregate (65) + : +- Exchange (64) + : +- * HashAggregate (63) + : +- * Project (62) + : +- * BroadcastHashJoin Inner BuildRight (61) + : :- * Project (59) + : : +- * BroadcastHashJoin Inner BuildRight (58) + : : :- * BroadcastHashJoin LeftSemi BuildRight (51) : : : :- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (52) - : : : +- * Project (51) - : : : +- * BroadcastHashJoin Inner BuildRight (50) + : : : +- BroadcastExchange (50) + : : : +- * Project (49) + : : : +- * BroadcastHashJoin Inner BuildRight (48) : : : :- * Filter (6) : : : : +- * ColumnarToRow (5) : : : : +- Scan parquet default.item (4) - : : : +- BroadcastExchange (49) - : : : +- * HashAggregate (48) - : : : +- * HashAggregate (47) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) - : : : :- * HashAggregate (35) - : : : : +- Exchange (34) - : : : : +- * HashAggregate (33) - : : : : +- * Project (32) - : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * Filter (9) - : : : : : : +- * ColumnarToRow (8) - : : : : : : +- Scan parquet default.store_sales (7) - : : : : : +- BroadcastExchange (27) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet default.item (10) - : : : : : +- BroadcastExchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * 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) - : : : : : +- ReusedExchange (22) - : : : : +- ReusedExchange (30) - : : : +- BroadcastExchange (45) - : : : +- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * Project (41) - : : : : +- * BroadcastHashJoin Inner BuildRight (40) - : : : : :- * Filter (38) - : : : : : +- * ColumnarToRow (37) - : : : : : +- Scan parquet default.web_sales (36) - : : : : +- ReusedExchange (39) - : : : +- ReusedExchange (42) - : : +- BroadcastExchange (59) - : : +- * BroadcastHashJoin LeftSemi BuildRight (58) - : : :- * Filter (56) - : : : +- * ColumnarToRow (55) - : : : +- Scan parquet default.item (54) - : : +- ReusedExchange (57) - : +- ReusedExchange (62) - +- BroadcastExchange (84) - +- * Filter (83) - +- * HashAggregate (82) - +- Exchange (81) - +- * HashAggregate (80) - +- * Project (79) - +- * BroadcastHashJoin Inner BuildRight (78) - :- * Project (76) - : +- * BroadcastHashJoin Inner BuildRight (75) - : :- * BroadcastHashJoin LeftSemi BuildRight (73) - : : :- * Filter (71) - : : : +- * ColumnarToRow (70) - : : : +- Scan parquet default.store_sales (69) - : : +- ReusedExchange (72) - : +- ReusedExchange (74) - +- ReusedExchange (77) + : : : +- BroadcastExchange (47) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) + : : : :- * HashAggregate (35) + : : : : +- Exchange (34) + : : : : +- * HashAggregate (33) + : : : : +- * Project (32) + : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : :- * Project (29) + : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : :- * Filter (9) + : : : : : : +- * ColumnarToRow (8) + : : : : : : +- Scan parquet default.store_sales (7) + : : : : : +- BroadcastExchange (27) + : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) + : : : : : :- * Filter (12) + : : : : : : +- * ColumnarToRow (11) + : : : : : : +- Scan parquet default.item (10) + : : : : : +- BroadcastExchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * 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) + : : : : : +- ReusedExchange (22) + : : : : +- ReusedExchange (30) + : : : +- BroadcastExchange (45) + : : : +- * Project (44) + : : : +- * BroadcastHashJoin Inner BuildRight (43) + : : : :- * Project (41) + : : : : +- * BroadcastHashJoin Inner BuildRight (40) + : : : : :- * Filter (38) + : : : : : +- * ColumnarToRow (37) + : : : : : +- Scan parquet default.web_sales (36) + : : : : +- ReusedExchange (39) + : : : +- ReusedExchange (42) + : : +- BroadcastExchange (57) + : : +- * BroadcastHashJoin LeftSemi BuildRight (56) + : : :- * Filter (54) + : : : +- * ColumnarToRow (53) + : : : +- Scan parquet default.item (52) + : : +- ReusedExchange (55) + : +- ReusedExchange (60) + +- BroadcastExchange (82) + +- * Filter (81) + +- * HashAggregate (80) + +- Exchange (79) + +- * HashAggregate (78) + +- * Project (77) + +- * BroadcastHashJoin Inner BuildRight (76) + :- * Project (74) + : +- * BroadcastHashJoin Inner BuildRight (73) + : :- * BroadcastHashJoin LeftSemi BuildRight (71) + : : :- * Filter (69) + : : : +- * ColumnarToRow (68) + : : : +- Scan parquet default.store_sales (67) + : : +- ReusedExchange (70) + : +- ReusedExchange (72) + +- ReusedExchange (75) (1) Scan parquet default.store_sales @@ -187,7 +185,7 @@ Join condition: None 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) ReusedExchange [Reuses operator id: 119] +(22) ReusedExchange [Reuses operator id: 117] Output [1]: [d_date_sk#24] (23) BroadcastHashJoin [codegen id : 3] @@ -221,7 +219,7 @@ Join condition: None 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] -(30) ReusedExchange [Reuses operator id: 119] +(30) ReusedExchange [Reuses operator id: 117] Output [1]: [d_date_sk#27] (31) BroadcastHashJoin [codegen id : 6] @@ -278,7 +276,7 @@ Join condition: None 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] -(42) ReusedExchange [Reuses operator id: 119] +(42) ReusedExchange [Reuses operator id: 117] Output [1]: [d_date_sk#38] (43) BroadcastHashJoin [codegen id : 9] @@ -299,112 +297,98 @@ Left keys [6]: [coalesce(brand_id#28, 0), isnull(brand_id#28), coalesce(class_id 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 -(47) 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] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] - -(48) 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] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] - -(49) BroadcastExchange +(47) 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] -(50) BroadcastHashJoin [codegen id : 11] +(48) 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] Join condition: None -(51) Project [codegen id : 11] +(49) 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] -(52) BroadcastExchange +(50) BroadcastExchange Input [1]: [ss_item_sk#41] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] -(53) BroadcastHashJoin [codegen id : 25] +(51) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#41] Join condition: None -(54) Scan parquet default.item +(52) Scan parquet default.item Output [4]: [i_item_sk#43, i_brand_id#44, i_class_id#45, i_category_id#46] 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 -(55) ColumnarToRow [codegen id : 23] +(53) ColumnarToRow [codegen id : 23] Input [4]: [i_item_sk#43, i_brand_id#44, i_class_id#45, i_category_id#46] -(56) Filter [codegen id : 23] +(54) Filter [codegen id : 23] Input [4]: [i_item_sk#43, i_brand_id#44, i_class_id#45, i_category_id#46] Condition : (((isnotnull(i_item_sk#43) AND isnotnull(i_brand_id#44)) AND isnotnull(i_class_id#45)) AND isnotnull(i_category_id#46)) -(57) ReusedExchange [Reuses operator id: 52] +(55) ReusedExchange [Reuses operator id: 50] Output [1]: [ss_item_sk#41] -(58) BroadcastHashJoin [codegen id : 23] +(56) BroadcastHashJoin [codegen id : 23] Left keys [1]: [i_item_sk#43] Right keys [1]: [ss_item_sk#41] Join condition: None -(59) BroadcastExchange +(57) BroadcastExchange Input [4]: [i_item_sk#43, i_brand_id#44, i_class_id#45, i_category_id#46] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#47] -(60) BroadcastHashJoin [codegen id : 25] +(58) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#43] Join condition: None -(61) Project [codegen id : 25] +(59) Project [codegen id : 25] Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#44, i_class_id#45, i_category_id#46] Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#43, i_brand_id#44, i_class_id#45, i_category_id#46] -(62) ReusedExchange [Reuses operator id: 110] +(60) ReusedExchange [Reuses operator id: 108] Output [1]: [d_date_sk#48] -(63) BroadcastHashJoin [codegen id : 25] +(61) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#48] Join condition: None -(64) Project [codegen id : 25] +(62) Project [codegen id : 25] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#44, i_class_id#45, i_category_id#46] Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#44, i_class_id#45, i_category_id#46, d_date_sk#48] -(65) HashAggregate [codegen id : 25] +(63) HashAggregate [codegen id : 25] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#44, i_class_id#45, i_category_id#46] Keys [3]: [i_brand_id#44, i_class_id#45, i_category_id#46] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)] Aggregate Attributes [3]: [sum#49, isEmpty#50, count#51] Results [6]: [i_brand_id#44, i_class_id#45, i_category_id#46, sum#52, isEmpty#53, count#54] -(66) Exchange +(64) Exchange Input [6]: [i_brand_id#44, i_class_id#45, i_category_id#46, sum#52, isEmpty#53, count#54] Arguments: hashpartitioning(i_brand_id#44, i_class_id#45, i_category_id#46, 5), ENSURE_REQUIREMENTS, [id=#55] -(67) HashAggregate [codegen id : 52] +(65) HashAggregate [codegen id : 52] Input [6]: [i_brand_id#44, i_class_id#45, i_category_id#46, sum#52, isEmpty#53, count#54] Keys [3]: [i_brand_id#44, i_class_id#45, i_category_id#46] Functions [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2))), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#56, count(1)#57] Results [6]: [store AS channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#56 AS sales#59, count(1)#57 AS number_sales#60] -(68) Filter [codegen id : 52] +(66) Filter [codegen id : 52] Input [6]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sales#59, number_sales#60] Condition : (isnotnull(sales#59) AND (cast(sales#59 as decimal(32,6)) > cast(Subquery scalar-subquery#61, [id=#62] as decimal(32,6)))) -(69) Scan parquet default.store_sales +(67) Scan parquet default.store_sales Output [4]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66] Batched: true Location: InMemoryFileIndex [] @@ -412,278 +396,278 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#66), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(70) ColumnarToRow [codegen id : 50] +(68) ColumnarToRow [codegen id : 50] Input [4]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66] -(71) Filter [codegen id : 50] +(69) 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) -(72) ReusedExchange [Reuses operator id: 52] +(70) ReusedExchange [Reuses operator id: 50] Output [1]: [ss_item_sk#41] -(73) BroadcastHashJoin [codegen id : 50] +(71) BroadcastHashJoin [codegen id : 50] Left keys [1]: [ss_item_sk#63] Right keys [1]: [ss_item_sk#41] Join condition: None -(74) ReusedExchange [Reuses operator id: 59] +(72) ReusedExchange [Reuses operator id: 57] Output [4]: [i_item_sk#68, i_brand_id#69, i_class_id#70, i_category_id#71] -(75) BroadcastHashJoin [codegen id : 50] +(73) BroadcastHashJoin [codegen id : 50] Left keys [1]: [ss_item_sk#63] Right keys [1]: [i_item_sk#68] Join condition: None -(76) Project [codegen id : 50] +(74) 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] -(77) ReusedExchange [Reuses operator id: 124] +(75) ReusedExchange [Reuses operator id: 122] Output [1]: [d_date_sk#72] -(78) BroadcastHashJoin [codegen id : 50] +(76) BroadcastHashJoin [codegen id : 50] Left keys [1]: [ss_sold_date_sk#66] Right keys [1]: [d_date_sk#72] Join condition: None -(79) Project [codegen id : 50] +(77) 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] -(80) HashAggregate [codegen id : 50] +(78) 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(ss_quantity#64 as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)] Aggregate Attributes [3]: [sum#73, isEmpty#74, count#75] Results [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#76, isEmpty#77, count#78] -(81) Exchange +(79) Exchange Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#76, isEmpty#77, count#78] Arguments: hashpartitioning(i_brand_id#69, i_class_id#70, i_category_id#71, 5), ENSURE_REQUIREMENTS, [id=#79] -(82) HashAggregate [codegen id : 51] +(80) HashAggregate [codegen id : 51] Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#76, isEmpty#77, count#78] Keys [3]: [i_brand_id#69, i_class_id#70, i_category_id#71] Functions [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#64 as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2))), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#64 as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2)))#80, count(1)#81] Results [6]: [store AS channel#82, i_brand_id#69, i_class_id#70, i_category_id#71, sum(CheckOverflow((promote_precision(cast(ss_quantity#64 as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2)))#80 AS sales#83, count(1)#81 AS number_sales#84] -(83) Filter [codegen id : 51] +(81) Filter [codegen id : 51] Input [6]: [channel#82, i_brand_id#69, i_class_id#70, i_category_id#71, sales#83, number_sales#84] Condition : (isnotnull(sales#83) AND (cast(sales#83 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#61, [id=#62] as decimal(32,6)))) -(84) BroadcastExchange +(82) BroadcastExchange Input [6]: [channel#82, i_brand_id#69, i_class_id#70, i_category_id#71, sales#83, number_sales#84] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#85] -(85) BroadcastHashJoin [codegen id : 52] +(83) BroadcastHashJoin [codegen id : 52] Left keys [3]: [i_brand_id#44, i_class_id#45, i_category_id#46] Right keys [3]: [i_brand_id#69, i_class_id#70, i_category_id#71] Join condition: None -(86) TakeOrderedAndProject +(84) TakeOrderedAndProject Input [12]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sales#59, number_sales#60, channel#82, i_brand_id#69, i_class_id#70, i_category_id#71, sales#83, number_sales#84] Arguments: 100, [i_brand_id#44 ASC NULLS FIRST, i_class_id#45 ASC NULLS FIRST, i_category_id#46 ASC NULLS FIRST], [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sales#59, number_sales#60, channel#82, i_brand_id#69, i_class_id#70, i_category_id#71, sales#83, number_sales#84] ===== Subqueries ===== -Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#61, [id=#62] -* HashAggregate (105) -+- Exchange (104) - +- * HashAggregate (103) - +- Union (102) - :- * Project (91) - : +- * BroadcastHashJoin Inner BuildRight (90) - : :- * ColumnarToRow (88) - : : +- Scan parquet default.store_sales (87) - : +- ReusedExchange (89) - :- * Project (96) - : +- * BroadcastHashJoin Inner BuildRight (95) - : :- * ColumnarToRow (93) - : : +- Scan parquet default.catalog_sales (92) - : +- ReusedExchange (94) - +- * Project (101) - +- * BroadcastHashJoin Inner BuildRight (100) - :- * ColumnarToRow (98) - : +- Scan parquet default.web_sales (97) - +- ReusedExchange (99) - - -(87) Scan parquet default.store_sales +Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#61, [id=#62] +* HashAggregate (103) ++- Exchange (102) + +- * HashAggregate (101) + +- Union (100) + :- * Project (89) + : +- * BroadcastHashJoin Inner BuildRight (88) + : :- * ColumnarToRow (86) + : : +- Scan parquet default.store_sales (85) + : +- ReusedExchange (87) + :- * Project (94) + : +- * BroadcastHashJoin Inner BuildRight (93) + : :- * ColumnarToRow (91) + : : +- Scan parquet default.catalog_sales (90) + : +- ReusedExchange (92) + +- * Project (99) + +- * BroadcastHashJoin Inner BuildRight (98) + :- * ColumnarToRow (96) + : +- Scan parquet default.web_sales (95) + +- ReusedExchange (97) + + +(85) Scan parquet default.store_sales Output [3]: [ss_quantity#86, ss_list_price#87, ss_sold_date_sk#88] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#88), dynamicpruningexpression(ss_sold_date_sk#88 IN dynamicpruning#12)] ReadSchema: struct -(88) ColumnarToRow [codegen id : 2] +(86) ColumnarToRow [codegen id : 2] Input [3]: [ss_quantity#86, ss_list_price#87, ss_sold_date_sk#88] -(89) ReusedExchange [Reuses operator id: 119] +(87) ReusedExchange [Reuses operator id: 117] Output [1]: [d_date_sk#89] -(90) BroadcastHashJoin [codegen id : 2] +(88) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#88] Right keys [1]: [d_date_sk#89] Join condition: None -(91) Project [codegen id : 2] +(89) Project [codegen id : 2] Output [2]: [ss_quantity#86 AS quantity#90, ss_list_price#87 AS list_price#91] Input [4]: [ss_quantity#86, ss_list_price#87, ss_sold_date_sk#88, d_date_sk#89] -(92) Scan parquet default.catalog_sales +(90) Scan parquet default.catalog_sales Output [3]: [cs_quantity#92, cs_list_price#93, cs_sold_date_sk#94] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#94), dynamicpruningexpression(cs_sold_date_sk#94 IN dynamicpruning#12)] ReadSchema: struct -(93) ColumnarToRow [codegen id : 4] +(91) ColumnarToRow [codegen id : 4] Input [3]: [cs_quantity#92, cs_list_price#93, cs_sold_date_sk#94] -(94) ReusedExchange [Reuses operator id: 119] +(92) ReusedExchange [Reuses operator id: 117] Output [1]: [d_date_sk#95] -(95) BroadcastHashJoin [codegen id : 4] +(93) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#94] Right keys [1]: [d_date_sk#95] Join condition: None -(96) Project [codegen id : 4] +(94) Project [codegen id : 4] Output [2]: [cs_quantity#92 AS quantity#96, cs_list_price#93 AS list_price#97] Input [4]: [cs_quantity#92, cs_list_price#93, cs_sold_date_sk#94, d_date_sk#95] -(97) Scan parquet default.web_sales +(95) Scan parquet default.web_sales Output [3]: [ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#100), dynamicpruningexpression(ws_sold_date_sk#100 IN dynamicpruning#12)] ReadSchema: struct -(98) ColumnarToRow [codegen id : 6] +(96) ColumnarToRow [codegen id : 6] Input [3]: [ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] -(99) ReusedExchange [Reuses operator id: 119] +(97) ReusedExchange [Reuses operator id: 117] Output [1]: [d_date_sk#101] -(100) BroadcastHashJoin [codegen id : 6] +(98) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#100] Right keys [1]: [d_date_sk#101] Join condition: None -(101) Project [codegen id : 6] +(99) Project [codegen id : 6] Output [2]: [ws_quantity#98 AS quantity#102, ws_list_price#99 AS list_price#103] Input [4]: [ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100, d_date_sk#101] -(102) Union +(100) Union -(103) HashAggregate [codegen id : 7] +(101) HashAggregate [codegen id : 7] Input [2]: [quantity#90, list_price#91] Keys: [] Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(quantity#90 as decimal(12,2))) * promote_precision(cast(list_price#91 as decimal(12,2)))), DecimalType(18,2)))] Aggregate Attributes [2]: [sum#104, count#105] Results [2]: [sum#106, count#107] -(104) Exchange +(102) Exchange Input [2]: [sum#106, count#107] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#108] -(105) HashAggregate [codegen id : 8] +(103) HashAggregate [codegen id : 8] Input [2]: [sum#106, count#107] Keys: [] Functions [1]: [avg(CheckOverflow((promote_precision(cast(quantity#90 as decimal(12,2))) * promote_precision(cast(list_price#91 as decimal(12,2)))), DecimalType(18,2)))] Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(quantity#90 as decimal(12,2))) * promote_precision(cast(list_price#91 as decimal(12,2)))), DecimalType(18,2)))#109] Results [1]: [avg(CheckOverflow((promote_precision(cast(quantity#90 as decimal(12,2))) * promote_precision(cast(list_price#91 as decimal(12,2)))), DecimalType(18,2)))#109 AS average_sales#110] -Subquery:2 Hosting operator id = 87 Hosting Expression = ss_sold_date_sk#88 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 85 Hosting Expression = ss_sold_date_sk#88 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 92 Hosting Expression = cs_sold_date_sk#94 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#94 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#100 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 95 Hosting Expression = ws_sold_date_sk#100 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (110) -+- * Project (109) - +- * Filter (108) - +- * ColumnarToRow (107) - +- Scan parquet default.date_dim (106) +BroadcastExchange (108) ++- * Project (107) + +- * Filter (106) + +- * ColumnarToRow (105) + +- Scan parquet default.date_dim (104) -(106) Scan parquet default.date_dim +(104) Scan parquet default.date_dim Output [2]: [d_date_sk#48, d_week_seq#111] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(107) ColumnarToRow [codegen id : 1] +(105) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#48, d_week_seq#111] -(108) Filter [codegen id : 1] +(106) Filter [codegen id : 1] Input [2]: [d_date_sk#48, d_week_seq#111] Condition : ((isnotnull(d_week_seq#111) AND (d_week_seq#111 = Subquery scalar-subquery#112, [id=#113])) AND isnotnull(d_date_sk#48)) -(109) Project [codegen id : 1] +(107) Project [codegen id : 1] Output [1]: [d_date_sk#48] Input [2]: [d_date_sk#48, d_week_seq#111] -(110) BroadcastExchange +(108) BroadcastExchange Input [1]: [d_date_sk#48] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#114] -Subquery:6 Hosting operator id = 108 Hosting Expression = Subquery scalar-subquery#112, [id=#113] -* Project (114) -+- * Filter (113) - +- * ColumnarToRow (112) - +- Scan parquet default.date_dim (111) +Subquery:6 Hosting operator id = 106 Hosting Expression = Subquery scalar-subquery#112, [id=#113] +* Project (112) ++- * Filter (111) + +- * ColumnarToRow (110) + +- Scan parquet default.date_dim (109) -(111) Scan parquet default.date_dim +(109) Scan parquet default.date_dim Output [4]: [d_week_seq#115, d_year#116, d_moy#117, d_dom#118] 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 -(112) ColumnarToRow [codegen id : 1] +(110) ColumnarToRow [codegen id : 1] Input [4]: [d_week_seq#115, d_year#116, d_moy#117, d_dom#118] -(113) Filter [codegen id : 1] +(111) Filter [codegen id : 1] Input [4]: [d_week_seq#115, d_year#116, d_moy#117, d_dom#118] Condition : (((((isnotnull(d_year#116) AND isnotnull(d_moy#117)) AND isnotnull(d_dom#118)) AND (d_year#116 = 2000)) AND (d_moy#117 = 12)) AND (d_dom#118 = 11)) -(114) Project [codegen id : 1] +(112) Project [codegen id : 1] Output [1]: [d_week_seq#115] Input [4]: [d_week_seq#115, d_year#116, d_moy#117, d_dom#118] Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (119) -+- * Project (118) - +- * Filter (117) - +- * ColumnarToRow (116) - +- Scan parquet default.date_dim (115) +BroadcastExchange (117) ++- * Project (116) + +- * Filter (115) + +- * ColumnarToRow (114) + +- Scan parquet default.date_dim (113) -(115) Scan parquet default.date_dim +(113) Scan parquet default.date_dim Output [2]: [d_date_sk#27, d_year#119] 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 -(116) ColumnarToRow [codegen id : 1] +(114) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#27, d_year#119] -(117) Filter [codegen id : 1] +(115) Filter [codegen id : 1] Input [2]: [d_date_sk#27, d_year#119] Condition : (((isnotnull(d_year#119) AND (d_year#119 >= 1999)) AND (d_year#119 <= 2001)) AND isnotnull(d_date_sk#27)) -(118) Project [codegen id : 1] +(116) Project [codegen id : 1] Output [1]: [d_date_sk#27] Input [2]: [d_date_sk#27, d_year#119] -(119) BroadcastExchange +(117) BroadcastExchange Input [1]: [d_date_sk#27] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#120] @@ -691,60 +675,60 @@ Subquery:8 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN d Subquery:9 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#12 -Subquery:10 Hosting operator id = 83 Hosting Expression = ReusedSubquery Subquery scalar-subquery#61, [id=#62] +Subquery:10 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#61, [id=#62] -Subquery:11 Hosting operator id = 69 Hosting Expression = ss_sold_date_sk#66 IN dynamicpruning#67 -BroadcastExchange (124) -+- * Project (123) - +- * Filter (122) - +- * ColumnarToRow (121) - +- Scan parquet default.date_dim (120) +Subquery:11 Hosting operator id = 67 Hosting Expression = ss_sold_date_sk#66 IN dynamicpruning#67 +BroadcastExchange (122) ++- * Project (121) + +- * Filter (120) + +- * ColumnarToRow (119) + +- Scan parquet default.date_dim (118) -(120) Scan parquet default.date_dim +(118) Scan parquet default.date_dim Output [2]: [d_date_sk#72, d_week_seq#121] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(121) ColumnarToRow [codegen id : 1] +(119) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#72, d_week_seq#121] -(122) Filter [codegen id : 1] +(120) Filter [codegen id : 1] Input [2]: [d_date_sk#72, d_week_seq#121] Condition : ((isnotnull(d_week_seq#121) AND (d_week_seq#121 = Subquery scalar-subquery#122, [id=#123])) AND isnotnull(d_date_sk#72)) -(123) Project [codegen id : 1] +(121) Project [codegen id : 1] Output [1]: [d_date_sk#72] Input [2]: [d_date_sk#72, d_week_seq#121] -(124) BroadcastExchange +(122) BroadcastExchange Input [1]: [d_date_sk#72] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#124] -Subquery:12 Hosting operator id = 122 Hosting Expression = Subquery scalar-subquery#122, [id=#123] -* Project (128) -+- * Filter (127) - +- * ColumnarToRow (126) - +- Scan parquet default.date_dim (125) +Subquery:12 Hosting operator id = 120 Hosting Expression = Subquery scalar-subquery#122, [id=#123] +* Project (126) ++- * Filter (125) + +- * ColumnarToRow (124) + +- Scan parquet default.date_dim (123) -(125) Scan parquet default.date_dim +(123) Scan parquet default.date_dim Output [4]: [d_week_seq#125, d_year#126, d_moy#127, d_dom#128] 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 -(126) ColumnarToRow [codegen id : 1] +(124) ColumnarToRow [codegen id : 1] Input [4]: [d_week_seq#125, d_year#126, d_moy#127, d_dom#128] -(127) Filter [codegen id : 1] +(125) Filter [codegen id : 1] Input [4]: [d_week_seq#125, d_year#126, d_moy#127, d_dom#128] Condition : (((((isnotnull(d_year#126) AND isnotnull(d_moy#127)) AND isnotnull(d_dom#128)) AND (d_year#126 = 1999)) AND (d_moy#127 = 12)) AND (d_dom#128 = 11)) -(128) Project [codegen id : 1] +(126) Project [codegen id : 1] Output [1]: [d_week_seq#125] Input [4]: [d_week_seq#125, d_year#126, d_moy#127, d_dom#128] 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 8f722e735172f..259178d0e432f 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 @@ -79,77 +79,75 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter BroadcastExchange #4 WholeStageCodegen (10) - HashAggregate [brand_id,class_id,category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] HashAggregate [brand_id,class_id,category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - 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 - ReusedExchange [d_date_sk] #6 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (9) + InputAdapter + Exchange [brand_id,class_id,category_id] #5 + WholeStageCodegen (6) + HashAggregate [brand_id,class_id,category_id] Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] + 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.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #6 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + BroadcastExchange #7 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (3) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + 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 + ReusedExchange [d_date_sk] #6 InputAdapter ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + InputAdapter + ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #11 WholeStageCodegen (23) 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 6011410caced0..3d266ee2c01c7 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 @@ -1,71 +1,64 @@ == Physical Plan == -* HashAggregate (67) -+- Exchange (66) - +- * HashAggregate (65) - +- * HashAggregate (64) - +- Exchange (63) - +- * HashAggregate (62) - +- * SortMergeJoin LeftSemi (61) - :- * Sort (43) - : +- Exchange (42) - : +- * HashAggregate (41) - : +- Exchange (40) - : +- * HashAggregate (39) - : +- * SortMergeJoin LeftSemi (38) - : :- * Sort (20) - : : +- Exchange (19) - : : +- * HashAggregate (18) - : : +- Exchange (17) - : : +- * HashAggregate (16) - : : +- * Project (15) - : : +- * SortMergeJoin Inner (14) - : : :- * Sort (8) - : : : +- Exchange (7) - : : : +- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- * Sort (13) - : : +- Exchange (12) - : : +- * Filter (11) - : : +- * ColumnarToRow (10) - : : +- Scan parquet default.customer (9) - : +- * Sort (37) - : +- Exchange (36) - : +- * HashAggregate (35) - : +- Exchange (34) - : +- * HashAggregate (33) - : +- * Project (32) - : +- * SortMergeJoin Inner (31) - : :- * Sort (28) - : : +- Exchange (27) - : : +- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Filter (23) - : : : +- * ColumnarToRow (22) - : : : +- Scan parquet default.catalog_sales (21) - : : +- ReusedExchange (24) - : +- * Sort (30) - : +- ReusedExchange (29) - +- * Sort (60) - +- Exchange (59) - +- * HashAggregate (58) - +- Exchange (57) - +- * HashAggregate (56) - +- * Project (55) - +- * SortMergeJoin Inner (54) - :- * Sort (51) - : +- Exchange (50) - : +- * Project (49) - : +- * BroadcastHashJoin Inner BuildRight (48) - : :- * Filter (46) - : : +- * ColumnarToRow (45) - : : +- Scan parquet default.web_sales (44) - : +- ReusedExchange (47) - +- * Sort (53) - +- ReusedExchange (52) +* HashAggregate (60) ++- Exchange (59) + +- * HashAggregate (58) + +- * Project (57) + +- * SortMergeJoin LeftSemi (56) + :- * SortMergeJoin LeftSemi (38) + : :- * Sort (20) + : : +- Exchange (19) + : : +- * HashAggregate (18) + : : +- Exchange (17) + : : +- * HashAggregate (16) + : : +- * Project (15) + : : +- * SortMergeJoin Inner (14) + : : :- * Sort (8) + : : : +- Exchange (7) + : : : +- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- * Sort (13) + : : +- Exchange (12) + : : +- * Filter (11) + : : +- * ColumnarToRow (10) + : : +- Scan parquet default.customer (9) + : +- * Sort (37) + : +- Exchange (36) + : +- * HashAggregate (35) + : +- Exchange (34) + : +- * HashAggregate (33) + : +- * Project (32) + : +- * SortMergeJoin Inner (31) + : :- * Sort (28) + : : +- Exchange (27) + : : +- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * Filter (23) + : : : +- * ColumnarToRow (22) + : : : +- Scan parquet default.catalog_sales (21) + : : +- ReusedExchange (24) + : +- * Sort (30) + : +- ReusedExchange (29) + +- * Sort (55) + +- Exchange (54) + +- * HashAggregate (53) + +- Exchange (52) + +- * HashAggregate (51) + +- * Project (50) + +- * SortMergeJoin Inner (49) + :- * Sort (46) + : +- Exchange (45) + : +- * Project (44) + : +- * BroadcastHashJoin Inner BuildRight (43) + : :- * Filter (41) + : : +- * ColumnarToRow (40) + : : +- Scan parquet default.web_sales (39) + : +- ReusedExchange (42) + +- * Sort (48) + +- ReusedExchange (47) (1) Scan parquet default.store_sales @@ -83,7 +76,7 @@ Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] Condition : isnotnull(ss_customer_sk#1) -(4) ReusedExchange [Reuses operator id: 72] +(4) ReusedExchange [Reuses operator id: 65] Output [2]: [d_date_sk#4, d_date#5] (5) BroadcastHashJoin [codegen id : 2] @@ -175,7 +168,7 @@ Input [2]: [cs_bill_customer_sk#13, cs_sold_date_sk#14] Input [2]: [cs_bill_customer_sk#13, cs_sold_date_sk#14] Condition : isnotnull(cs_bill_customer_sk#13) -(24) ReusedExchange [Reuses operator id: 72] +(24) ReusedExchange [Reuses operator id: 65] Output [2]: [d_date_sk#15, d_date#16] (25) BroadcastHashJoin [codegen id : 10] @@ -242,184 +235,144 @@ Left keys [6]: [coalesce(c_last_name#9, ), isnull(c_last_name#9), coalesce(c_fir Right keys [6]: [coalesce(c_last_name#20, ), isnull(c_last_name#20), coalesce(c_first_name#19, ), isnull(c_first_name#19), coalesce(d_date#16, 1970-01-01), isnull(d_date#16)] Join condition: None -(39) HashAggregate [codegen id : 17] -Input [3]: [c_last_name#9, c_first_name#8, d_date#5] -Keys [3]: [c_last_name#9, c_first_name#8, d_date#5] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#9, c_first_name#8, d_date#5] - -(40) Exchange -Input [3]: [c_last_name#9, c_first_name#8, d_date#5] -Arguments: hashpartitioning(c_last_name#9, c_first_name#8, d_date#5, 5), ENSURE_REQUIREMENTS, [id=#23] - -(41) HashAggregate [codegen id : 18] -Input [3]: [c_last_name#9, c_first_name#8, d_date#5] -Keys [3]: [c_last_name#9, c_first_name#8, d_date#5] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#9, c_first_name#8, d_date#5] - -(42) Exchange -Input [3]: [c_last_name#9, c_first_name#8, d_date#5] -Arguments: hashpartitioning(coalesce(c_last_name#9, ), isnull(c_last_name#9), coalesce(c_first_name#8, ), isnull(c_first_name#8), coalesce(d_date#5, 1970-01-01), isnull(d_date#5), 5), ENSURE_REQUIREMENTS, [id=#24] - -(43) Sort [codegen id : 19] -Input [3]: [c_last_name#9, c_first_name#8, d_date#5] -Arguments: [coalesce(c_last_name#9, ) ASC NULLS FIRST, isnull(c_last_name#9) ASC NULLS FIRST, coalesce(c_first_name#8, ) ASC NULLS FIRST, isnull(c_first_name#8) ASC NULLS FIRST, coalesce(d_date#5, 1970-01-01) ASC NULLS FIRST, isnull(d_date#5) ASC NULLS FIRST], false, 0 - -(44) Scan parquet default.web_sales -Output [2]: [ws_bill_customer_sk#25, ws_sold_date_sk#26] +(39) Scan parquet default.web_sales +Output [2]: [ws_bill_customer_sk#23, ws_sold_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#26), dynamicpruningexpression(ws_sold_date_sk#26 IN dynamicpruning#3)] +PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#3)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(45) ColumnarToRow [codegen id : 21] -Input [2]: [ws_bill_customer_sk#25, ws_sold_date_sk#26] +(40) ColumnarToRow [codegen id : 19] +Input [2]: [ws_bill_customer_sk#23, ws_sold_date_sk#24] -(46) Filter [codegen id : 21] -Input [2]: [ws_bill_customer_sk#25, ws_sold_date_sk#26] -Condition : isnotnull(ws_bill_customer_sk#25) +(41) Filter [codegen id : 19] +Input [2]: [ws_bill_customer_sk#23, ws_sold_date_sk#24] +Condition : isnotnull(ws_bill_customer_sk#23) -(47) ReusedExchange [Reuses operator id: 72] -Output [2]: [d_date_sk#27, d_date#28] +(42) ReusedExchange [Reuses operator id: 65] +Output [2]: [d_date_sk#25, d_date#26] -(48) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [ws_sold_date_sk#26] -Right keys [1]: [d_date_sk#27] +(43) BroadcastHashJoin [codegen id : 19] +Left keys [1]: [ws_sold_date_sk#24] +Right keys [1]: [d_date_sk#25] Join condition: None -(49) Project [codegen id : 21] -Output [2]: [ws_bill_customer_sk#25, d_date#28] -Input [4]: [ws_bill_customer_sk#25, ws_sold_date_sk#26, d_date_sk#27, d_date#28] +(44) Project [codegen id : 19] +Output [2]: [ws_bill_customer_sk#23, d_date#26] +Input [4]: [ws_bill_customer_sk#23, ws_sold_date_sk#24, d_date_sk#25, d_date#26] -(50) Exchange -Input [2]: [ws_bill_customer_sk#25, d_date#28] -Arguments: hashpartitioning(ws_bill_customer_sk#25, 5), ENSURE_REQUIREMENTS, [id=#29] +(45) Exchange +Input [2]: [ws_bill_customer_sk#23, d_date#26] +Arguments: hashpartitioning(ws_bill_customer_sk#23, 5), ENSURE_REQUIREMENTS, [id=#27] -(51) Sort [codegen id : 22] -Input [2]: [ws_bill_customer_sk#25, d_date#28] -Arguments: [ws_bill_customer_sk#25 ASC NULLS FIRST], false, 0 +(46) Sort [codegen id : 20] +Input [2]: [ws_bill_customer_sk#23, d_date#26] +Arguments: [ws_bill_customer_sk#23 ASC NULLS FIRST], false, 0 -(52) ReusedExchange [Reuses operator id: 12] -Output [3]: [c_customer_sk#30, c_first_name#31, c_last_name#32] +(47) ReusedExchange [Reuses operator id: 12] +Output [3]: [c_customer_sk#28, c_first_name#29, c_last_name#30] -(53) Sort [codegen id : 24] -Input [3]: [c_customer_sk#30, c_first_name#31, c_last_name#32] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +(48) Sort [codegen id : 22] +Input [3]: [c_customer_sk#28, c_first_name#29, c_last_name#30] +Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 -(54) SortMergeJoin [codegen id : 25] -Left keys [1]: [ws_bill_customer_sk#25] -Right keys [1]: [c_customer_sk#30] +(49) SortMergeJoin [codegen id : 23] +Left keys [1]: [ws_bill_customer_sk#23] +Right keys [1]: [c_customer_sk#28] Join condition: None -(55) Project [codegen id : 25] -Output [3]: [c_last_name#32, c_first_name#31, d_date#28] -Input [5]: [ws_bill_customer_sk#25, d_date#28, c_customer_sk#30, c_first_name#31, c_last_name#32] +(50) Project [codegen id : 23] +Output [3]: [c_last_name#30, c_first_name#29, d_date#26] +Input [5]: [ws_bill_customer_sk#23, d_date#26, c_customer_sk#28, c_first_name#29, c_last_name#30] -(56) HashAggregate [codegen id : 25] -Input [3]: [c_last_name#32, c_first_name#31, d_date#28] -Keys [3]: [c_last_name#32, c_first_name#31, d_date#28] +(51) HashAggregate [codegen id : 23] +Input [3]: [c_last_name#30, c_first_name#29, d_date#26] +Keys [3]: [c_last_name#30, c_first_name#29, d_date#26] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#32, c_first_name#31, d_date#28] +Results [3]: [c_last_name#30, c_first_name#29, d_date#26] -(57) Exchange -Input [3]: [c_last_name#32, c_first_name#31, d_date#28] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, d_date#28, 5), ENSURE_REQUIREMENTS, [id=#33] +(52) Exchange +Input [3]: [c_last_name#30, c_first_name#29, d_date#26] +Arguments: hashpartitioning(c_last_name#30, c_first_name#29, d_date#26, 5), ENSURE_REQUIREMENTS, [id=#31] -(58) HashAggregate [codegen id : 26] -Input [3]: [c_last_name#32, c_first_name#31, d_date#28] -Keys [3]: [c_last_name#32, c_first_name#31, d_date#28] +(53) HashAggregate [codegen id : 24] +Input [3]: [c_last_name#30, c_first_name#29, d_date#26] +Keys [3]: [c_last_name#30, c_first_name#29, d_date#26] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#32, c_first_name#31, d_date#28] +Results [3]: [c_last_name#30, c_first_name#29, d_date#26] -(59) Exchange -Input [3]: [c_last_name#32, c_first_name#31, d_date#28] -Arguments: hashpartitioning(coalesce(c_last_name#32, ), isnull(c_last_name#32), coalesce(c_first_name#31, ), isnull(c_first_name#31), coalesce(d_date#28, 1970-01-01), isnull(d_date#28), 5), ENSURE_REQUIREMENTS, [id=#34] +(54) Exchange +Input [3]: [c_last_name#30, c_first_name#29, d_date#26] +Arguments: hashpartitioning(coalesce(c_last_name#30, ), isnull(c_last_name#30), coalesce(c_first_name#29, ), isnull(c_first_name#29), coalesce(d_date#26, 1970-01-01), isnull(d_date#26), 5), ENSURE_REQUIREMENTS, [id=#32] -(60) Sort [codegen id : 27] -Input [3]: [c_last_name#32, c_first_name#31, d_date#28] -Arguments: [coalesce(c_last_name#32, ) ASC NULLS FIRST, isnull(c_last_name#32) ASC NULLS FIRST, coalesce(c_first_name#31, ) ASC NULLS FIRST, isnull(c_first_name#31) ASC NULLS FIRST, coalesce(d_date#28, 1970-01-01) ASC NULLS FIRST, isnull(d_date#28) ASC NULLS FIRST], false, 0 +(55) Sort [codegen id : 25] +Input [3]: [c_last_name#30, c_first_name#29, d_date#26] +Arguments: [coalesce(c_last_name#30, ) ASC NULLS FIRST, isnull(c_last_name#30) ASC NULLS FIRST, coalesce(c_first_name#29, ) ASC NULLS FIRST, isnull(c_first_name#29) ASC NULLS FIRST, coalesce(d_date#26, 1970-01-01) ASC NULLS FIRST, isnull(d_date#26) ASC NULLS FIRST], false, 0 -(61) SortMergeJoin [codegen id : 28] +(56) SortMergeJoin [codegen id : 26] Left keys [6]: [coalesce(c_last_name#9, ), isnull(c_last_name#9), coalesce(c_first_name#8, ), isnull(c_first_name#8), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#32, ), isnull(c_last_name#32), coalesce(c_first_name#31, ), isnull(c_first_name#31), coalesce(d_date#28, 1970-01-01), isnull(d_date#28)] +Right keys [6]: [coalesce(c_last_name#30, ), isnull(c_last_name#30), coalesce(c_first_name#29, ), isnull(c_first_name#29), coalesce(d_date#26, 1970-01-01), isnull(d_date#26)] Join condition: None -(62) HashAggregate [codegen id : 28] +(57) Project [codegen id : 26] +Output: [] Input [3]: [c_last_name#9, c_first_name#8, d_date#5] -Keys [3]: [c_last_name#9, c_first_name#8, d_date#5] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#9, c_first_name#8, d_date#5] -(63) Exchange -Input [3]: [c_last_name#9, c_first_name#8, d_date#5] -Arguments: hashpartitioning(c_last_name#9, c_first_name#8, d_date#5, 5), ENSURE_REQUIREMENTS, [id=#35] - -(64) HashAggregate [codegen id : 29] -Input [3]: [c_last_name#9, c_first_name#8, d_date#5] -Keys [3]: [c_last_name#9, c_first_name#8, d_date#5] -Functions: [] -Aggregate Attributes: [] -Results: [] - -(65) HashAggregate [codegen id : 29] +(58) HashAggregate [codegen id : 26] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#36] -Results [1]: [count#37] +Aggregate Attributes [1]: [count#33] +Results [1]: [count#34] -(66) Exchange -Input [1]: [count#37] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#38] +(59) Exchange +Input [1]: [count#34] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#35] -(67) HashAggregate [codegen id : 30] -Input [1]: [count#37] +(60) HashAggregate [codegen id : 27] +Input [1]: [count#34] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#39] -Results [1]: [count(1)#39 AS count(1)#40] +Aggregate Attributes [1]: [count(1)#36] +Results [1]: [count(1)#36 AS count(1)#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 -BroadcastExchange (72) -+- * Project (71) - +- * Filter (70) - +- * ColumnarToRow (69) - +- Scan parquet default.date_dim (68) +BroadcastExchange (65) ++- * Project (64) + +- * Filter (63) + +- * ColumnarToRow (62) + +- Scan parquet default.date_dim (61) -(68) Scan parquet default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#41] +(61) Scan parquet default.date_dim +Output [3]: [d_date_sk#4, d_date#5, d_month_seq#38] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(69) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#41] +(62) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#4, d_date#5, d_month_seq#38] -(70) Filter [codegen id : 1] -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#41] -Condition : (((isnotnull(d_month_seq#41) AND (d_month_seq#41 >= 1200)) AND (d_month_seq#41 <= 1211)) AND isnotnull(d_date_sk#4)) +(63) Filter [codegen id : 1] +Input [3]: [d_date_sk#4, d_date#5, d_month_seq#38] +Condition : (((isnotnull(d_month_seq#38) AND (d_month_seq#38 >= 1200)) AND (d_month_seq#38 <= 1211)) AND isnotnull(d_date_sk#4)) -(71) Project [codegen id : 1] +(64) Project [codegen id : 1] Output [2]: [d_date_sk#4, d_date#5] -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#41] +Input [3]: [d_date_sk#4, d_date#5, d_month_seq#38] -(72) BroadcastExchange +(65) BroadcastExchange Input [2]: [d_date_sk#4, d_date#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] Subquery:2 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#3 -Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#26 IN dynamicpruning#3 +Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/simplified.txt index eda0d4b03f483..cc66a0040ef9a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/simplified.txt @@ -1,135 +1,122 @@ -WholeStageCodegen (30) +WholeStageCodegen (27) HashAggregate [count] [count(1),count(1),count] InputAdapter Exchange #1 - WholeStageCodegen (29) + WholeStageCodegen (26) HashAggregate [count,count] - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (28) - HashAggregate [c_last_name,c_first_name,d_date] - SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - InputAdapter - WholeStageCodegen (19) - Sort [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #3 - WholeStageCodegen (18) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #4 - WholeStageCodegen (17) - HashAggregate [c_last_name,c_first_name,d_date] - SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + Project + SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + InputAdapter + WholeStageCodegen (17) + SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + InputAdapter + WholeStageCodegen (8) + Sort [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #2 + WholeStageCodegen (7) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #3 + WholeStageCodegen (6) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (8) - Sort [c_last_name,c_first_name,d_date] + WholeStageCodegen (3) + Sort [ss_customer_sk] InputAdapter - Exchange [c_last_name,c_first_name,d_date] #5 - WholeStageCodegen (7) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #6 - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - SortMergeJoin [ss_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (3) - Sort [ss_customer_sk] - InputAdapter - Exchange [ss_customer_sk] #7 - WholeStageCodegen (2) - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #8 - WholeStageCodegen (1) - Project [d_date_sk,d_date] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #8 - InputAdapter - WholeStageCodegen (5) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #9 - WholeStageCodegen (4) - Filter [c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] + Exchange [ss_customer_sk] #4 + WholeStageCodegen (2) + Project [ss_customer_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + Project [d_date_sk,d_date] + Filter [d_month_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk,d_date] #5 InputAdapter - WholeStageCodegen (16) - Sort [c_last_name,c_first_name,d_date] + WholeStageCodegen (5) + Sort [c_customer_sk] InputAdapter - Exchange [c_last_name,c_first_name,d_date] #10 - WholeStageCodegen (15) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #11 - WholeStageCodegen (14) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - SortMergeJoin [cs_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (11) - Sort [cs_bill_customer_sk] - InputAdapter - Exchange [cs_bill_customer_sk] #12 - WholeStageCodegen (10) - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #8 - InputAdapter - WholeStageCodegen (13) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #9 - InputAdapter - WholeStageCodegen (27) - Sort [c_last_name,c_first_name,d_date] + Exchange [c_customer_sk] #6 + WholeStageCodegen (4) + Filter [c_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] + InputAdapter + WholeStageCodegen (16) + Sort [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #7 + WholeStageCodegen (15) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #8 + WholeStageCodegen (14) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (11) + Sort [cs_bill_customer_sk] + InputAdapter + Exchange [cs_bill_customer_sk] #9 + WholeStageCodegen (10) + Project [cs_bill_customer_sk,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_date] #5 + InputAdapter + WholeStageCodegen (13) + Sort [c_customer_sk] + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #6 + InputAdapter + WholeStageCodegen (25) + Sort [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #10 + WholeStageCodegen (24) + HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_last_name,c_first_name,d_date] #13 - WholeStageCodegen (26) + Exchange [c_last_name,c_first_name,d_date] #11 + WholeStageCodegen (23) HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #14 - WholeStageCodegen (25) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - SortMergeJoin [ws_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (22) - Sort [ws_bill_customer_sk] - InputAdapter - Exchange [ws_bill_customer_sk] #15 - WholeStageCodegen (21) - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #8 - InputAdapter - WholeStageCodegen (24) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #9 + Project [c_last_name,c_first_name,d_date] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (20) + Sort [ws_bill_customer_sk] + InputAdapter + Exchange [ws_bill_customer_sk] #12 + WholeStageCodegen (19) + Project [ws_bill_customer_sk,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_date] #5 + InputAdapter + WholeStageCodegen (22) + Sort [c_customer_sk] + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #6 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 ca4a34d7b6087..60190c9f39e43 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 @@ -1,54 +1,51 @@ == Physical Plan == -* HashAggregate (50) -+- Exchange (49) - +- * HashAggregate (48) - +- * HashAggregate (47) - +- * HashAggregate (46) - +- * BroadcastHashJoin LeftSemi BuildRight (45) - :- * HashAggregate (31) - : +- * HashAggregate (30) - : +- * BroadcastHashJoin LeftSemi BuildRight (29) - : :- * HashAggregate (15) - : : +- Exchange (14) - : : +- * HashAggregate (13) - : : +- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (10) - : : +- * Filter (9) - : : +- * ColumnarToRow (8) - : : +- Scan parquet default.customer (7) - : +- BroadcastExchange (28) - : +- * HashAggregate (27) - : +- Exchange (26) - : +- * HashAggregate (25) - : +- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (21) - : : +- * BroadcastHashJoin Inner BuildRight (20) - : : :- * Filter (18) - : : : +- * ColumnarToRow (17) - : : : +- Scan parquet default.catalog_sales (16) - : : +- ReusedExchange (19) - : +- ReusedExchange (22) - +- BroadcastExchange (44) - +- * HashAggregate (43) - +- Exchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Filter (34) - : : +- * ColumnarToRow (33) - : : +- Scan parquet default.web_sales (32) - : +- ReusedExchange (35) - +- ReusedExchange (38) +* HashAggregate (47) ++- Exchange (46) + +- * HashAggregate (45) + +- * Project (44) + +- * BroadcastHashJoin LeftSemi BuildRight (43) + :- * BroadcastHashJoin LeftSemi BuildRight (29) + : :- * HashAggregate (15) + : : +- Exchange (14) + : : +- * HashAggregate (13) + : : +- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (10) + : : +- * Filter (9) + : : +- * ColumnarToRow (8) + : : +- Scan parquet default.customer (7) + : +- BroadcastExchange (28) + : +- * HashAggregate (27) + : +- Exchange (26) + : +- * HashAggregate (25) + : +- * Project (24) + : +- * BroadcastHashJoin Inner BuildRight (23) + : :- * Project (21) + : : +- * BroadcastHashJoin Inner BuildRight (20) + : : :- * Filter (18) + : : : +- * ColumnarToRow (17) + : : : +- Scan parquet default.catalog_sales (16) + : : +- ReusedExchange (19) + : +- ReusedExchange (22) + +- BroadcastExchange (42) + +- * HashAggregate (41) + +- Exchange (40) + +- * HashAggregate (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * Filter (32) + : : +- * ColumnarToRow (31) + : : +- Scan parquet default.web_sales (30) + : +- ReusedExchange (33) + +- ReusedExchange (36) (1) Scan parquet default.store_sales @@ -66,7 +63,7 @@ Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] Condition : isnotnull(ss_customer_sk#1) -(4) ReusedExchange [Reuses operator id: 55] +(4) ReusedExchange [Reuses operator id: 52] Output [2]: [d_date_sk#4, d_date#5] (5) BroadcastHashJoin [codegen id : 3] @@ -138,7 +135,7 @@ Input [2]: [cs_bill_customer_sk#11, cs_sold_date_sk#12] Input [2]: [cs_bill_customer_sk#11, cs_sold_date_sk#12] Condition : isnotnull(cs_bill_customer_sk#11) -(19) ReusedExchange [Reuses operator id: 55] +(19) ReusedExchange [Reuses operator id: 52] Output [2]: [d_date_sk#13, d_date#14] (20) BroadcastHashJoin [codegen id : 6] @@ -189,21 +186,7 @@ Left keys [6]: [coalesce(c_last_name#8, ), isnull(c_last_name#8), coalesce(c_fir Right keys [6]: [coalesce(c_last_name#17, ), isnull(c_last_name#17), coalesce(c_first_name#16, ), isnull(c_first_name#16), coalesce(d_date#14, 1970-01-01), isnull(d_date#14)] Join condition: None -(30) HashAggregate [codegen id : 12] -Input [3]: [c_last_name#8, c_first_name#7, d_date#5] -Keys [3]: [c_last_name#8, c_first_name#7, d_date#5] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#8, c_first_name#7, d_date#5] - -(31) HashAggregate [codegen id : 12] -Input [3]: [c_last_name#8, c_first_name#7, d_date#5] -Keys [3]: [c_last_name#8, c_first_name#7, d_date#5] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#8, c_first_name#7, d_date#5] - -(32) Scan parquet default.web_sales +(30) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] @@ -211,90 +194,80 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#21), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(33) ColumnarToRow [codegen id : 10] +(31) ColumnarToRow [codegen id : 10] Input [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -(34) Filter [codegen id : 10] +(32) Filter [codegen id : 10] Input [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] Condition : isnotnull(ws_bill_customer_sk#20) -(35) ReusedExchange [Reuses operator id: 55] +(33) ReusedExchange [Reuses operator id: 52] Output [2]: [d_date_sk#22, d_date#23] -(36) BroadcastHashJoin [codegen id : 10] +(34) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ws_sold_date_sk#21] Right keys [1]: [d_date_sk#22] Join condition: None -(37) Project [codegen id : 10] +(35) Project [codegen id : 10] Output [2]: [ws_bill_customer_sk#20, d_date#23] Input [4]: [ws_bill_customer_sk#20, ws_sold_date_sk#21, d_date_sk#22, d_date#23] -(38) ReusedExchange [Reuses operator id: 10] +(36) ReusedExchange [Reuses operator id: 10] Output [3]: [c_customer_sk#24, c_first_name#25, c_last_name#26] -(39) BroadcastHashJoin [codegen id : 10] +(37) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ws_bill_customer_sk#20] Right keys [1]: [c_customer_sk#24] Join condition: None -(40) Project [codegen id : 10] +(38) Project [codegen id : 10] Output [3]: [c_last_name#26, c_first_name#25, d_date#23] Input [5]: [ws_bill_customer_sk#20, d_date#23, c_customer_sk#24, c_first_name#25, c_last_name#26] -(41) HashAggregate [codegen id : 10] +(39) HashAggregate [codegen id : 10] Input [3]: [c_last_name#26, c_first_name#25, d_date#23] Keys [3]: [c_last_name#26, c_first_name#25, d_date#23] Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#26, c_first_name#25, d_date#23] -(42) Exchange +(40) Exchange Input [3]: [c_last_name#26, c_first_name#25, d_date#23] Arguments: hashpartitioning(c_last_name#26, c_first_name#25, d_date#23, 5), ENSURE_REQUIREMENTS, [id=#27] -(43) HashAggregate [codegen id : 11] +(41) HashAggregate [codegen id : 11] Input [3]: [c_last_name#26, c_first_name#25, d_date#23] Keys [3]: [c_last_name#26, c_first_name#25, d_date#23] Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#26, c_first_name#25, d_date#23] -(44) BroadcastExchange +(42) BroadcastExchange Input [3]: [c_last_name#26, c_first_name#25, d_date#23] 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=#28] -(45) BroadcastHashJoin [codegen id : 12] +(43) BroadcastHashJoin [codegen id : 12] Left keys [6]: [coalesce(c_last_name#8, ), isnull(c_last_name#8), coalesce(c_first_name#7, ), isnull(c_first_name#7), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] Right keys [6]: [coalesce(c_last_name#26, ), isnull(c_last_name#26), coalesce(c_first_name#25, ), isnull(c_first_name#25), coalesce(d_date#23, 1970-01-01), isnull(d_date#23)] Join condition: None -(46) HashAggregate [codegen id : 12] -Input [3]: [c_last_name#8, c_first_name#7, d_date#5] -Keys [3]: [c_last_name#8, c_first_name#7, d_date#5] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#8, c_first_name#7, d_date#5] - -(47) HashAggregate [codegen id : 12] +(44) Project [codegen id : 12] +Output: [] Input [3]: [c_last_name#8, c_first_name#7, d_date#5] -Keys [3]: [c_last_name#8, c_first_name#7, d_date#5] -Functions: [] -Aggregate Attributes: [] -Results: [] -(48) HashAggregate [codegen id : 12] +(45) HashAggregate [codegen id : 12] Input: [] Keys: [] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#29] Results [1]: [count#30] -(49) Exchange +(46) Exchange Input [1]: [count#30] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] -(50) HashAggregate [codegen id : 13] +(47) HashAggregate [codegen id : 13] Input [1]: [count#30] Keys: [] Functions [1]: [count(1)] @@ -304,37 +277,37 @@ Results [1]: [count(1)#32 AS count(1)#33] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 -BroadcastExchange (55) -+- * Project (54) - +- * Filter (53) - +- * ColumnarToRow (52) - +- Scan parquet default.date_dim (51) +BroadcastExchange (52) ++- * Project (51) + +- * Filter (50) + +- * ColumnarToRow (49) + +- Scan parquet default.date_dim (48) -(51) Scan parquet default.date_dim +(48) Scan parquet default.date_dim Output [3]: [d_date_sk#4, d_date#5, d_month_seq#34] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(52) ColumnarToRow [codegen id : 1] +(49) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#4, d_date#5, d_month_seq#34] -(53) Filter [codegen id : 1] +(50) Filter [codegen id : 1] Input [3]: [d_date_sk#4, d_date#5, d_month_seq#34] Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= 1200)) AND (d_month_seq#34 <= 1211)) AND isnotnull(d_date_sk#4)) -(54) Project [codegen id : 1] +(51) Project [codegen id : 1] Output [2]: [d_date_sk#4, d_date#5] Input [3]: [d_date_sk#4, d_date#5, d_month_seq#34] -(55) BroadcastExchange +(52) BroadcastExchange Input [2]: [d_date_sk#4, d_date#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] Subquery:2 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#12 IN dynamicpruning#3 -Subquery:3 Hosting operator id = 32 Hosting Expression = ws_sold_date_sk#21 IN dynamicpruning#3 +Subquery:3 Hosting operator id = 30 Hosting Expression = ws_sold_date_sk#21 IN dynamicpruning#3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt index 7f96f5657836a..34d46c5671774 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt @@ -4,81 +4,78 @@ WholeStageCodegen (13) Exchange #1 WholeStageCodegen (12) HashAggregate [count,count] - HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] + Project + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk,d_date] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Filter [c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #2 + WholeStageCodegen (3) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + Project [d_date_sk,d_date] + Filter [d_month_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk,d_date] #3 InputAdapter - Exchange [c_last_name,c_first_name,d_date] #6 - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + BroadcastExchange #4 + WholeStageCodegen (2) + Filter [c_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (11) + BroadcastExchange #5 + WholeStageCodegen (7) HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (10) + Exchange [c_last_name,c_first_name,d_date] #6 + WholeStageCodegen (6) HashAggregate [c_last_name,c_first_name,d_date] Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_bill_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk,d_date] #3 InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (11) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #8 + WholeStageCodegen (10) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + Project [ws_bill_customer_sk,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/explain.txt index 408b0defda53c..38ecc6f3ed822 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 @@ -1,71 +1,64 @@ == Physical Plan == -* HashAggregate (67) -+- Exchange (66) - +- * HashAggregate (65) - +- * HashAggregate (64) - +- Exchange (63) - +- * HashAggregate (62) - +- * SortMergeJoin LeftAnti (61) - :- * Sort (43) - : +- Exchange (42) - : +- * HashAggregate (41) - : +- Exchange (40) - : +- * HashAggregate (39) - : +- * SortMergeJoin LeftAnti (38) - : :- * Sort (20) - : : +- Exchange (19) - : : +- * HashAggregate (18) - : : +- Exchange (17) - : : +- * HashAggregate (16) - : : +- * Project (15) - : : +- * SortMergeJoin Inner (14) - : : :- * Sort (8) - : : : +- Exchange (7) - : : : +- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- * Sort (13) - : : +- Exchange (12) - : : +- * Filter (11) - : : +- * ColumnarToRow (10) - : : +- Scan parquet default.customer (9) - : +- * Sort (37) - : +- Exchange (36) - : +- * HashAggregate (35) - : +- Exchange (34) - : +- * HashAggregate (33) - : +- * Project (32) - : +- * SortMergeJoin Inner (31) - : :- * Sort (28) - : : +- Exchange (27) - : : +- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Filter (23) - : : : +- * ColumnarToRow (22) - : : : +- Scan parquet default.catalog_sales (21) - : : +- ReusedExchange (24) - : +- * Sort (30) - : +- ReusedExchange (29) - +- * Sort (60) - +- Exchange (59) - +- * HashAggregate (58) - +- Exchange (57) - +- * HashAggregate (56) - +- * Project (55) - +- * SortMergeJoin Inner (54) - :- * Sort (51) - : +- Exchange (50) - : +- * Project (49) - : +- * BroadcastHashJoin Inner BuildRight (48) - : :- * Filter (46) - : : +- * ColumnarToRow (45) - : : +- Scan parquet default.web_sales (44) - : +- ReusedExchange (47) - +- * Sort (53) - +- ReusedExchange (52) +* HashAggregate (60) ++- Exchange (59) + +- * HashAggregate (58) + +- * Project (57) + +- * SortMergeJoin LeftAnti (56) + :- * SortMergeJoin LeftAnti (38) + : :- * Sort (20) + : : +- Exchange (19) + : : +- * HashAggregate (18) + : : +- Exchange (17) + : : +- * HashAggregate (16) + : : +- * Project (15) + : : +- * SortMergeJoin Inner (14) + : : :- * Sort (8) + : : : +- Exchange (7) + : : : +- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- * Sort (13) + : : +- Exchange (12) + : : +- * Filter (11) + : : +- * ColumnarToRow (10) + : : +- Scan parquet default.customer (9) + : +- * Sort (37) + : +- Exchange (36) + : +- * HashAggregate (35) + : +- Exchange (34) + : +- * HashAggregate (33) + : +- * Project (32) + : +- * SortMergeJoin Inner (31) + : :- * Sort (28) + : : +- Exchange (27) + : : +- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * Filter (23) + : : : +- * ColumnarToRow (22) + : : : +- Scan parquet default.catalog_sales (21) + : : +- ReusedExchange (24) + : +- * Sort (30) + : +- ReusedExchange (29) + +- * Sort (55) + +- Exchange (54) + +- * HashAggregate (53) + +- Exchange (52) + +- * HashAggregate (51) + +- * Project (50) + +- * SortMergeJoin Inner (49) + :- * Sort (46) + : +- Exchange (45) + : +- * Project (44) + : +- * BroadcastHashJoin Inner BuildRight (43) + : :- * Filter (41) + : : +- * ColumnarToRow (40) + : : +- Scan parquet default.web_sales (39) + : +- ReusedExchange (42) + +- * Sort (48) + +- ReusedExchange (47) (1) Scan parquet default.store_sales @@ -83,7 +76,7 @@ Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] Condition : isnotnull(ss_customer_sk#1) -(4) ReusedExchange [Reuses operator id: 72] +(4) ReusedExchange [Reuses operator id: 65] Output [2]: [d_date_sk#4, d_date#5] (5) BroadcastHashJoin [codegen id : 2] @@ -175,7 +168,7 @@ Input [2]: [cs_bill_customer_sk#13, cs_sold_date_sk#14] Input [2]: [cs_bill_customer_sk#13, cs_sold_date_sk#14] Condition : isnotnull(cs_bill_customer_sk#13) -(24) ReusedExchange [Reuses operator id: 72] +(24) ReusedExchange [Reuses operator id: 65] Output [2]: [d_date_sk#15, d_date#16] (25) BroadcastHashJoin [codegen id : 10] @@ -242,184 +235,144 @@ Left keys [6]: [coalesce(c_last_name#9, ), isnull(c_last_name#9), coalesce(c_fir Right keys [6]: [coalesce(c_last_name#20, ), isnull(c_last_name#20), coalesce(c_first_name#19, ), isnull(c_first_name#19), coalesce(d_date#16, 1970-01-01), isnull(d_date#16)] Join condition: None -(39) HashAggregate [codegen id : 17] -Input [3]: [c_last_name#9, c_first_name#8, d_date#5] -Keys [3]: [c_last_name#9, c_first_name#8, d_date#5] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#9, c_first_name#8, d_date#5] - -(40) Exchange -Input [3]: [c_last_name#9, c_first_name#8, d_date#5] -Arguments: hashpartitioning(c_last_name#9, c_first_name#8, d_date#5, 5), ENSURE_REQUIREMENTS, [id=#23] - -(41) HashAggregate [codegen id : 18] -Input [3]: [c_last_name#9, c_first_name#8, d_date#5] -Keys [3]: [c_last_name#9, c_first_name#8, d_date#5] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#9, c_first_name#8, d_date#5] - -(42) Exchange -Input [3]: [c_last_name#9, c_first_name#8, d_date#5] -Arguments: hashpartitioning(coalesce(c_last_name#9, ), isnull(c_last_name#9), coalesce(c_first_name#8, ), isnull(c_first_name#8), coalesce(d_date#5, 1970-01-01), isnull(d_date#5), 5), ENSURE_REQUIREMENTS, [id=#24] - -(43) Sort [codegen id : 19] -Input [3]: [c_last_name#9, c_first_name#8, d_date#5] -Arguments: [coalesce(c_last_name#9, ) ASC NULLS FIRST, isnull(c_last_name#9) ASC NULLS FIRST, coalesce(c_first_name#8, ) ASC NULLS FIRST, isnull(c_first_name#8) ASC NULLS FIRST, coalesce(d_date#5, 1970-01-01) ASC NULLS FIRST, isnull(d_date#5) ASC NULLS FIRST], false, 0 - -(44) Scan parquet default.web_sales -Output [2]: [ws_bill_customer_sk#25, ws_sold_date_sk#26] +(39) Scan parquet default.web_sales +Output [2]: [ws_bill_customer_sk#23, ws_sold_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#26), dynamicpruningexpression(ws_sold_date_sk#26 IN dynamicpruning#3)] +PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#3)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(45) ColumnarToRow [codegen id : 21] -Input [2]: [ws_bill_customer_sk#25, ws_sold_date_sk#26] +(40) ColumnarToRow [codegen id : 19] +Input [2]: [ws_bill_customer_sk#23, ws_sold_date_sk#24] -(46) Filter [codegen id : 21] -Input [2]: [ws_bill_customer_sk#25, ws_sold_date_sk#26] -Condition : isnotnull(ws_bill_customer_sk#25) +(41) Filter [codegen id : 19] +Input [2]: [ws_bill_customer_sk#23, ws_sold_date_sk#24] +Condition : isnotnull(ws_bill_customer_sk#23) -(47) ReusedExchange [Reuses operator id: 72] -Output [2]: [d_date_sk#27, d_date#28] +(42) ReusedExchange [Reuses operator id: 65] +Output [2]: [d_date_sk#25, d_date#26] -(48) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [ws_sold_date_sk#26] -Right keys [1]: [d_date_sk#27] +(43) BroadcastHashJoin [codegen id : 19] +Left keys [1]: [ws_sold_date_sk#24] +Right keys [1]: [d_date_sk#25] Join condition: None -(49) Project [codegen id : 21] -Output [2]: [ws_bill_customer_sk#25, d_date#28] -Input [4]: [ws_bill_customer_sk#25, ws_sold_date_sk#26, d_date_sk#27, d_date#28] +(44) Project [codegen id : 19] +Output [2]: [ws_bill_customer_sk#23, d_date#26] +Input [4]: [ws_bill_customer_sk#23, ws_sold_date_sk#24, d_date_sk#25, d_date#26] -(50) Exchange -Input [2]: [ws_bill_customer_sk#25, d_date#28] -Arguments: hashpartitioning(ws_bill_customer_sk#25, 5), ENSURE_REQUIREMENTS, [id=#29] +(45) Exchange +Input [2]: [ws_bill_customer_sk#23, d_date#26] +Arguments: hashpartitioning(ws_bill_customer_sk#23, 5), ENSURE_REQUIREMENTS, [id=#27] -(51) Sort [codegen id : 22] -Input [2]: [ws_bill_customer_sk#25, d_date#28] -Arguments: [ws_bill_customer_sk#25 ASC NULLS FIRST], false, 0 +(46) Sort [codegen id : 20] +Input [2]: [ws_bill_customer_sk#23, d_date#26] +Arguments: [ws_bill_customer_sk#23 ASC NULLS FIRST], false, 0 -(52) ReusedExchange [Reuses operator id: 12] -Output [3]: [c_customer_sk#30, c_first_name#31, c_last_name#32] +(47) ReusedExchange [Reuses operator id: 12] +Output [3]: [c_customer_sk#28, c_first_name#29, c_last_name#30] -(53) Sort [codegen id : 24] -Input [3]: [c_customer_sk#30, c_first_name#31, c_last_name#32] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +(48) Sort [codegen id : 22] +Input [3]: [c_customer_sk#28, c_first_name#29, c_last_name#30] +Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 -(54) SortMergeJoin [codegen id : 25] -Left keys [1]: [ws_bill_customer_sk#25] -Right keys [1]: [c_customer_sk#30] +(49) SortMergeJoin [codegen id : 23] +Left keys [1]: [ws_bill_customer_sk#23] +Right keys [1]: [c_customer_sk#28] Join condition: None -(55) Project [codegen id : 25] -Output [3]: [c_last_name#32, c_first_name#31, d_date#28] -Input [5]: [ws_bill_customer_sk#25, d_date#28, c_customer_sk#30, c_first_name#31, c_last_name#32] +(50) Project [codegen id : 23] +Output [3]: [c_last_name#30, c_first_name#29, d_date#26] +Input [5]: [ws_bill_customer_sk#23, d_date#26, c_customer_sk#28, c_first_name#29, c_last_name#30] -(56) HashAggregate [codegen id : 25] -Input [3]: [c_last_name#32, c_first_name#31, d_date#28] -Keys [3]: [c_last_name#32, c_first_name#31, d_date#28] +(51) HashAggregate [codegen id : 23] +Input [3]: [c_last_name#30, c_first_name#29, d_date#26] +Keys [3]: [c_last_name#30, c_first_name#29, d_date#26] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#32, c_first_name#31, d_date#28] +Results [3]: [c_last_name#30, c_first_name#29, d_date#26] -(57) Exchange -Input [3]: [c_last_name#32, c_first_name#31, d_date#28] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, d_date#28, 5), ENSURE_REQUIREMENTS, [id=#33] +(52) Exchange +Input [3]: [c_last_name#30, c_first_name#29, d_date#26] +Arguments: hashpartitioning(c_last_name#30, c_first_name#29, d_date#26, 5), ENSURE_REQUIREMENTS, [id=#31] -(58) HashAggregate [codegen id : 26] -Input [3]: [c_last_name#32, c_first_name#31, d_date#28] -Keys [3]: [c_last_name#32, c_first_name#31, d_date#28] +(53) HashAggregate [codegen id : 24] +Input [3]: [c_last_name#30, c_first_name#29, d_date#26] +Keys [3]: [c_last_name#30, c_first_name#29, d_date#26] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#32, c_first_name#31, d_date#28] +Results [3]: [c_last_name#30, c_first_name#29, d_date#26] -(59) Exchange -Input [3]: [c_last_name#32, c_first_name#31, d_date#28] -Arguments: hashpartitioning(coalesce(c_last_name#32, ), isnull(c_last_name#32), coalesce(c_first_name#31, ), isnull(c_first_name#31), coalesce(d_date#28, 1970-01-01), isnull(d_date#28), 5), ENSURE_REQUIREMENTS, [id=#34] +(54) Exchange +Input [3]: [c_last_name#30, c_first_name#29, d_date#26] +Arguments: hashpartitioning(coalesce(c_last_name#30, ), isnull(c_last_name#30), coalesce(c_first_name#29, ), isnull(c_first_name#29), coalesce(d_date#26, 1970-01-01), isnull(d_date#26), 5), ENSURE_REQUIREMENTS, [id=#32] -(60) Sort [codegen id : 27] -Input [3]: [c_last_name#32, c_first_name#31, d_date#28] -Arguments: [coalesce(c_last_name#32, ) ASC NULLS FIRST, isnull(c_last_name#32) ASC NULLS FIRST, coalesce(c_first_name#31, ) ASC NULLS FIRST, isnull(c_first_name#31) ASC NULLS FIRST, coalesce(d_date#28, 1970-01-01) ASC NULLS FIRST, isnull(d_date#28) ASC NULLS FIRST], false, 0 +(55) Sort [codegen id : 25] +Input [3]: [c_last_name#30, c_first_name#29, d_date#26] +Arguments: [coalesce(c_last_name#30, ) ASC NULLS FIRST, isnull(c_last_name#30) ASC NULLS FIRST, coalesce(c_first_name#29, ) ASC NULLS FIRST, isnull(c_first_name#29) ASC NULLS FIRST, coalesce(d_date#26, 1970-01-01) ASC NULLS FIRST, isnull(d_date#26) ASC NULLS FIRST], false, 0 -(61) SortMergeJoin [codegen id : 28] +(56) SortMergeJoin [codegen id : 26] Left keys [6]: [coalesce(c_last_name#9, ), isnull(c_last_name#9), coalesce(c_first_name#8, ), isnull(c_first_name#8), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#32, ), isnull(c_last_name#32), coalesce(c_first_name#31, ), isnull(c_first_name#31), coalesce(d_date#28, 1970-01-01), isnull(d_date#28)] +Right keys [6]: [coalesce(c_last_name#30, ), isnull(c_last_name#30), coalesce(c_first_name#29, ), isnull(c_first_name#29), coalesce(d_date#26, 1970-01-01), isnull(d_date#26)] Join condition: None -(62) HashAggregate [codegen id : 28] +(57) Project [codegen id : 26] +Output: [] Input [3]: [c_last_name#9, c_first_name#8, d_date#5] -Keys [3]: [c_last_name#9, c_first_name#8, d_date#5] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#9, c_first_name#8, d_date#5] -(63) Exchange -Input [3]: [c_last_name#9, c_first_name#8, d_date#5] -Arguments: hashpartitioning(c_last_name#9, c_first_name#8, d_date#5, 5), ENSURE_REQUIREMENTS, [id=#35] - -(64) HashAggregate [codegen id : 29] -Input [3]: [c_last_name#9, c_first_name#8, d_date#5] -Keys [3]: [c_last_name#9, c_first_name#8, d_date#5] -Functions: [] -Aggregate Attributes: [] -Results: [] - -(65) HashAggregate [codegen id : 29] +(58) HashAggregate [codegen id : 26] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#36] -Results [1]: [count#37] +Aggregate Attributes [1]: [count#33] +Results [1]: [count#34] -(66) Exchange -Input [1]: [count#37] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#38] +(59) Exchange +Input [1]: [count#34] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#35] -(67) HashAggregate [codegen id : 30] -Input [1]: [count#37] +(60) HashAggregate [codegen id : 27] +Input [1]: [count#34] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#39] -Results [1]: [count(1)#39 AS count(1)#40] +Aggregate Attributes [1]: [count(1)#36] +Results [1]: [count(1)#36 AS count(1)#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 -BroadcastExchange (72) -+- * Project (71) - +- * Filter (70) - +- * ColumnarToRow (69) - +- Scan parquet default.date_dim (68) +BroadcastExchange (65) ++- * Project (64) + +- * Filter (63) + +- * ColumnarToRow (62) + +- Scan parquet default.date_dim (61) -(68) Scan parquet default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#41] +(61) Scan parquet default.date_dim +Output [3]: [d_date_sk#4, d_date#5, d_month_seq#38] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(69) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#41] +(62) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#4, d_date#5, d_month_seq#38] -(70) Filter [codegen id : 1] -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#41] -Condition : (((isnotnull(d_month_seq#41) AND (d_month_seq#41 >= 1200)) AND (d_month_seq#41 <= 1211)) AND isnotnull(d_date_sk#4)) +(63) Filter [codegen id : 1] +Input [3]: [d_date_sk#4, d_date#5, d_month_seq#38] +Condition : (((isnotnull(d_month_seq#38) AND (d_month_seq#38 >= 1200)) AND (d_month_seq#38 <= 1211)) AND isnotnull(d_date_sk#4)) -(71) Project [codegen id : 1] +(64) Project [codegen id : 1] Output [2]: [d_date_sk#4, d_date#5] -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#41] +Input [3]: [d_date_sk#4, d_date#5, d_month_seq#38] -(72) BroadcastExchange +(65) BroadcastExchange Input [2]: [d_date_sk#4, d_date#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] Subquery:2 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#3 -Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#26 IN dynamicpruning#3 +Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/simplified.txt index eda0d4b03f483..cc66a0040ef9a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/simplified.txt @@ -1,135 +1,122 @@ -WholeStageCodegen (30) +WholeStageCodegen (27) HashAggregate [count] [count(1),count(1),count] InputAdapter Exchange #1 - WholeStageCodegen (29) + WholeStageCodegen (26) HashAggregate [count,count] - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (28) - HashAggregate [c_last_name,c_first_name,d_date] - SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - InputAdapter - WholeStageCodegen (19) - Sort [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #3 - WholeStageCodegen (18) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #4 - WholeStageCodegen (17) - HashAggregate [c_last_name,c_first_name,d_date] - SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + Project + SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + InputAdapter + WholeStageCodegen (17) + SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + InputAdapter + WholeStageCodegen (8) + Sort [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #2 + WholeStageCodegen (7) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #3 + WholeStageCodegen (6) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (8) - Sort [c_last_name,c_first_name,d_date] + WholeStageCodegen (3) + Sort [ss_customer_sk] InputAdapter - Exchange [c_last_name,c_first_name,d_date] #5 - WholeStageCodegen (7) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #6 - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - SortMergeJoin [ss_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (3) - Sort [ss_customer_sk] - InputAdapter - Exchange [ss_customer_sk] #7 - WholeStageCodegen (2) - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #8 - WholeStageCodegen (1) - Project [d_date_sk,d_date] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #8 - InputAdapter - WholeStageCodegen (5) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #9 - WholeStageCodegen (4) - Filter [c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] + Exchange [ss_customer_sk] #4 + WholeStageCodegen (2) + Project [ss_customer_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + Project [d_date_sk,d_date] + Filter [d_month_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk,d_date] #5 InputAdapter - WholeStageCodegen (16) - Sort [c_last_name,c_first_name,d_date] + WholeStageCodegen (5) + Sort [c_customer_sk] InputAdapter - Exchange [c_last_name,c_first_name,d_date] #10 - WholeStageCodegen (15) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #11 - WholeStageCodegen (14) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - SortMergeJoin [cs_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (11) - Sort [cs_bill_customer_sk] - InputAdapter - Exchange [cs_bill_customer_sk] #12 - WholeStageCodegen (10) - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #8 - InputAdapter - WholeStageCodegen (13) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #9 - InputAdapter - WholeStageCodegen (27) - Sort [c_last_name,c_first_name,d_date] + Exchange [c_customer_sk] #6 + WholeStageCodegen (4) + Filter [c_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] + InputAdapter + WholeStageCodegen (16) + Sort [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #7 + WholeStageCodegen (15) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #8 + WholeStageCodegen (14) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (11) + Sort [cs_bill_customer_sk] + InputAdapter + Exchange [cs_bill_customer_sk] #9 + WholeStageCodegen (10) + Project [cs_bill_customer_sk,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_date] #5 + InputAdapter + WholeStageCodegen (13) + Sort [c_customer_sk] + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #6 + InputAdapter + WholeStageCodegen (25) + Sort [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #10 + WholeStageCodegen (24) + HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_last_name,c_first_name,d_date] #13 - WholeStageCodegen (26) + Exchange [c_last_name,c_first_name,d_date] #11 + WholeStageCodegen (23) HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #14 - WholeStageCodegen (25) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - SortMergeJoin [ws_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (22) - Sort [ws_bill_customer_sk] - InputAdapter - Exchange [ws_bill_customer_sk] #15 - WholeStageCodegen (21) - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #8 - InputAdapter - WholeStageCodegen (24) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #9 + Project [c_last_name,c_first_name,d_date] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (20) + Sort [ws_bill_customer_sk] + InputAdapter + Exchange [ws_bill_customer_sk] #12 + WholeStageCodegen (19) + Project [ws_bill_customer_sk,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_date] #5 + InputAdapter + WholeStageCodegen (22) + Sort [c_customer_sk] + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #6 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 7193c4f8c57ef..ed2a97704b2f7 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 @@ -1,54 +1,51 @@ == Physical Plan == -* HashAggregate (50) -+- Exchange (49) - +- * HashAggregate (48) - +- * HashAggregate (47) - +- * HashAggregate (46) - +- * BroadcastHashJoin LeftAnti BuildRight (45) - :- * HashAggregate (31) - : +- * HashAggregate (30) - : +- * BroadcastHashJoin LeftAnti BuildRight (29) - : :- * HashAggregate (15) - : : +- Exchange (14) - : : +- * HashAggregate (13) - : : +- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (10) - : : +- * Filter (9) - : : +- * ColumnarToRow (8) - : : +- Scan parquet default.customer (7) - : +- BroadcastExchange (28) - : +- * HashAggregate (27) - : +- Exchange (26) - : +- * HashAggregate (25) - : +- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (21) - : : +- * BroadcastHashJoin Inner BuildRight (20) - : : :- * Filter (18) - : : : +- * ColumnarToRow (17) - : : : +- Scan parquet default.catalog_sales (16) - : : +- ReusedExchange (19) - : +- ReusedExchange (22) - +- BroadcastExchange (44) - +- * HashAggregate (43) - +- Exchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Filter (34) - : : +- * ColumnarToRow (33) - : : +- Scan parquet default.web_sales (32) - : +- ReusedExchange (35) - +- ReusedExchange (38) +* HashAggregate (47) ++- Exchange (46) + +- * HashAggregate (45) + +- * Project (44) + +- * BroadcastHashJoin LeftAnti BuildRight (43) + :- * BroadcastHashJoin LeftAnti BuildRight (29) + : :- * HashAggregate (15) + : : +- Exchange (14) + : : +- * HashAggregate (13) + : : +- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (10) + : : +- * Filter (9) + : : +- * ColumnarToRow (8) + : : +- Scan parquet default.customer (7) + : +- BroadcastExchange (28) + : +- * HashAggregate (27) + : +- Exchange (26) + : +- * HashAggregate (25) + : +- * Project (24) + : +- * BroadcastHashJoin Inner BuildRight (23) + : :- * Project (21) + : : +- * BroadcastHashJoin Inner BuildRight (20) + : : :- * Filter (18) + : : : +- * ColumnarToRow (17) + : : : +- Scan parquet default.catalog_sales (16) + : : +- ReusedExchange (19) + : +- ReusedExchange (22) + +- BroadcastExchange (42) + +- * HashAggregate (41) + +- Exchange (40) + +- * HashAggregate (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * Filter (32) + : : +- * ColumnarToRow (31) + : : +- Scan parquet default.web_sales (30) + : +- ReusedExchange (33) + +- ReusedExchange (36) (1) Scan parquet default.store_sales @@ -66,7 +63,7 @@ Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] Condition : isnotnull(ss_customer_sk#1) -(4) ReusedExchange [Reuses operator id: 55] +(4) ReusedExchange [Reuses operator id: 52] Output [2]: [d_date_sk#4, d_date#5] (5) BroadcastHashJoin [codegen id : 3] @@ -138,7 +135,7 @@ Input [2]: [cs_bill_customer_sk#11, cs_sold_date_sk#12] Input [2]: [cs_bill_customer_sk#11, cs_sold_date_sk#12] Condition : isnotnull(cs_bill_customer_sk#11) -(19) ReusedExchange [Reuses operator id: 55] +(19) ReusedExchange [Reuses operator id: 52] Output [2]: [d_date_sk#13, d_date#14] (20) BroadcastHashJoin [codegen id : 6] @@ -189,21 +186,7 @@ Left keys [6]: [coalesce(c_last_name#8, ), isnull(c_last_name#8), coalesce(c_fir Right keys [6]: [coalesce(c_last_name#17, ), isnull(c_last_name#17), coalesce(c_first_name#16, ), isnull(c_first_name#16), coalesce(d_date#14, 1970-01-01), isnull(d_date#14)] Join condition: None -(30) HashAggregate [codegen id : 12] -Input [3]: [c_last_name#8, c_first_name#7, d_date#5] -Keys [3]: [c_last_name#8, c_first_name#7, d_date#5] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#8, c_first_name#7, d_date#5] - -(31) HashAggregate [codegen id : 12] -Input [3]: [c_last_name#8, c_first_name#7, d_date#5] -Keys [3]: [c_last_name#8, c_first_name#7, d_date#5] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#8, c_first_name#7, d_date#5] - -(32) Scan parquet default.web_sales +(30) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] @@ -211,90 +194,80 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#21), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(33) ColumnarToRow [codegen id : 10] +(31) ColumnarToRow [codegen id : 10] Input [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -(34) Filter [codegen id : 10] +(32) Filter [codegen id : 10] Input [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] Condition : isnotnull(ws_bill_customer_sk#20) -(35) ReusedExchange [Reuses operator id: 55] +(33) ReusedExchange [Reuses operator id: 52] Output [2]: [d_date_sk#22, d_date#23] -(36) BroadcastHashJoin [codegen id : 10] +(34) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ws_sold_date_sk#21] Right keys [1]: [d_date_sk#22] Join condition: None -(37) Project [codegen id : 10] +(35) Project [codegen id : 10] Output [2]: [ws_bill_customer_sk#20, d_date#23] Input [4]: [ws_bill_customer_sk#20, ws_sold_date_sk#21, d_date_sk#22, d_date#23] -(38) ReusedExchange [Reuses operator id: 10] +(36) ReusedExchange [Reuses operator id: 10] Output [3]: [c_customer_sk#24, c_first_name#25, c_last_name#26] -(39) BroadcastHashJoin [codegen id : 10] +(37) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ws_bill_customer_sk#20] Right keys [1]: [c_customer_sk#24] Join condition: None -(40) Project [codegen id : 10] +(38) Project [codegen id : 10] Output [3]: [c_last_name#26, c_first_name#25, d_date#23] Input [5]: [ws_bill_customer_sk#20, d_date#23, c_customer_sk#24, c_first_name#25, c_last_name#26] -(41) HashAggregate [codegen id : 10] +(39) HashAggregate [codegen id : 10] Input [3]: [c_last_name#26, c_first_name#25, d_date#23] Keys [3]: [c_last_name#26, c_first_name#25, d_date#23] Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#26, c_first_name#25, d_date#23] -(42) Exchange +(40) Exchange Input [3]: [c_last_name#26, c_first_name#25, d_date#23] Arguments: hashpartitioning(c_last_name#26, c_first_name#25, d_date#23, 5), ENSURE_REQUIREMENTS, [id=#27] -(43) HashAggregate [codegen id : 11] +(41) HashAggregate [codegen id : 11] Input [3]: [c_last_name#26, c_first_name#25, d_date#23] Keys [3]: [c_last_name#26, c_first_name#25, d_date#23] Functions: [] Aggregate Attributes: [] Results [3]: [c_last_name#26, c_first_name#25, d_date#23] -(44) BroadcastExchange +(42) BroadcastExchange Input [3]: [c_last_name#26, c_first_name#25, d_date#23] 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=#28] -(45) BroadcastHashJoin [codegen id : 12] +(43) BroadcastHashJoin [codegen id : 12] Left keys [6]: [coalesce(c_last_name#8, ), isnull(c_last_name#8), coalesce(c_first_name#7, ), isnull(c_first_name#7), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] Right keys [6]: [coalesce(c_last_name#26, ), isnull(c_last_name#26), coalesce(c_first_name#25, ), isnull(c_first_name#25), coalesce(d_date#23, 1970-01-01), isnull(d_date#23)] Join condition: None -(46) HashAggregate [codegen id : 12] -Input [3]: [c_last_name#8, c_first_name#7, d_date#5] -Keys [3]: [c_last_name#8, c_first_name#7, d_date#5] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#8, c_first_name#7, d_date#5] - -(47) HashAggregate [codegen id : 12] +(44) Project [codegen id : 12] +Output: [] Input [3]: [c_last_name#8, c_first_name#7, d_date#5] -Keys [3]: [c_last_name#8, c_first_name#7, d_date#5] -Functions: [] -Aggregate Attributes: [] -Results: [] -(48) HashAggregate [codegen id : 12] +(45) HashAggregate [codegen id : 12] Input: [] Keys: [] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#29] Results [1]: [count#30] -(49) Exchange +(46) Exchange Input [1]: [count#30] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] -(50) HashAggregate [codegen id : 13] +(47) HashAggregate [codegen id : 13] Input [1]: [count#30] Keys: [] Functions [1]: [count(1)] @@ -304,37 +277,37 @@ Results [1]: [count(1)#32 AS count(1)#33] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 -BroadcastExchange (55) -+- * Project (54) - +- * Filter (53) - +- * ColumnarToRow (52) - +- Scan parquet default.date_dim (51) +BroadcastExchange (52) ++- * Project (51) + +- * Filter (50) + +- * ColumnarToRow (49) + +- Scan parquet default.date_dim (48) -(51) Scan parquet default.date_dim +(48) Scan parquet default.date_dim Output [3]: [d_date_sk#4, d_date#5, d_month_seq#34] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(52) ColumnarToRow [codegen id : 1] +(49) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#4, d_date#5, d_month_seq#34] -(53) Filter [codegen id : 1] +(50) Filter [codegen id : 1] Input [3]: [d_date_sk#4, d_date#5, d_month_seq#34] Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= 1200)) AND (d_month_seq#34 <= 1211)) AND isnotnull(d_date_sk#4)) -(54) Project [codegen id : 1] +(51) Project [codegen id : 1] Output [2]: [d_date_sk#4, d_date#5] Input [3]: [d_date_sk#4, d_date#5, d_month_seq#34] -(55) BroadcastExchange +(52) BroadcastExchange Input [2]: [d_date_sk#4, d_date#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] Subquery:2 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#12 IN dynamicpruning#3 -Subquery:3 Hosting operator id = 32 Hosting Expression = ws_sold_date_sk#21 IN dynamicpruning#3 +Subquery:3 Hosting operator id = 30 Hosting Expression = ws_sold_date_sk#21 IN dynamicpruning#3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt index 7f96f5657836a..34d46c5671774 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt @@ -4,81 +4,78 @@ WholeStageCodegen (13) Exchange #1 WholeStageCodegen (12) HashAggregate [count,count] - HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] + Project + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk,d_date] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Filter [c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #2 + WholeStageCodegen (3) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + Project [d_date_sk,d_date] + Filter [d_month_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk,d_date] #3 InputAdapter - Exchange [c_last_name,c_first_name,d_date] #6 - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + BroadcastExchange #4 + WholeStageCodegen (2) + Filter [c_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (11) + BroadcastExchange #5 + WholeStageCodegen (7) HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (10) + Exchange [c_last_name,c_first_name,d_date] #6 + WholeStageCodegen (6) HashAggregate [c_last_name,c_first_name,d_date] Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_bill_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk,d_date] #3 InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (11) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #8 + WholeStageCodegen (10) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + Project [ws_bill_customer_sk,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt index ae613fa051425..92b80b4085c67 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt @@ -1,106 +1,103 @@ == Physical Plan == -TakeOrderedAndProject (102) -+- * BroadcastHashJoin Inner BuildRight (101) - :- * Filter (81) - : +- * HashAggregate (80) - : +- Exchange (79) - : +- * HashAggregate (78) - : +- * Project (77) - : +- * BroadcastHashJoin Inner BuildRight (76) - : :- * Project (66) - : : +- * BroadcastHashJoin Inner BuildRight (65) - : : :- * SortMergeJoin LeftSemi (63) +TakeOrderedAndProject (99) ++- * BroadcastHashJoin Inner BuildRight (98) + :- * Filter (78) + : +- * HashAggregate (77) + : +- Exchange (76) + : +- * HashAggregate (75) + : +- * Project (74) + : +- * BroadcastHashJoin Inner BuildRight (73) + : :- * Project (63) + : : +- * BroadcastHashJoin Inner BuildRight (62) + : : :- * SortMergeJoin LeftSemi (60) : : : :- * Sort (5) : : : : +- Exchange (4) : : : : +- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.store_sales (1) - : : : +- * Sort (62) - : : : +- Exchange (61) - : : : +- * Project (60) - : : : +- * BroadcastHashJoin Inner BuildRight (59) + : : : +- * Sort (59) + : : : +- Exchange (58) + : : : +- * Project (57) + : : : +- * BroadcastHashJoin Inner BuildRight (56) : : : :- * Filter (8) : : : : +- * ColumnarToRow (7) : : : : +- Scan parquet default.item (6) - : : : +- BroadcastExchange (58) - : : : +- * HashAggregate (57) - : : : +- Exchange (56) - : : : +- * HashAggregate (55) - : : : +- * SortMergeJoin LeftSemi (54) - : : : :- * Sort (42) - : : : : +- Exchange (41) - : : : : +- * HashAggregate (40) - : : : : +- Exchange (39) - : : : : +- * HashAggregate (38) - : : : : +- * Project (37) - : : : : +- * BroadcastHashJoin Inner BuildRight (36) - : : : : :- * Project (14) - : : : : : +- * BroadcastHashJoin Inner BuildRight (13) - : : : : : :- * Filter (11) - : : : : : : +- * ColumnarToRow (10) - : : : : : : +- Scan parquet default.store_sales (9) - : : : : : +- ReusedExchange (12) - : : : : +- BroadcastExchange (35) - : : : : +- * SortMergeJoin LeftSemi (34) - : : : : :- * Sort (19) - : : : : : +- Exchange (18) - : : : : : +- * Filter (17) - : : : : : +- * ColumnarToRow (16) - : : : : : +- Scan parquet default.item (15) - : : : : +- * Sort (33) - : : : : +- Exchange (32) - : : : : +- * Project (31) - : : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : : :- * Project (25) - : : : : : +- * BroadcastHashJoin Inner BuildRight (24) - : : : : : :- * Filter (22) - : : : : : : +- * ColumnarToRow (21) - : : : : : : +- Scan parquet default.catalog_sales (20) - : : : : : +- ReusedExchange (23) - : : : : +- BroadcastExchange (29) - : : : : +- * Filter (28) - : : : : +- * ColumnarToRow (27) - : : : : +- Scan parquet default.item (26) - : : : +- * Sort (53) - : : : +- Exchange (52) - : : : +- * Project (51) - : : : +- * BroadcastHashJoin Inner BuildRight (50) - : : : :- * Project (48) - : : : : +- * BroadcastHashJoin Inner BuildRight (47) - : : : : :- * Filter (45) - : : : : : +- * ColumnarToRow (44) - : : : : : +- Scan parquet default.web_sales (43) - : : : : +- ReusedExchange (46) - : : : +- ReusedExchange (49) - : : +- ReusedExchange (64) - : +- BroadcastExchange (75) - : +- * SortMergeJoin LeftSemi (74) - : :- * Sort (71) - : : +- Exchange (70) - : : +- * Filter (69) - : : +- * ColumnarToRow (68) - : : +- Scan parquet default.item (67) - : +- * Sort (73) - : +- ReusedExchange (72) - +- BroadcastExchange (100) - +- * Filter (99) - +- * HashAggregate (98) - +- Exchange (97) - +- * HashAggregate (96) - +- * Project (95) - +- * BroadcastHashJoin Inner BuildRight (94) - :- * Project (92) - : +- * BroadcastHashJoin Inner BuildRight (91) - : :- * SortMergeJoin LeftSemi (89) - : : :- * Sort (86) - : : : +- Exchange (85) - : : : +- * Filter (84) - : : : +- * ColumnarToRow (83) - : : : +- Scan parquet default.store_sales (82) - : : +- * Sort (88) - : : +- ReusedExchange (87) - : +- ReusedExchange (90) - +- ReusedExchange (93) + : : : +- BroadcastExchange (55) + : : : +- * SortMergeJoin LeftSemi (54) + : : : :- * Sort (42) + : : : : +- Exchange (41) + : : : : +- * HashAggregate (40) + : : : : +- Exchange (39) + : : : : +- * HashAggregate (38) + : : : : +- * Project (37) + : : : : +- * BroadcastHashJoin Inner BuildRight (36) + : : : : :- * Project (14) + : : : : : +- * BroadcastHashJoin Inner BuildRight (13) + : : : : : :- * Filter (11) + : : : : : : +- * ColumnarToRow (10) + : : : : : : +- Scan parquet default.store_sales (9) + : : : : : +- ReusedExchange (12) + : : : : +- BroadcastExchange (35) + : : : : +- * SortMergeJoin LeftSemi (34) + : : : : :- * Sort (19) + : : : : : +- Exchange (18) + : : : : : +- * Filter (17) + : : : : : +- * ColumnarToRow (16) + : : : : : +- Scan parquet default.item (15) + : : : : +- * Sort (33) + : : : : +- Exchange (32) + : : : : +- * Project (31) + : : : : +- * BroadcastHashJoin Inner BuildRight (30) + : : : : :- * Project (25) + : : : : : +- * BroadcastHashJoin Inner BuildRight (24) + : : : : : :- * Filter (22) + : : : : : : +- * ColumnarToRow (21) + : : : : : : +- Scan parquet default.catalog_sales (20) + : : : : : +- ReusedExchange (23) + : : : : +- BroadcastExchange (29) + : : : : +- * Filter (28) + : : : : +- * ColumnarToRow (27) + : : : : +- Scan parquet default.item (26) + : : : +- * Sort (53) + : : : +- Exchange (52) + : : : +- * Project (51) + : : : +- * BroadcastHashJoin Inner BuildRight (50) + : : : :- * Project (48) + : : : : +- * BroadcastHashJoin Inner BuildRight (47) + : : : : :- * Filter (45) + : : : : : +- * ColumnarToRow (44) + : : : : : +- Scan parquet default.web_sales (43) + : : : : +- ReusedExchange (46) + : : : +- ReusedExchange (49) + : : +- ReusedExchange (61) + : +- BroadcastExchange (72) + : +- * SortMergeJoin LeftSemi (71) + : :- * Sort (68) + : : +- Exchange (67) + : : +- * Filter (66) + : : +- * ColumnarToRow (65) + : : +- Scan parquet default.item (64) + : +- * Sort (70) + : +- ReusedExchange (69) + +- BroadcastExchange (97) + +- * Filter (96) + +- * HashAggregate (95) + +- Exchange (94) + +- * HashAggregate (93) + +- * Project (92) + +- * BroadcastHashJoin Inner BuildRight (91) + :- * Project (89) + : +- * BroadcastHashJoin Inner BuildRight (88) + : :- * SortMergeJoin LeftSemi (86) + : : :- * Sort (83) + : : : +- Exchange (82) + : : : +- * Filter (81) + : : : +- * ColumnarToRow (80) + : : : +- Scan parquet default.store_sales (79) + : : +- * Sort (85) + : : +- ReusedExchange (84) + : +- ReusedExchange (87) + +- ReusedExchange (90) (1) Scan parquet default.store_sales @@ -133,10 +130,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)) @@ -155,7 +152,7 @@ Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] Condition : isnotnull(ss_item_sk#11) -(12) ReusedExchange [Reuses operator id: 135] +(12) ReusedExchange [Reuses operator id: 132] Output [1]: [d_date_sk#14] (13) BroadcastHashJoin [codegen id : 11] @@ -204,7 +201,7 @@ Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] Condition : isnotnull(cs_item_sk#20) -(23) ReusedExchange [Reuses operator id: 135] +(23) ReusedExchange [Reuses operator id: 132] Output [1]: [d_date_sk#22] (24) BroadcastHashJoin [codegen id : 8] @@ -310,7 +307,7 @@ Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] Condition : isnotnull(ws_item_sk#35) -(46) ReusedExchange [Reuses operator id: 135] +(46) ReusedExchange [Reuses operator id: 132] Output [1]: [d_date_sk#37] (47) BroadcastHashJoin [codegen id : 16] @@ -347,485 +344,467 @@ Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id 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 -(55) HashAggregate [codegen id : 18] +(55) BroadcastExchange 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#30, class_id#31, category_id#32] - -(56) 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] - -(57) HashAggregate [codegen id : 19] -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#30, class_id#31, category_id#32] - -(58) 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] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#43] -(59) BroadcastHashJoin [codegen id : 20] +(56) 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] Join condition: None -(60) Project [codegen id : 20] -Output [1]: [i_item_sk#7 AS ss_item_sk#45] +(57) Project [codegen id : 19] +Output [1]: [i_item_sk#7 AS ss_item_sk#44] 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] -(61) Exchange -Input [1]: [ss_item_sk#45] -Arguments: hashpartitioning(ss_item_sk#45, 5), ENSURE_REQUIREMENTS, [id=#46] +(58) Exchange +Input [1]: [ss_item_sk#44] +Arguments: hashpartitioning(ss_item_sk#44, 5), ENSURE_REQUIREMENTS, [id=#45] -(62) Sort [codegen id : 21] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +(59) Sort [codegen id : 20] +Input [1]: [ss_item_sk#44] +Arguments: [ss_item_sk#44 ASC NULLS FIRST], false, 0 -(63) SortMergeJoin [codegen id : 45] +(60) SortMergeJoin [codegen id : 43] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#45] +Right keys [1]: [ss_item_sk#44] Join condition: None -(64) ReusedExchange [Reuses operator id: 126] -Output [1]: [d_date_sk#47] +(61) ReusedExchange [Reuses operator id: 123] +Output [1]: [d_date_sk#46] -(65) BroadcastHashJoin [codegen id : 45] +(62) BroadcastHashJoin [codegen id : 43] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#47] +Right keys [1]: [d_date_sk#46] Join condition: None -(66) Project [codegen id : 45] +(63) Project [codegen id : 43] 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#47] +Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#46] -(67) Scan parquet default.item -Output [4]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51] +(64) Scan parquet default.item +Output [4]: [i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] 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 -(68) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51] +(65) ColumnarToRow [codegen id : 22] +Input [4]: [i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] -(69) Filter [codegen id : 23] -Input [4]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51] -Condition : (((isnotnull(i_item_sk#48) AND isnotnull(i_brand_id#49)) AND isnotnull(i_class_id#50)) AND isnotnull(i_category_id#51)) +(66) Filter [codegen id : 22] +Input [4]: [i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] +Condition : (((isnotnull(i_item_sk#47) AND isnotnull(i_brand_id#48)) AND isnotnull(i_class_id#49)) AND isnotnull(i_category_id#50)) -(70) Exchange -Input [4]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51] -Arguments: hashpartitioning(i_item_sk#48, 5), ENSURE_REQUIREMENTS, [id=#52] +(67) Exchange +Input [4]: [i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] +Arguments: hashpartitioning(i_item_sk#47, 5), ENSURE_REQUIREMENTS, [id=#51] -(71) Sort [codegen id : 24] -Input [4]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51] -Arguments: [i_item_sk#48 ASC NULLS FIRST], false, 0 +(68) Sort [codegen id : 23] +Input [4]: [i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] +Arguments: [i_item_sk#47 ASC NULLS FIRST], false, 0 -(72) ReusedExchange [Reuses operator id: 61] -Output [1]: [ss_item_sk#45] +(69) ReusedExchange [Reuses operator id: 58] +Output [1]: [ss_item_sk#44] -(73) Sort [codegen id : 43] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +(70) Sort [codegen id : 41] +Input [1]: [ss_item_sk#44] +Arguments: [ss_item_sk#44 ASC NULLS FIRST], false, 0 -(74) SortMergeJoin [codegen id : 44] -Left keys [1]: [i_item_sk#48] -Right keys [1]: [ss_item_sk#45] +(71) SortMergeJoin [codegen id : 42] +Left keys [1]: [i_item_sk#47] +Right keys [1]: [ss_item_sk#44] Join condition: None -(75) BroadcastExchange -Input [4]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#53] +(72) BroadcastExchange +Input [4]: [i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#52] -(76) BroadcastHashJoin [codegen id : 45] +(73) BroadcastHashJoin [codegen id : 43] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#48] +Right keys [1]: [i_item_sk#47] Join condition: None -(77) Project [codegen id : 45] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#49, i_class_id#50, i_category_id#51] -Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51] +(74) Project [codegen id : 43] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#48, i_class_id#49, i_category_id#50] +Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] -(78) HashAggregate [codegen id : 45] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#49, i_class_id#50, i_category_id#51] -Keys [3]: [i_brand_id#49, i_class_id#50, i_category_id#51] +(75) HashAggregate [codegen id : 43] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#48, i_class_id#49, i_category_id#50] +Keys [3]: [i_brand_id#48, i_class_id#49, i_category_id#50] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)] -Aggregate Attributes [3]: [sum#54, isEmpty#55, count#56] -Results [6]: [i_brand_id#49, i_class_id#50, i_category_id#51, sum#57, isEmpty#58, count#59] +Aggregate Attributes [3]: [sum#53, isEmpty#54, count#55] +Results [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum#56, isEmpty#57, count#58] -(79) Exchange -Input [6]: [i_brand_id#49, i_class_id#50, i_category_id#51, sum#57, isEmpty#58, count#59] -Arguments: hashpartitioning(i_brand_id#49, i_class_id#50, i_category_id#51, 5), ENSURE_REQUIREMENTS, [id=#60] +(76) Exchange +Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum#56, isEmpty#57, count#58] +Arguments: hashpartitioning(i_brand_id#48, i_class_id#49, i_category_id#50, 5), ENSURE_REQUIREMENTS, [id=#59] -(80) HashAggregate [codegen id : 92] -Input [6]: [i_brand_id#49, i_class_id#50, i_category_id#51, sum#57, isEmpty#58, count#59] -Keys [3]: [i_brand_id#49, i_class_id#50, i_category_id#51] +(77) HashAggregate [codegen id : 88] +Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum#56, isEmpty#57, count#58] +Keys [3]: [i_brand_id#48, i_class_id#49, i_category_id#50] Functions [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2))), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#61, count(1)#62] -Results [6]: [store AS channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#61 AS sales#64, count(1)#62 AS number_sales#65] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#60, count(1)#61] +Results [6]: [store AS channel#62, i_brand_id#48, i_class_id#49, i_category_id#50, sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#60 AS sales#63, count(1)#61 AS number_sales#64] -(81) Filter [codegen id : 92] -Input [6]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sales#64, number_sales#65] -Condition : (isnotnull(sales#64) AND (cast(sales#64 as decimal(32,6)) > cast(Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +(78) Filter [codegen id : 88] +Input [6]: [channel#62, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64] +Condition : (isnotnull(sales#63) AND (cast(sales#63 as decimal(32,6)) > cast(Subquery scalar-subquery#65, [id=#66] as decimal(32,6)))) -(82) Scan parquet default.store_sales -Output [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] +(79) Scan parquet default.store_sales +Output [4]: [ss_item_sk#67, ss_quantity#68, ss_list_price#69, ss_sold_date_sk#70] 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#70), dynamicpruningexpression(ss_sold_date_sk#70 IN dynamicpruning#71)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(83) ColumnarToRow [codegen id : 46] -Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] +(80) ColumnarToRow [codegen id : 44] +Input [4]: [ss_item_sk#67, ss_quantity#68, ss_list_price#69, ss_sold_date_sk#70] -(84) Filter [codegen id : 46] -Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] -Condition : isnotnull(ss_item_sk#68) +(81) Filter [codegen id : 44] +Input [4]: [ss_item_sk#67, ss_quantity#68, ss_list_price#69, ss_sold_date_sk#70] +Condition : isnotnull(ss_item_sk#67) -(85) 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] +(82) Exchange +Input [4]: [ss_item_sk#67, ss_quantity#68, ss_list_price#69, ss_sold_date_sk#70] +Arguments: hashpartitioning(ss_item_sk#67, 5), ENSURE_REQUIREMENTS, [id=#72] -(86) Sort [codegen id : 47] -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 +(83) Sort [codegen id : 45] +Input [4]: [ss_item_sk#67, ss_quantity#68, ss_list_price#69, ss_sold_date_sk#70] +Arguments: [ss_item_sk#67 ASC NULLS FIRST], false, 0 -(87) ReusedExchange [Reuses operator id: 61] -Output [1]: [ss_item_sk#45] +(84) ReusedExchange [Reuses operator id: 58] +Output [1]: [ss_item_sk#44] -(88) Sort [codegen id : 66] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +(85) Sort [codegen id : 63] +Input [1]: [ss_item_sk#44] +Arguments: [ss_item_sk#44 ASC NULLS FIRST], false, 0 -(89) SortMergeJoin [codegen id : 90] -Left keys [1]: [ss_item_sk#68] -Right keys [1]: [ss_item_sk#45] +(86) SortMergeJoin [codegen id : 86] +Left keys [1]: [ss_item_sk#67] +Right keys [1]: [ss_item_sk#44] Join condition: None -(90) ReusedExchange [Reuses operator id: 140] -Output [1]: [d_date_sk#74] +(87) ReusedExchange [Reuses operator id: 137] +Output [1]: [d_date_sk#73] -(91) BroadcastHashJoin [codegen id : 90] -Left keys [1]: [ss_sold_date_sk#71] -Right keys [1]: [d_date_sk#74] +(88) BroadcastHashJoin [codegen id : 86] +Left keys [1]: [ss_sold_date_sk#70] +Right keys [1]: [d_date_sk#73] Join condition: None -(92) Project [codegen id : 90] -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] +(89) Project [codegen id : 86] +Output [3]: [ss_item_sk#67, ss_quantity#68, ss_list_price#69] +Input [5]: [ss_item_sk#67, ss_quantity#68, ss_list_price#69, ss_sold_date_sk#70, d_date_sk#73] -(93) ReusedExchange [Reuses operator id: 75] -Output [4]: [i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] +(90) ReusedExchange [Reuses operator id: 72] +Output [4]: [i_item_sk#74, i_brand_id#75, i_class_id#76, i_category_id#77] -(94) BroadcastHashJoin [codegen id : 90] -Left keys [1]: [ss_item_sk#68] -Right keys [1]: [i_item_sk#75] +(91) BroadcastHashJoin [codegen id : 86] +Left keys [1]: [ss_item_sk#67] +Right keys [1]: [i_item_sk#74] Join condition: None -(95) Project [codegen id : 90] -Output [5]: [ss_quantity#69, ss_list_price#70, i_brand_id#76, i_class_id#77, i_category_id#78] -Input [7]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] - -(96) HashAggregate [codegen id : 90] -Input [5]: [ss_quantity#69, ss_list_price#70, 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(ss_quantity#69 as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)] -Aggregate Attributes [3]: [sum#79, isEmpty#80, count#81] -Results [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#82, isEmpty#83, count#84] - -(97) Exchange -Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#82, isEmpty#83, count#84] -Arguments: hashpartitioning(i_brand_id#76, i_class_id#77, i_category_id#78, 5), ENSURE_REQUIREMENTS, [id=#85] - -(98) HashAggregate [codegen id : 91] -Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#82, isEmpty#83, count#84] -Keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#69 as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2))), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#69 as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2)))#86, count(1)#87] -Results [6]: [store AS channel#88, i_brand_id#76, i_class_id#77, i_category_id#78, sum(CheckOverflow((promote_precision(cast(ss_quantity#69 as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2)))#86 AS sales#89, count(1)#87 AS number_sales#90] - -(99) Filter [codegen id : 91] -Input [6]: [channel#88, i_brand_id#76, i_class_id#77, i_category_id#78, sales#89, number_sales#90] -Condition : (isnotnull(sales#89) AND (cast(sales#89 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) - -(100) BroadcastExchange -Input [6]: [channel#88, i_brand_id#76, i_class_id#77, i_category_id#78, sales#89, number_sales#90] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#91] - -(101) BroadcastHashJoin [codegen id : 92] -Left keys [3]: [i_brand_id#49, i_class_id#50, i_category_id#51] -Right keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] +(92) Project [codegen id : 86] +Output [5]: [ss_quantity#68, ss_list_price#69, i_brand_id#75, i_class_id#76, i_category_id#77] +Input [7]: [ss_item_sk#67, ss_quantity#68, ss_list_price#69, i_item_sk#74, i_brand_id#75, i_class_id#76, i_category_id#77] + +(93) HashAggregate [codegen id : 86] +Input [5]: [ss_quantity#68, ss_list_price#69, i_brand_id#75, i_class_id#76, i_category_id#77] +Keys [3]: [i_brand_id#75, i_class_id#76, i_category_id#77] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(ss_quantity#68 as decimal(12,2))) * promote_precision(cast(ss_list_price#69 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)] +Aggregate Attributes [3]: [sum#78, isEmpty#79, count#80] +Results [6]: [i_brand_id#75, i_class_id#76, i_category_id#77, sum#81, isEmpty#82, count#83] + +(94) Exchange +Input [6]: [i_brand_id#75, i_class_id#76, i_category_id#77, sum#81, isEmpty#82, count#83] +Arguments: hashpartitioning(i_brand_id#75, i_class_id#76, i_category_id#77, 5), ENSURE_REQUIREMENTS, [id=#84] + +(95) HashAggregate [codegen id : 87] +Input [6]: [i_brand_id#75, i_class_id#76, i_category_id#77, sum#81, isEmpty#82, count#83] +Keys [3]: [i_brand_id#75, i_class_id#76, i_category_id#77] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#68 as decimal(12,2))) * promote_precision(cast(ss_list_price#69 as decimal(12,2)))), DecimalType(18,2))), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#68 as decimal(12,2))) * promote_precision(cast(ss_list_price#69 as decimal(12,2)))), DecimalType(18,2)))#85, count(1)#86] +Results [6]: [store AS channel#87, i_brand_id#75, i_class_id#76, i_category_id#77, sum(CheckOverflow((promote_precision(cast(ss_quantity#68 as decimal(12,2))) * promote_precision(cast(ss_list_price#69 as decimal(12,2)))), DecimalType(18,2)))#85 AS sales#88, count(1)#86 AS number_sales#89] + +(96) Filter [codegen id : 87] +Input [6]: [channel#87, i_brand_id#75, i_class_id#76, i_category_id#77, sales#88, number_sales#89] +Condition : (isnotnull(sales#88) AND (cast(sales#88 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#65, [id=#66] as decimal(32,6)))) + +(97) BroadcastExchange +Input [6]: [channel#87, i_brand_id#75, i_class_id#76, i_category_id#77, sales#88, number_sales#89] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#90] + +(98) BroadcastHashJoin [codegen id : 88] +Left keys [3]: [i_brand_id#48, i_class_id#49, i_category_id#50] +Right keys [3]: [i_brand_id#75, i_class_id#76, i_category_id#77] Join condition: None -(102) TakeOrderedAndProject -Input [12]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sales#64, number_sales#65, channel#88, i_brand_id#76, i_class_id#77, i_category_id#78, sales#89, number_sales#90] -Arguments: 100, [i_brand_id#49 ASC NULLS FIRST, i_class_id#50 ASC NULLS FIRST, i_category_id#51 ASC NULLS FIRST], [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sales#64, number_sales#65, channel#88, i_brand_id#76, i_class_id#77, i_category_id#78, sales#89, number_sales#90] +(99) TakeOrderedAndProject +Input [12]: [channel#62, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64, channel#87, i_brand_id#75, i_class_id#76, i_category_id#77, sales#88, number_sales#89] +Arguments: 100, [i_brand_id#48 ASC NULLS FIRST, i_class_id#49 ASC NULLS FIRST, i_category_id#50 ASC NULLS FIRST], [channel#62, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64, channel#87, i_brand_id#75, i_class_id#76, i_category_id#77, sales#88, number_sales#89] ===== Subqueries ===== -Subquery:1 Hosting operator id = 81 Hosting Expression = Subquery scalar-subquery#66, [id=#67] -* HashAggregate (121) -+- Exchange (120) - +- * HashAggregate (119) - +- Union (118) - :- * Project (107) - : +- * BroadcastHashJoin Inner BuildRight (106) - : :- * ColumnarToRow (104) - : : +- Scan parquet default.store_sales (103) - : +- ReusedExchange (105) - :- * Project (112) - : +- * BroadcastHashJoin Inner BuildRight (111) - : :- * ColumnarToRow (109) - : : +- Scan parquet default.catalog_sales (108) - : +- ReusedExchange (110) - +- * Project (117) - +- * BroadcastHashJoin Inner BuildRight (116) - :- * ColumnarToRow (114) - : +- Scan parquet default.web_sales (113) - +- ReusedExchange (115) - - -(103) Scan parquet default.store_sales -Output [3]: [ss_quantity#92, ss_list_price#93, ss_sold_date_sk#94] +Subquery:1 Hosting operator id = 78 Hosting Expression = Subquery scalar-subquery#65, [id=#66] +* HashAggregate (118) ++- Exchange (117) + +- * HashAggregate (116) + +- Union (115) + :- * Project (104) + : +- * BroadcastHashJoin Inner BuildRight (103) + : :- * ColumnarToRow (101) + : : +- Scan parquet default.store_sales (100) + : +- ReusedExchange (102) + :- * Project (109) + : +- * BroadcastHashJoin Inner BuildRight (108) + : :- * ColumnarToRow (106) + : : +- Scan parquet default.catalog_sales (105) + : +- ReusedExchange (107) + +- * Project (114) + +- * BroadcastHashJoin Inner BuildRight (113) + :- * ColumnarToRow (111) + : +- Scan parquet default.web_sales (110) + +- ReusedExchange (112) + + +(100) Scan parquet default.store_sales +Output [3]: [ss_quantity#91, ss_list_price#92, ss_sold_date_sk#93] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#94), dynamicpruningexpression(ss_sold_date_sk#94 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ss_sold_date_sk#93), dynamicpruningexpression(ss_sold_date_sk#93 IN dynamicpruning#13)] ReadSchema: struct -(104) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#92, ss_list_price#93, ss_sold_date_sk#94] +(101) ColumnarToRow [codegen id : 2] +Input [3]: [ss_quantity#91, ss_list_price#92, ss_sold_date_sk#93] -(105) ReusedExchange [Reuses operator id: 135] -Output [1]: [d_date_sk#95] +(102) ReusedExchange [Reuses operator id: 132] +Output [1]: [d_date_sk#94] -(106) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#94] -Right keys [1]: [d_date_sk#95] +(103) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#93] +Right keys [1]: [d_date_sk#94] Join condition: None -(107) Project [codegen id : 2] -Output [2]: [ss_quantity#92 AS quantity#96, ss_list_price#93 AS list_price#97] -Input [4]: [ss_quantity#92, ss_list_price#93, ss_sold_date_sk#94, d_date_sk#95] +(104) Project [codegen id : 2] +Output [2]: [ss_quantity#91 AS quantity#95, ss_list_price#92 AS list_price#96] +Input [4]: [ss_quantity#91, ss_list_price#92, ss_sold_date_sk#93, d_date_sk#94] -(108) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#98, cs_list_price#99, cs_sold_date_sk#100] +(105) Scan parquet default.catalog_sales +Output [3]: [cs_quantity#97, cs_list_price#98, cs_sold_date_sk#99] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#100), dynamicpruningexpression(cs_sold_date_sk#100 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(cs_sold_date_sk#99), dynamicpruningexpression(cs_sold_date_sk#99 IN dynamicpruning#13)] ReadSchema: struct -(109) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#98, cs_list_price#99, cs_sold_date_sk#100] +(106) ColumnarToRow [codegen id : 4] +Input [3]: [cs_quantity#97, cs_list_price#98, cs_sold_date_sk#99] -(110) ReusedExchange [Reuses operator id: 135] -Output [1]: [d_date_sk#101] +(107) ReusedExchange [Reuses operator id: 132] +Output [1]: [d_date_sk#100] -(111) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#100] -Right keys [1]: [d_date_sk#101] +(108) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_sold_date_sk#99] +Right keys [1]: [d_date_sk#100] Join condition: None -(112) Project [codegen id : 4] -Output [2]: [cs_quantity#98 AS quantity#102, cs_list_price#99 AS list_price#103] -Input [4]: [cs_quantity#98, cs_list_price#99, cs_sold_date_sk#100, d_date_sk#101] +(109) Project [codegen id : 4] +Output [2]: [cs_quantity#97 AS quantity#101, cs_list_price#98 AS list_price#102] +Input [4]: [cs_quantity#97, cs_list_price#98, cs_sold_date_sk#99, d_date_sk#100] -(113) Scan parquet default.web_sales -Output [3]: [ws_quantity#104, ws_list_price#105, ws_sold_date_sk#106] +(110) Scan parquet default.web_sales +Output [3]: [ws_quantity#103, ws_list_price#104, ws_sold_date_sk#105] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#106), dynamicpruningexpression(ws_sold_date_sk#106 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ws_sold_date_sk#105), dynamicpruningexpression(ws_sold_date_sk#105 IN dynamicpruning#13)] ReadSchema: struct -(114) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#104, ws_list_price#105, ws_sold_date_sk#106] +(111) ColumnarToRow [codegen id : 6] +Input [3]: [ws_quantity#103, ws_list_price#104, ws_sold_date_sk#105] -(115) ReusedExchange [Reuses operator id: 135] -Output [1]: [d_date_sk#107] +(112) ReusedExchange [Reuses operator id: 132] +Output [1]: [d_date_sk#106] -(116) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#106] -Right keys [1]: [d_date_sk#107] +(113) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#105] +Right keys [1]: [d_date_sk#106] Join condition: None -(117) Project [codegen id : 6] -Output [2]: [ws_quantity#104 AS quantity#108, ws_list_price#105 AS list_price#109] -Input [4]: [ws_quantity#104, ws_list_price#105, ws_sold_date_sk#106, d_date_sk#107] +(114) Project [codegen id : 6] +Output [2]: [ws_quantity#103 AS quantity#107, ws_list_price#104 AS list_price#108] +Input [4]: [ws_quantity#103, ws_list_price#104, ws_sold_date_sk#105, d_date_sk#106] -(118) Union +(115) Union -(119) HashAggregate [codegen id : 7] -Input [2]: [quantity#96, list_price#97] +(116) HashAggregate [codegen id : 7] +Input [2]: [quantity#95, list_price#96] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(quantity#96 as decimal(12,2))) * promote_precision(cast(list_price#97 as decimal(12,2)))), DecimalType(18,2)))] -Aggregate Attributes [2]: [sum#110, count#111] -Results [2]: [sum#112, count#113] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(quantity#95 as decimal(12,2))) * promote_precision(cast(list_price#96 as decimal(12,2)))), DecimalType(18,2)))] +Aggregate Attributes [2]: [sum#109, count#110] +Results [2]: [sum#111, count#112] -(120) Exchange -Input [2]: [sum#112, count#113] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#114] +(117) Exchange +Input [2]: [sum#111, count#112] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#113] -(121) HashAggregate [codegen id : 8] -Input [2]: [sum#112, count#113] +(118) HashAggregate [codegen id : 8] +Input [2]: [sum#111, count#112] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(quantity#96 as decimal(12,2))) * promote_precision(cast(list_price#97 as decimal(12,2)))), DecimalType(18,2)))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(quantity#96 as decimal(12,2))) * promote_precision(cast(list_price#97 as decimal(12,2)))), DecimalType(18,2)))#115] -Results [1]: [avg(CheckOverflow((promote_precision(cast(quantity#96 as decimal(12,2))) * promote_precision(cast(list_price#97 as decimal(12,2)))), DecimalType(18,2)))#115 AS average_sales#116] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(quantity#95 as decimal(12,2))) * promote_precision(cast(list_price#96 as decimal(12,2)))), DecimalType(18,2)))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(quantity#95 as decimal(12,2))) * promote_precision(cast(list_price#96 as decimal(12,2)))), DecimalType(18,2)))#114] +Results [1]: [avg(CheckOverflow((promote_precision(cast(quantity#95 as decimal(12,2))) * promote_precision(cast(list_price#96 as decimal(12,2)))), DecimalType(18,2)))#114 AS average_sales#115] -Subquery:2 Hosting operator id = 103 Hosting Expression = ss_sold_date_sk#94 IN dynamicpruning#13 +Subquery:2 Hosting operator id = 100 Hosting Expression = ss_sold_date_sk#93 IN dynamicpruning#13 -Subquery:3 Hosting operator id = 108 Hosting Expression = cs_sold_date_sk#100 IN dynamicpruning#13 +Subquery:3 Hosting operator id = 105 Hosting Expression = cs_sold_date_sk#99 IN dynamicpruning#13 -Subquery:4 Hosting operator id = 113 Hosting Expression = ws_sold_date_sk#106 IN dynamicpruning#13 +Subquery:4 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#105 IN dynamicpruning#13 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (126) -+- * Project (125) - +- * Filter (124) - +- * ColumnarToRow (123) - +- Scan parquet default.date_dim (122) +BroadcastExchange (123) ++- * Project (122) + +- * Filter (121) + +- * ColumnarToRow (120) + +- Scan parquet default.date_dim (119) -(122) Scan parquet default.date_dim -Output [2]: [d_date_sk#47, d_week_seq#117] +(119) Scan parquet default.date_dim +Output [2]: [d_date_sk#46, d_week_seq#116] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(123) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#47, d_week_seq#117] +(120) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#46, d_week_seq#116] -(124) Filter [codegen id : 1] -Input [2]: [d_date_sk#47, d_week_seq#117] -Condition : ((isnotnull(d_week_seq#117) AND (d_week_seq#117 = Subquery scalar-subquery#118, [id=#119])) AND isnotnull(d_date_sk#47)) +(121) Filter [codegen id : 1] +Input [2]: [d_date_sk#46, d_week_seq#116] +Condition : ((isnotnull(d_week_seq#116) AND (d_week_seq#116 = Subquery scalar-subquery#117, [id=#118])) AND isnotnull(d_date_sk#46)) -(125) Project [codegen id : 1] -Output [1]: [d_date_sk#47] -Input [2]: [d_date_sk#47, d_week_seq#117] +(122) Project [codegen id : 1] +Output [1]: [d_date_sk#46] +Input [2]: [d_date_sk#46, d_week_seq#116] -(126) BroadcastExchange -Input [1]: [d_date_sk#47] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#120] +(123) BroadcastExchange +Input [1]: [d_date_sk#46] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#119] -Subquery:6 Hosting operator id = 124 Hosting Expression = Subquery scalar-subquery#118, [id=#119] -* Project (130) -+- * Filter (129) - +- * ColumnarToRow (128) - +- Scan parquet default.date_dim (127) +Subquery:6 Hosting operator id = 121 Hosting Expression = Subquery scalar-subquery#117, [id=#118] +* Project (127) ++- * Filter (126) + +- * ColumnarToRow (125) + +- Scan parquet default.date_dim (124) -(127) Scan parquet default.date_dim -Output [4]: [d_week_seq#121, d_year#122, d_moy#123, d_dom#124] +(124) Scan parquet default.date_dim +Output [4]: [d_week_seq#120, d_year#121, d_moy#122, d_dom#123] 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#121, d_year#122, d_moy#123, d_dom#124] +(125) ColumnarToRow [codegen id : 1] +Input [4]: [d_week_seq#120, d_year#121, d_moy#122, d_dom#123] -(129) Filter [codegen id : 1] -Input [4]: [d_week_seq#121, d_year#122, d_moy#123, d_dom#124] -Condition : (((((isnotnull(d_year#122) AND isnotnull(d_moy#123)) AND isnotnull(d_dom#124)) AND (d_year#122 = 1999)) AND (d_moy#123 = 12)) AND (d_dom#124 = 16)) +(126) Filter [codegen id : 1] +Input [4]: [d_week_seq#120, d_year#121, d_moy#122, d_dom#123] +Condition : (((((isnotnull(d_year#121) AND isnotnull(d_moy#122)) AND isnotnull(d_dom#123)) AND (d_year#121 = 1999)) AND (d_moy#122 = 12)) AND (d_dom#123 = 16)) -(130) Project [codegen id : 1] -Output [1]: [d_week_seq#121] -Input [4]: [d_week_seq#121, d_year#122, d_moy#123, d_dom#124] +(127) Project [codegen id : 1] +Output [1]: [d_week_seq#120] +Input [4]: [d_week_seq#120, d_year#121, d_moy#122, d_dom#123] Subquery:7 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (135) -+- * Project (134) - +- * Filter (133) - +- * ColumnarToRow (132) - +- Scan parquet default.date_dim (131) +BroadcastExchange (132) ++- * Project (131) + +- * Filter (130) + +- * ColumnarToRow (129) + +- Scan parquet default.date_dim (128) -(131) Scan parquet default.date_dim -Output [2]: [d_date_sk#14, d_year#125] +(128) Scan parquet default.date_dim +Output [2]: [d_date_sk#14, d_year#124] 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 -(132) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#14, d_year#125] +(129) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#14, d_year#124] -(133) Filter [codegen id : 1] -Input [2]: [d_date_sk#14, d_year#125] -Condition : (((isnotnull(d_year#125) AND (d_year#125 >= 1998)) AND (d_year#125 <= 2000)) AND isnotnull(d_date_sk#14)) +(130) Filter [codegen id : 1] +Input [2]: [d_date_sk#14, d_year#124] +Condition : (((isnotnull(d_year#124) AND (d_year#124 >= 1998)) AND (d_year#124 <= 2000)) AND isnotnull(d_date_sk#14)) -(134) Project [codegen id : 1] +(131) Project [codegen id : 1] Output [1]: [d_date_sk#14] -Input [2]: [d_date_sk#14, d_year#125] +Input [2]: [d_date_sk#14, d_year#124] -(135) BroadcastExchange +(132) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#126] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#125] Subquery:8 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#13 Subquery:9 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#13 -Subquery:10 Hosting operator id = 99 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#65, [id=#66] -Subquery:11 Hosting operator id = 82 Hosting Expression = ss_sold_date_sk#71 IN dynamicpruning#72 -BroadcastExchange (140) -+- * Project (139) - +- * Filter (138) - +- * ColumnarToRow (137) - +- Scan parquet default.date_dim (136) +Subquery:11 Hosting operator id = 79 Hosting Expression = ss_sold_date_sk#70 IN dynamicpruning#71 +BroadcastExchange (137) ++- * Project (136) + +- * Filter (135) + +- * ColumnarToRow (134) + +- Scan parquet default.date_dim (133) -(136) Scan parquet default.date_dim -Output [2]: [d_date_sk#74, d_week_seq#127] +(133) Scan parquet default.date_dim +Output [2]: [d_date_sk#73, d_week_seq#126] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(137) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#74, d_week_seq#127] +(134) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#73, d_week_seq#126] -(138) Filter [codegen id : 1] -Input [2]: [d_date_sk#74, d_week_seq#127] -Condition : ((isnotnull(d_week_seq#127) AND (d_week_seq#127 = Subquery scalar-subquery#128, [id=#129])) AND isnotnull(d_date_sk#74)) +(135) Filter [codegen id : 1] +Input [2]: [d_date_sk#73, d_week_seq#126] +Condition : ((isnotnull(d_week_seq#126) AND (d_week_seq#126 = Subquery scalar-subquery#127, [id=#128])) AND isnotnull(d_date_sk#73)) -(139) Project [codegen id : 1] -Output [1]: [d_date_sk#74] -Input [2]: [d_date_sk#74, d_week_seq#127] +(136) Project [codegen id : 1] +Output [1]: [d_date_sk#73] +Input [2]: [d_date_sk#73, d_week_seq#126] -(140) BroadcastExchange -Input [1]: [d_date_sk#74] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#130] +(137) BroadcastExchange +Input [1]: [d_date_sk#73] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#129] -Subquery:12 Hosting operator id = 138 Hosting Expression = Subquery scalar-subquery#128, [id=#129] -* Project (144) -+- * Filter (143) - +- * ColumnarToRow (142) - +- Scan parquet default.date_dim (141) +Subquery:12 Hosting operator id = 135 Hosting Expression = Subquery scalar-subquery#127, [id=#128] +* Project (141) ++- * Filter (140) + +- * ColumnarToRow (139) + +- Scan parquet default.date_dim (138) -(141) Scan parquet default.date_dim -Output [4]: [d_week_seq#131, d_year#132, d_moy#133, d_dom#134] +(138) Scan parquet default.date_dim +Output [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(142) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#131, d_year#132, d_moy#133, d_dom#134] +(139) ColumnarToRow [codegen id : 1] +Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] -(143) Filter [codegen id : 1] -Input [4]: [d_week_seq#131, d_year#132, d_moy#133, d_dom#134] -Condition : (((((isnotnull(d_year#132) AND isnotnull(d_moy#133)) AND isnotnull(d_dom#134)) AND (d_year#132 = 1998)) AND (d_moy#133 = 12)) AND (d_dom#134 = 16)) +(140) 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)) -(144) Project [codegen id : 1] -Output [1]: [d_week_seq#131] -Input [4]: [d_week_seq#131, d_year#132, d_moy#133, d_dom#134] +(141) 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] 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 e7d3f84db0c72..82e338515f431 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,12 +1,12 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (92) + WholeStageCodegen (88) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [sales] Subquery #4 WholeStageCodegen (8) HashAggregate [sum,count] [avg(CheckOverflow((promote_precision(cast(quantity as decimal(12,2))) * promote_precision(cast(list_price as decimal(12,2)))), DecimalType(18,2))),average_sales,sum,count] InputAdapter - Exchange #17 + Exchange #16 WholeStageCodegen (7) HashAggregate [quantity,list_price] [sum,count,sum,count] InputAdapter @@ -19,7 +19,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #3 InputAdapter - ReusedExchange [d_date_sk] #9 + ReusedExchange [d_date_sk] #8 WholeStageCodegen (4) Project [cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] @@ -28,7 +28,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 InputAdapter - ReusedExchange [d_date_sk] #9 + ReusedExchange [d_date_sk] #8 WholeStageCodegen (6) Project [ws_quantity,ws_list_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] @@ -37,11 +37,11 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 InputAdapter - ReusedExchange [d_date_sk] #9 + ReusedExchange [d_date_sk] #8 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(ss_quantity as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2))),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (45) + WholeStageCodegen (43) 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] @@ -74,11 +74,11 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter Scan parquet default.date_dim [d_date_sk,d_week_seq] InputAdapter - 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] @@ -87,129 +87,124 @@ 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) - HashAggregate [brand_id,class_id,category_id] + WholeStageCodegen (18) + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] InputAdapter - Exchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (18) - HashAggregate [brand_id,class_id,category_id] - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - InputAdapter - WholeStageCodegen (13) - Sort [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (12) - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #8 - WholeStageCodegen (11) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #9 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] + WholeStageCodegen (13) + Sort [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #6 + WholeStageCodegen (12) + HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #7 + 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] #3 + BroadcastExchange #8 + WholeStageCodegen (1) + 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 #9 + WholeStageCodegen (10) + SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (5) + Sort [i_brand_id,i_class_id,i_category_id] InputAdapter - ReusedExchange [d_date_sk] #9 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (10) - SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - InputAdapter - WholeStageCodegen (5) - Sort [i_brand_id,i_class_id,i_category_id] + Exchange [i_brand_id,i_class_id,i_category_id] #10 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow 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] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (9) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #11 + 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] #3 + InputAdapter + ReusedExchange [d_date_sk] #8 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (7) + Filter [i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - WholeStageCodegen (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] #3 - InputAdapter - ReusedExchange [d_date_sk] #9 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (7) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - WholeStageCodegen (17) - Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (17) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #13 + WholeStageCodegen (16) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #8 InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (16) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #9 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #15 - WholeStageCodegen (44) + BroadcastExchange #14 + WholeStageCodegen (42) SortMergeJoin [i_item_sk,ss_item_sk] InputAdapter - WholeStageCodegen (24) + WholeStageCodegen (23) Sort [i_item_sk] InputAdapter - Exchange [i_item_sk] #16 - WholeStageCodegen (23) + Exchange [i_item_sk] #15 + 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] InputAdapter - WholeStageCodegen (43) + WholeStageCodegen (41) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #4 InputAdapter - BroadcastExchange #18 - WholeStageCodegen (91) + BroadcastExchange #17 + WholeStageCodegen (87) Filter [sales] ReusedSubquery [average_sales] #4 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(ss_quantity as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2))),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #19 - WholeStageCodegen (90) + Exchange [i_brand_id,i_class_id,i_category_id] #18 + WholeStageCodegen (86) 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] @@ -217,17 +212,17 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_sold_date_sk,d_date_sk] SortMergeJoin [ss_item_sk,ss_item_sk] InputAdapter - WholeStageCodegen (47) + WholeStageCodegen (45) Sort [ss_item_sk] InputAdapter - Exchange [ss_item_sk] #20 - WholeStageCodegen (46) + Exchange [ss_item_sk] #19 + WholeStageCodegen (44) 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] #5 - BroadcastExchange #21 + BroadcastExchange #20 WholeStageCodegen (1) Project [d_date_sk] Filter [d_week_seq,d_date_sk] @@ -242,11 +237,11 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter Scan parquet default.date_dim [d_date_sk,d_week_seq] InputAdapter - WholeStageCodegen (66) + WholeStageCodegen (63) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #4 InputAdapter - ReusedExchange [d_date_sk] #21 + ReusedExchange [d_date_sk] #20 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt index a5e01db243952..86bbc553e8c31 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt @@ -1,90 +1,88 @@ == Physical Plan == -TakeOrderedAndProject (86) -+- * BroadcastHashJoin Inner BuildRight (85) - :- * Filter (68) - : +- * HashAggregate (67) - : +- Exchange (66) - : +- * HashAggregate (65) - : +- * Project (64) - : +- * BroadcastHashJoin Inner BuildRight (63) - : :- * Project (61) - : : +- * BroadcastHashJoin Inner BuildRight (60) - : : :- * BroadcastHashJoin LeftSemi BuildRight (53) +TakeOrderedAndProject (84) ++- * BroadcastHashJoin Inner BuildRight (83) + :- * Filter (66) + : +- * HashAggregate (65) + : +- Exchange (64) + : +- * HashAggregate (63) + : +- * Project (62) + : +- * BroadcastHashJoin Inner BuildRight (61) + : :- * Project (59) + : : +- * BroadcastHashJoin Inner BuildRight (58) + : : :- * BroadcastHashJoin LeftSemi BuildRight (51) : : : :- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (52) - : : : +- * Project (51) - : : : +- * BroadcastHashJoin Inner BuildRight (50) + : : : +- BroadcastExchange (50) + : : : +- * Project (49) + : : : +- * BroadcastHashJoin Inner BuildRight (48) : : : :- * Filter (6) : : : : +- * ColumnarToRow (5) : : : : +- Scan parquet default.item (4) - : : : +- BroadcastExchange (49) - : : : +- * HashAggregate (48) - : : : +- * HashAggregate (47) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) - : : : :- * HashAggregate (35) - : : : : +- Exchange (34) - : : : : +- * HashAggregate (33) - : : : : +- * Project (32) - : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * Filter (9) - : : : : : : +- * ColumnarToRow (8) - : : : : : : +- Scan parquet default.store_sales (7) - : : : : : +- BroadcastExchange (27) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet default.item (10) - : : : : : +- BroadcastExchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * 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) - : : : : : +- ReusedExchange (22) - : : : : +- ReusedExchange (30) - : : : +- BroadcastExchange (45) - : : : +- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * Project (41) - : : : : +- * BroadcastHashJoin Inner BuildRight (40) - : : : : :- * Filter (38) - : : : : : +- * ColumnarToRow (37) - : : : : : +- Scan parquet default.web_sales (36) - : : : : +- ReusedExchange (39) - : : : +- ReusedExchange (42) - : : +- BroadcastExchange (59) - : : +- * BroadcastHashJoin LeftSemi BuildRight (58) - : : :- * Filter (56) - : : : +- * ColumnarToRow (55) - : : : +- Scan parquet default.item (54) - : : +- ReusedExchange (57) - : +- ReusedExchange (62) - +- BroadcastExchange (84) - +- * Filter (83) - +- * HashAggregate (82) - +- Exchange (81) - +- * HashAggregate (80) - +- * Project (79) - +- * BroadcastHashJoin Inner BuildRight (78) - :- * Project (76) - : +- * BroadcastHashJoin Inner BuildRight (75) - : :- * BroadcastHashJoin LeftSemi BuildRight (73) - : : :- * Filter (71) - : : : +- * ColumnarToRow (70) - : : : +- Scan parquet default.store_sales (69) - : : +- ReusedExchange (72) - : +- ReusedExchange (74) - +- ReusedExchange (77) + : : : +- BroadcastExchange (47) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) + : : : :- * HashAggregate (35) + : : : : +- Exchange (34) + : : : : +- * HashAggregate (33) + : : : : +- * Project (32) + : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : :- * Project (29) + : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : :- * Filter (9) + : : : : : : +- * ColumnarToRow (8) + : : : : : : +- Scan parquet default.store_sales (7) + : : : : : +- BroadcastExchange (27) + : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) + : : : : : :- * Filter (12) + : : : : : : +- * ColumnarToRow (11) + : : : : : : +- Scan parquet default.item (10) + : : : : : +- BroadcastExchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * 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) + : : : : : +- ReusedExchange (22) + : : : : +- ReusedExchange (30) + : : : +- BroadcastExchange (45) + : : : +- * Project (44) + : : : +- * BroadcastHashJoin Inner BuildRight (43) + : : : :- * Project (41) + : : : : +- * BroadcastHashJoin Inner BuildRight (40) + : : : : :- * Filter (38) + : : : : : +- * ColumnarToRow (37) + : : : : : +- Scan parquet default.web_sales (36) + : : : : +- ReusedExchange (39) + : : : +- ReusedExchange (42) + : : +- BroadcastExchange (57) + : : +- * BroadcastHashJoin LeftSemi BuildRight (56) + : : :- * Filter (54) + : : : +- * ColumnarToRow (53) + : : : +- Scan parquet default.item (52) + : : +- ReusedExchange (55) + : +- ReusedExchange (60) + +- BroadcastExchange (82) + +- * Filter (81) + +- * HashAggregate (80) + +- Exchange (79) + +- * HashAggregate (78) + +- * Project (77) + +- * BroadcastHashJoin Inner BuildRight (76) + :- * Project (74) + : +- * BroadcastHashJoin Inner BuildRight (73) + : :- * BroadcastHashJoin LeftSemi BuildRight (71) + : : :- * Filter (69) + : : : +- * ColumnarToRow (68) + : : : +- Scan parquet default.store_sales (67) + : : +- ReusedExchange (70) + : +- ReusedExchange (72) + +- ReusedExchange (75) (1) Scan parquet default.store_sales @@ -187,7 +185,7 @@ Join condition: None 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) ReusedExchange [Reuses operator id: 119] +(22) ReusedExchange [Reuses operator id: 117] Output [1]: [d_date_sk#24] (23) BroadcastHashJoin [codegen id : 3] @@ -221,7 +219,7 @@ Join condition: None 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] -(30) ReusedExchange [Reuses operator id: 119] +(30) ReusedExchange [Reuses operator id: 117] Output [1]: [d_date_sk#27] (31) BroadcastHashJoin [codegen id : 6] @@ -278,7 +276,7 @@ Join condition: None 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] -(42) ReusedExchange [Reuses operator id: 119] +(42) ReusedExchange [Reuses operator id: 117] Output [1]: [d_date_sk#38] (43) BroadcastHashJoin [codegen id : 9] @@ -299,112 +297,98 @@ Left keys [6]: [coalesce(brand_id#28, 0), isnull(brand_id#28), coalesce(class_id 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 -(47) 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] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] - -(48) 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] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] - -(49) BroadcastExchange +(47) 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] -(50) BroadcastHashJoin [codegen id : 11] +(48) 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] Join condition: None -(51) Project [codegen id : 11] +(49) 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] -(52) BroadcastExchange +(50) BroadcastExchange Input [1]: [ss_item_sk#41] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] -(53) BroadcastHashJoin [codegen id : 25] +(51) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#41] Join condition: None -(54) Scan parquet default.item +(52) Scan parquet default.item Output [4]: [i_item_sk#43, i_brand_id#44, i_class_id#45, i_category_id#46] 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 -(55) ColumnarToRow [codegen id : 23] +(53) ColumnarToRow [codegen id : 23] Input [4]: [i_item_sk#43, i_brand_id#44, i_class_id#45, i_category_id#46] -(56) Filter [codegen id : 23] +(54) Filter [codegen id : 23] Input [4]: [i_item_sk#43, i_brand_id#44, i_class_id#45, i_category_id#46] Condition : (((isnotnull(i_item_sk#43) AND isnotnull(i_brand_id#44)) AND isnotnull(i_class_id#45)) AND isnotnull(i_category_id#46)) -(57) ReusedExchange [Reuses operator id: 52] +(55) ReusedExchange [Reuses operator id: 50] Output [1]: [ss_item_sk#41] -(58) BroadcastHashJoin [codegen id : 23] +(56) BroadcastHashJoin [codegen id : 23] Left keys [1]: [i_item_sk#43] Right keys [1]: [ss_item_sk#41] Join condition: None -(59) BroadcastExchange +(57) BroadcastExchange Input [4]: [i_item_sk#43, i_brand_id#44, i_class_id#45, i_category_id#46] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#47] -(60) BroadcastHashJoin [codegen id : 25] +(58) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#43] Join condition: None -(61) Project [codegen id : 25] +(59) Project [codegen id : 25] Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#44, i_class_id#45, i_category_id#46] Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#43, i_brand_id#44, i_class_id#45, i_category_id#46] -(62) ReusedExchange [Reuses operator id: 110] +(60) ReusedExchange [Reuses operator id: 108] Output [1]: [d_date_sk#48] -(63) BroadcastHashJoin [codegen id : 25] +(61) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#48] Join condition: None -(64) Project [codegen id : 25] +(62) Project [codegen id : 25] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#44, i_class_id#45, i_category_id#46] Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#44, i_class_id#45, i_category_id#46, d_date_sk#48] -(65) HashAggregate [codegen id : 25] +(63) HashAggregate [codegen id : 25] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#44, i_class_id#45, i_category_id#46] Keys [3]: [i_brand_id#44, i_class_id#45, i_category_id#46] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)] Aggregate Attributes [3]: [sum#49, isEmpty#50, count#51] Results [6]: [i_brand_id#44, i_class_id#45, i_category_id#46, sum#52, isEmpty#53, count#54] -(66) Exchange +(64) Exchange Input [6]: [i_brand_id#44, i_class_id#45, i_category_id#46, sum#52, isEmpty#53, count#54] Arguments: hashpartitioning(i_brand_id#44, i_class_id#45, i_category_id#46, 5), ENSURE_REQUIREMENTS, [id=#55] -(67) HashAggregate [codegen id : 52] +(65) HashAggregate [codegen id : 52] Input [6]: [i_brand_id#44, i_class_id#45, i_category_id#46, sum#52, isEmpty#53, count#54] Keys [3]: [i_brand_id#44, i_class_id#45, i_category_id#46] Functions [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2))), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#56, count(1)#57] Results [6]: [store AS channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#56 AS sales#59, count(1)#57 AS number_sales#60] -(68) Filter [codegen id : 52] +(66) Filter [codegen id : 52] Input [6]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sales#59, number_sales#60] Condition : (isnotnull(sales#59) AND (cast(sales#59 as decimal(32,6)) > cast(Subquery scalar-subquery#61, [id=#62] as decimal(32,6)))) -(69) Scan parquet default.store_sales +(67) Scan parquet default.store_sales Output [4]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66] Batched: true Location: InMemoryFileIndex [] @@ -412,278 +396,278 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#66), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(70) ColumnarToRow [codegen id : 50] +(68) ColumnarToRow [codegen id : 50] Input [4]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66] -(71) Filter [codegen id : 50] +(69) 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) -(72) ReusedExchange [Reuses operator id: 52] +(70) ReusedExchange [Reuses operator id: 50] Output [1]: [ss_item_sk#41] -(73) BroadcastHashJoin [codegen id : 50] +(71) BroadcastHashJoin [codegen id : 50] Left keys [1]: [ss_item_sk#63] Right keys [1]: [ss_item_sk#41] Join condition: None -(74) ReusedExchange [Reuses operator id: 59] +(72) ReusedExchange [Reuses operator id: 57] Output [4]: [i_item_sk#68, i_brand_id#69, i_class_id#70, i_category_id#71] -(75) BroadcastHashJoin [codegen id : 50] +(73) BroadcastHashJoin [codegen id : 50] Left keys [1]: [ss_item_sk#63] Right keys [1]: [i_item_sk#68] Join condition: None -(76) Project [codegen id : 50] +(74) 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] -(77) ReusedExchange [Reuses operator id: 124] +(75) ReusedExchange [Reuses operator id: 122] Output [1]: [d_date_sk#72] -(78) BroadcastHashJoin [codegen id : 50] +(76) BroadcastHashJoin [codegen id : 50] Left keys [1]: [ss_sold_date_sk#66] Right keys [1]: [d_date_sk#72] Join condition: None -(79) Project [codegen id : 50] +(77) 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] -(80) HashAggregate [codegen id : 50] +(78) 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(ss_quantity#64 as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)] Aggregate Attributes [3]: [sum#73, isEmpty#74, count#75] Results [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#76, isEmpty#77, count#78] -(81) Exchange +(79) Exchange Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#76, isEmpty#77, count#78] Arguments: hashpartitioning(i_brand_id#69, i_class_id#70, i_category_id#71, 5), ENSURE_REQUIREMENTS, [id=#79] -(82) HashAggregate [codegen id : 51] +(80) HashAggregate [codegen id : 51] Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#76, isEmpty#77, count#78] Keys [3]: [i_brand_id#69, i_class_id#70, i_category_id#71] Functions [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#64 as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2))), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#64 as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2)))#80, count(1)#81] Results [6]: [store AS channel#82, i_brand_id#69, i_class_id#70, i_category_id#71, sum(CheckOverflow((promote_precision(cast(ss_quantity#64 as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2)))#80 AS sales#83, count(1)#81 AS number_sales#84] -(83) Filter [codegen id : 51] +(81) Filter [codegen id : 51] Input [6]: [channel#82, i_brand_id#69, i_class_id#70, i_category_id#71, sales#83, number_sales#84] Condition : (isnotnull(sales#83) AND (cast(sales#83 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#61, [id=#62] as decimal(32,6)))) -(84) BroadcastExchange +(82) BroadcastExchange Input [6]: [channel#82, i_brand_id#69, i_class_id#70, i_category_id#71, sales#83, number_sales#84] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#85] -(85) BroadcastHashJoin [codegen id : 52] +(83) BroadcastHashJoin [codegen id : 52] Left keys [3]: [i_brand_id#44, i_class_id#45, i_category_id#46] Right keys [3]: [i_brand_id#69, i_class_id#70, i_category_id#71] Join condition: None -(86) TakeOrderedAndProject +(84) TakeOrderedAndProject Input [12]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sales#59, number_sales#60, channel#82, i_brand_id#69, i_class_id#70, i_category_id#71, sales#83, number_sales#84] Arguments: 100, [i_brand_id#44 ASC NULLS FIRST, i_class_id#45 ASC NULLS FIRST, i_category_id#46 ASC NULLS FIRST], [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sales#59, number_sales#60, channel#82, i_brand_id#69, i_class_id#70, i_category_id#71, sales#83, number_sales#84] ===== Subqueries ===== -Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#61, [id=#62] -* HashAggregate (105) -+- Exchange (104) - +- * HashAggregate (103) - +- Union (102) - :- * Project (91) - : +- * BroadcastHashJoin Inner BuildRight (90) - : :- * ColumnarToRow (88) - : : +- Scan parquet default.store_sales (87) - : +- ReusedExchange (89) - :- * Project (96) - : +- * BroadcastHashJoin Inner BuildRight (95) - : :- * ColumnarToRow (93) - : : +- Scan parquet default.catalog_sales (92) - : +- ReusedExchange (94) - +- * Project (101) - +- * BroadcastHashJoin Inner BuildRight (100) - :- * ColumnarToRow (98) - : +- Scan parquet default.web_sales (97) - +- ReusedExchange (99) - - -(87) Scan parquet default.store_sales +Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#61, [id=#62] +* HashAggregate (103) ++- Exchange (102) + +- * HashAggregate (101) + +- Union (100) + :- * Project (89) + : +- * BroadcastHashJoin Inner BuildRight (88) + : :- * ColumnarToRow (86) + : : +- Scan parquet default.store_sales (85) + : +- ReusedExchange (87) + :- * Project (94) + : +- * BroadcastHashJoin Inner BuildRight (93) + : :- * ColumnarToRow (91) + : : +- Scan parquet default.catalog_sales (90) + : +- ReusedExchange (92) + +- * Project (99) + +- * BroadcastHashJoin Inner BuildRight (98) + :- * ColumnarToRow (96) + : +- Scan parquet default.web_sales (95) + +- ReusedExchange (97) + + +(85) Scan parquet default.store_sales Output [3]: [ss_quantity#86, ss_list_price#87, ss_sold_date_sk#88] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#88), dynamicpruningexpression(ss_sold_date_sk#88 IN dynamicpruning#12)] ReadSchema: struct -(88) ColumnarToRow [codegen id : 2] +(86) ColumnarToRow [codegen id : 2] Input [3]: [ss_quantity#86, ss_list_price#87, ss_sold_date_sk#88] -(89) ReusedExchange [Reuses operator id: 119] +(87) ReusedExchange [Reuses operator id: 117] Output [1]: [d_date_sk#89] -(90) BroadcastHashJoin [codegen id : 2] +(88) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#88] Right keys [1]: [d_date_sk#89] Join condition: None -(91) Project [codegen id : 2] +(89) Project [codegen id : 2] Output [2]: [ss_quantity#86 AS quantity#90, ss_list_price#87 AS list_price#91] Input [4]: [ss_quantity#86, ss_list_price#87, ss_sold_date_sk#88, d_date_sk#89] -(92) Scan parquet default.catalog_sales +(90) Scan parquet default.catalog_sales Output [3]: [cs_quantity#92, cs_list_price#93, cs_sold_date_sk#94] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#94), dynamicpruningexpression(cs_sold_date_sk#94 IN dynamicpruning#12)] ReadSchema: struct -(93) ColumnarToRow [codegen id : 4] +(91) ColumnarToRow [codegen id : 4] Input [3]: [cs_quantity#92, cs_list_price#93, cs_sold_date_sk#94] -(94) ReusedExchange [Reuses operator id: 119] +(92) ReusedExchange [Reuses operator id: 117] Output [1]: [d_date_sk#95] -(95) BroadcastHashJoin [codegen id : 4] +(93) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#94] Right keys [1]: [d_date_sk#95] Join condition: None -(96) Project [codegen id : 4] +(94) Project [codegen id : 4] Output [2]: [cs_quantity#92 AS quantity#96, cs_list_price#93 AS list_price#97] Input [4]: [cs_quantity#92, cs_list_price#93, cs_sold_date_sk#94, d_date_sk#95] -(97) Scan parquet default.web_sales +(95) Scan parquet default.web_sales Output [3]: [ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#100), dynamicpruningexpression(ws_sold_date_sk#100 IN dynamicpruning#12)] ReadSchema: struct -(98) ColumnarToRow [codegen id : 6] +(96) ColumnarToRow [codegen id : 6] Input [3]: [ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] -(99) ReusedExchange [Reuses operator id: 119] +(97) ReusedExchange [Reuses operator id: 117] Output [1]: [d_date_sk#101] -(100) BroadcastHashJoin [codegen id : 6] +(98) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#100] Right keys [1]: [d_date_sk#101] Join condition: None -(101) Project [codegen id : 6] +(99) Project [codegen id : 6] Output [2]: [ws_quantity#98 AS quantity#102, ws_list_price#99 AS list_price#103] Input [4]: [ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100, d_date_sk#101] -(102) Union +(100) Union -(103) HashAggregate [codegen id : 7] +(101) HashAggregate [codegen id : 7] Input [2]: [quantity#90, list_price#91] Keys: [] Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(quantity#90 as decimal(12,2))) * promote_precision(cast(list_price#91 as decimal(12,2)))), DecimalType(18,2)))] Aggregate Attributes [2]: [sum#104, count#105] Results [2]: [sum#106, count#107] -(104) Exchange +(102) Exchange Input [2]: [sum#106, count#107] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#108] -(105) HashAggregate [codegen id : 8] +(103) HashAggregate [codegen id : 8] Input [2]: [sum#106, count#107] Keys: [] Functions [1]: [avg(CheckOverflow((promote_precision(cast(quantity#90 as decimal(12,2))) * promote_precision(cast(list_price#91 as decimal(12,2)))), DecimalType(18,2)))] Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(quantity#90 as decimal(12,2))) * promote_precision(cast(list_price#91 as decimal(12,2)))), DecimalType(18,2)))#109] Results [1]: [avg(CheckOverflow((promote_precision(cast(quantity#90 as decimal(12,2))) * promote_precision(cast(list_price#91 as decimal(12,2)))), DecimalType(18,2)))#109 AS average_sales#110] -Subquery:2 Hosting operator id = 87 Hosting Expression = ss_sold_date_sk#88 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 85 Hosting Expression = ss_sold_date_sk#88 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 92 Hosting Expression = cs_sold_date_sk#94 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#94 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#100 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 95 Hosting Expression = ws_sold_date_sk#100 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (110) -+- * Project (109) - +- * Filter (108) - +- * ColumnarToRow (107) - +- Scan parquet default.date_dim (106) +BroadcastExchange (108) ++- * Project (107) + +- * Filter (106) + +- * ColumnarToRow (105) + +- Scan parquet default.date_dim (104) -(106) Scan parquet default.date_dim +(104) Scan parquet default.date_dim Output [2]: [d_date_sk#48, d_week_seq#111] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(107) ColumnarToRow [codegen id : 1] +(105) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#48, d_week_seq#111] -(108) Filter [codegen id : 1] +(106) Filter [codegen id : 1] Input [2]: [d_date_sk#48, d_week_seq#111] Condition : ((isnotnull(d_week_seq#111) AND (d_week_seq#111 = Subquery scalar-subquery#112, [id=#113])) AND isnotnull(d_date_sk#48)) -(109) Project [codegen id : 1] +(107) Project [codegen id : 1] Output [1]: [d_date_sk#48] Input [2]: [d_date_sk#48, d_week_seq#111] -(110) BroadcastExchange +(108) BroadcastExchange Input [1]: [d_date_sk#48] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#114] -Subquery:6 Hosting operator id = 108 Hosting Expression = Subquery scalar-subquery#112, [id=#113] -* Project (114) -+- * Filter (113) - +- * ColumnarToRow (112) - +- Scan parquet default.date_dim (111) +Subquery:6 Hosting operator id = 106 Hosting Expression = Subquery scalar-subquery#112, [id=#113] +* Project (112) ++- * Filter (111) + +- * ColumnarToRow (110) + +- Scan parquet default.date_dim (109) -(111) Scan parquet default.date_dim +(109) Scan parquet default.date_dim Output [4]: [d_week_seq#115, d_year#116, d_moy#117, d_dom#118] 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 -(112) ColumnarToRow [codegen id : 1] +(110) ColumnarToRow [codegen id : 1] Input [4]: [d_week_seq#115, d_year#116, d_moy#117, d_dom#118] -(113) Filter [codegen id : 1] +(111) Filter [codegen id : 1] Input [4]: [d_week_seq#115, d_year#116, d_moy#117, d_dom#118] Condition : (((((isnotnull(d_year#116) AND isnotnull(d_moy#117)) AND isnotnull(d_dom#118)) AND (d_year#116 = 1999)) AND (d_moy#117 = 12)) AND (d_dom#118 = 16)) -(114) Project [codegen id : 1] +(112) Project [codegen id : 1] Output [1]: [d_week_seq#115] Input [4]: [d_week_seq#115, d_year#116, d_moy#117, d_dom#118] Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (119) -+- * Project (118) - +- * Filter (117) - +- * ColumnarToRow (116) - +- Scan parquet default.date_dim (115) +BroadcastExchange (117) ++- * Project (116) + +- * Filter (115) + +- * ColumnarToRow (114) + +- Scan parquet default.date_dim (113) -(115) Scan parquet default.date_dim +(113) Scan parquet default.date_dim Output [2]: [d_date_sk#27, d_year#119] 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 -(116) ColumnarToRow [codegen id : 1] +(114) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#27, d_year#119] -(117) Filter [codegen id : 1] +(115) Filter [codegen id : 1] Input [2]: [d_date_sk#27, d_year#119] Condition : (((isnotnull(d_year#119) AND (d_year#119 >= 1998)) AND (d_year#119 <= 2000)) AND isnotnull(d_date_sk#27)) -(118) Project [codegen id : 1] +(116) Project [codegen id : 1] Output [1]: [d_date_sk#27] Input [2]: [d_date_sk#27, d_year#119] -(119) BroadcastExchange +(117) BroadcastExchange Input [1]: [d_date_sk#27] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#120] @@ -691,60 +675,60 @@ Subquery:8 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN d Subquery:9 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#12 -Subquery:10 Hosting operator id = 83 Hosting Expression = ReusedSubquery Subquery scalar-subquery#61, [id=#62] +Subquery:10 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#61, [id=#62] -Subquery:11 Hosting operator id = 69 Hosting Expression = ss_sold_date_sk#66 IN dynamicpruning#67 -BroadcastExchange (124) -+- * Project (123) - +- * Filter (122) - +- * ColumnarToRow (121) - +- Scan parquet default.date_dim (120) +Subquery:11 Hosting operator id = 67 Hosting Expression = ss_sold_date_sk#66 IN dynamicpruning#67 +BroadcastExchange (122) ++- * Project (121) + +- * Filter (120) + +- * ColumnarToRow (119) + +- Scan parquet default.date_dim (118) -(120) Scan parquet default.date_dim +(118) Scan parquet default.date_dim Output [2]: [d_date_sk#72, d_week_seq#121] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(121) ColumnarToRow [codegen id : 1] +(119) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#72, d_week_seq#121] -(122) Filter [codegen id : 1] +(120) Filter [codegen id : 1] Input [2]: [d_date_sk#72, d_week_seq#121] Condition : ((isnotnull(d_week_seq#121) AND (d_week_seq#121 = Subquery scalar-subquery#122, [id=#123])) AND isnotnull(d_date_sk#72)) -(123) Project [codegen id : 1] +(121) Project [codegen id : 1] Output [1]: [d_date_sk#72] Input [2]: [d_date_sk#72, d_week_seq#121] -(124) BroadcastExchange +(122) BroadcastExchange Input [1]: [d_date_sk#72] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#124] -Subquery:12 Hosting operator id = 122 Hosting Expression = Subquery scalar-subquery#122, [id=#123] -* Project (128) -+- * Filter (127) - +- * ColumnarToRow (126) - +- Scan parquet default.date_dim (125) +Subquery:12 Hosting operator id = 120 Hosting Expression = Subquery scalar-subquery#122, [id=#123] +* Project (126) ++- * Filter (125) + +- * ColumnarToRow (124) + +- Scan parquet default.date_dim (123) -(125) Scan parquet default.date_dim +(123) Scan parquet default.date_dim Output [4]: [d_week_seq#125, d_year#126, d_moy#127, d_dom#128] 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 -(126) ColumnarToRow [codegen id : 1] +(124) ColumnarToRow [codegen id : 1] Input [4]: [d_week_seq#125, d_year#126, d_moy#127, d_dom#128] -(127) Filter [codegen id : 1] +(125) Filter [codegen id : 1] Input [4]: [d_week_seq#125, d_year#126, d_moy#127, d_dom#128] Condition : (((((isnotnull(d_year#126) AND isnotnull(d_moy#127)) AND isnotnull(d_dom#128)) AND (d_year#126 = 1998)) AND (d_moy#127 = 12)) AND (d_dom#128 = 16)) -(128) Project [codegen id : 1] +(126) Project [codegen id : 1] Output [1]: [d_week_seq#125] Input [4]: [d_week_seq#125, d_year#126, d_moy#127, d_dom#128] 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 8f722e735172f..259178d0e432f 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 @@ -79,77 +79,75 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter BroadcastExchange #4 WholeStageCodegen (10) - HashAggregate [brand_id,class_id,category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] HashAggregate [brand_id,class_id,category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - 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 - ReusedExchange [d_date_sk] #6 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (9) + InputAdapter + Exchange [brand_id,class_id,category_id] #5 + WholeStageCodegen (6) + HashAggregate [brand_id,class_id,category_id] Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] + 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.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #6 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + BroadcastExchange #7 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (3) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + 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 + ReusedExchange [d_date_sk] #6 InputAdapter ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + InputAdapter + ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #11 WholeStageCodegen (23) 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 e3ad267942560..88d71316966c6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt @@ -1,150 +1,147 @@ == Physical Plan == -TakeOrderedAndProject (146) -+- * HashAggregate (145) - +- Exchange (144) - +- * HashAggregate (143) - +- Union (142) - :- * HashAggregate (121) - : +- Exchange (120) - : +- * HashAggregate (119) - : +- Union (118) - : :- * Filter (81) - : : +- * HashAggregate (80) - : : +- Exchange (79) - : : +- * HashAggregate (78) - : : +- * Project (77) - : : +- * BroadcastHashJoin Inner BuildRight (76) - : : :- * Project (66) - : : : +- * BroadcastHashJoin Inner BuildRight (65) - : : : :- * SortMergeJoin LeftSemi (63) +TakeOrderedAndProject (143) ++- * HashAggregate (142) + +- Exchange (141) + +- * HashAggregate (140) + +- Union (139) + :- * HashAggregate (118) + : +- Exchange (117) + : +- * HashAggregate (116) + : +- Union (115) + : :- * Filter (78) + : : +- * HashAggregate (77) + : : +- Exchange (76) + : : +- * HashAggregate (75) + : : +- * Project (74) + : : +- * BroadcastHashJoin Inner BuildRight (73) + : : :- * Project (63) + : : : +- * BroadcastHashJoin Inner BuildRight (62) + : : : :- * SortMergeJoin LeftSemi (60) : : : : :- * Sort (5) : : : : : +- Exchange (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.store_sales (1) - : : : : +- * Sort (62) - : : : : +- Exchange (61) - : : : : +- * Project (60) - : : : : +- * BroadcastHashJoin Inner BuildRight (59) + : : : : +- * Sort (59) + : : : : +- Exchange (58) + : : : : +- * Project (57) + : : : : +- * BroadcastHashJoin Inner BuildRight (56) : : : : :- * Filter (8) : : : : : +- * ColumnarToRow (7) : : : : : +- Scan parquet default.item (6) - : : : : +- BroadcastExchange (58) - : : : : +- * HashAggregate (57) - : : : : +- Exchange (56) - : : : : +- * HashAggregate (55) - : : : : +- * SortMergeJoin LeftSemi (54) - : : : : :- * Sort (42) - : : : : : +- Exchange (41) - : : : : : +- * HashAggregate (40) - : : : : : +- Exchange (39) - : : : : : +- * HashAggregate (38) - : : : : : +- * Project (37) - : : : : : +- * BroadcastHashJoin Inner BuildRight (36) - : : : : : :- * Project (14) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (13) - : : : : : : :- * Filter (11) - : : : : : : : +- * ColumnarToRow (10) - : : : : : : : +- Scan parquet default.store_sales (9) - : : : : : : +- ReusedExchange (12) - : : : : : +- BroadcastExchange (35) - : : : : : +- * SortMergeJoin LeftSemi (34) - : : : : : :- * Sort (19) - : : : : : : +- Exchange (18) - : : : : : : +- * Filter (17) - : : : : : : +- * ColumnarToRow (16) - : : : : : : +- Scan parquet default.item (15) - : : : : : +- * Sort (33) - : : : : : +- Exchange (32) - : : : : : +- * Project (31) - : : : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : : : :- * Project (25) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (24) - : : : : : : :- * Filter (22) - : : : : : : : +- * ColumnarToRow (21) - : : : : : : : +- Scan parquet default.catalog_sales (20) - : : : : : : +- ReusedExchange (23) - : : : : : +- BroadcastExchange (29) - : : : : : +- * Filter (28) - : : : : : +- * ColumnarToRow (27) - : : : : : +- Scan parquet default.item (26) - : : : : +- * Sort (53) - : : : : +- Exchange (52) - : : : : +- * Project (51) - : : : : +- * BroadcastHashJoin Inner BuildRight (50) - : : : : :- * Project (48) - : : : : : +- * BroadcastHashJoin Inner BuildRight (47) - : : : : : :- * Filter (45) - : : : : : : +- * ColumnarToRow (44) - : : : : : : +- Scan parquet default.web_sales (43) - : : : : : +- ReusedExchange (46) - : : : : +- ReusedExchange (49) - : : : +- ReusedExchange (64) - : : +- BroadcastExchange (75) - : : +- * SortMergeJoin LeftSemi (74) - : : :- * Sort (71) - : : : +- Exchange (70) - : : : +- * Filter (69) - : : : +- * ColumnarToRow (68) - : : : +- Scan parquet default.item (67) - : : +- * Sort (73) - : : +- ReusedExchange (72) - : :- * Filter (99) - : : +- * HashAggregate (98) - : : +- Exchange (97) - : : +- * HashAggregate (96) - : : +- * Project (95) - : : +- * BroadcastHashJoin Inner BuildRight (94) - : : :- * Project (92) - : : : +- * BroadcastHashJoin Inner BuildRight (91) - : : : :- * SortMergeJoin LeftSemi (89) - : : : : :- * Sort (86) - : : : : : +- Exchange (85) - : : : : : +- * Filter (84) - : : : : : +- * ColumnarToRow (83) - : : : : : +- Scan parquet default.catalog_sales (82) - : : : : +- * Sort (88) - : : : : +- ReusedExchange (87) - : : : +- ReusedExchange (90) - : : +- ReusedExchange (93) - : +- * Filter (117) - : +- * HashAggregate (116) - : +- Exchange (115) - : +- * HashAggregate (114) - : +- * Project (113) - : +- * BroadcastHashJoin Inner BuildRight (112) - : :- * Project (110) - : : +- * BroadcastHashJoin Inner BuildRight (109) - : : :- * SortMergeJoin LeftSemi (107) - : : : :- * Sort (104) - : : : : +- Exchange (103) - : : : : +- * Filter (102) - : : : : +- * ColumnarToRow (101) - : : : : +- Scan parquet default.web_sales (100) - : : : +- * Sort (106) - : : : +- ReusedExchange (105) - : : +- ReusedExchange (108) - : +- ReusedExchange (111) - :- * HashAggregate (126) - : +- Exchange (125) - : +- * HashAggregate (124) - : +- * HashAggregate (123) - : +- ReusedExchange (122) - :- * HashAggregate (131) - : +- Exchange (130) - : +- * HashAggregate (129) - : +- * HashAggregate (128) - : +- ReusedExchange (127) - :- * HashAggregate (136) - : +- Exchange (135) - : +- * HashAggregate (134) - : +- * HashAggregate (133) - : +- ReusedExchange (132) - +- * HashAggregate (141) - +- Exchange (140) - +- * HashAggregate (139) - +- * HashAggregate (138) - +- ReusedExchange (137) + : : : : +- BroadcastExchange (55) + : : : : +- * SortMergeJoin LeftSemi (54) + : : : : :- * Sort (42) + : : : : : +- Exchange (41) + : : : : : +- * HashAggregate (40) + : : : : : +- Exchange (39) + : : : : : +- * HashAggregate (38) + : : : : : +- * Project (37) + : : : : : +- * BroadcastHashJoin Inner BuildRight (36) + : : : : : :- * Project (14) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (13) + : : : : : : :- * Filter (11) + : : : : : : : +- * ColumnarToRow (10) + : : : : : : : +- Scan parquet default.store_sales (9) + : : : : : : +- ReusedExchange (12) + : : : : : +- BroadcastExchange (35) + : : : : : +- * SortMergeJoin LeftSemi (34) + : : : : : :- * Sort (19) + : : : : : : +- Exchange (18) + : : : : : : +- * Filter (17) + : : : : : : +- * ColumnarToRow (16) + : : : : : : +- Scan parquet default.item (15) + : : : : : +- * Sort (33) + : : : : : +- Exchange (32) + : : : : : +- * Project (31) + : : : : : +- * BroadcastHashJoin Inner BuildRight (30) + : : : : : :- * Project (25) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (24) + : : : : : : :- * Filter (22) + : : : : : : : +- * ColumnarToRow (21) + : : : : : : : +- Scan parquet default.catalog_sales (20) + : : : : : : +- ReusedExchange (23) + : : : : : +- BroadcastExchange (29) + : : : : : +- * Filter (28) + : : : : : +- * ColumnarToRow (27) + : : : : : +- Scan parquet default.item (26) + : : : : +- * Sort (53) + : : : : +- Exchange (52) + : : : : +- * Project (51) + : : : : +- * BroadcastHashJoin Inner BuildRight (50) + : : : : :- * Project (48) + : : : : : +- * BroadcastHashJoin Inner BuildRight (47) + : : : : : :- * Filter (45) + : : : : : : +- * ColumnarToRow (44) + : : : : : : +- Scan parquet default.web_sales (43) + : : : : : +- ReusedExchange (46) + : : : : +- ReusedExchange (49) + : : : +- ReusedExchange (61) + : : +- BroadcastExchange (72) + : : +- * SortMergeJoin LeftSemi (71) + : : :- * Sort (68) + : : : +- Exchange (67) + : : : +- * Filter (66) + : : : +- * ColumnarToRow (65) + : : : +- Scan parquet default.item (64) + : : +- * Sort (70) + : : +- ReusedExchange (69) + : :- * Filter (96) + : : +- * HashAggregate (95) + : : +- Exchange (94) + : : +- * HashAggregate (93) + : : +- * Project (92) + : : +- * BroadcastHashJoin Inner BuildRight (91) + : : :- * Project (89) + : : : +- * BroadcastHashJoin Inner BuildRight (88) + : : : :- * SortMergeJoin LeftSemi (86) + : : : : :- * Sort (83) + : : : : : +- Exchange (82) + : : : : : +- * Filter (81) + : : : : : +- * ColumnarToRow (80) + : : : : : +- Scan parquet default.catalog_sales (79) + : : : : +- * Sort (85) + : : : : +- ReusedExchange (84) + : : : +- ReusedExchange (87) + : : +- ReusedExchange (90) + : +- * Filter (114) + : +- * HashAggregate (113) + : +- Exchange (112) + : +- * HashAggregate (111) + : +- * Project (110) + : +- * BroadcastHashJoin Inner BuildRight (109) + : :- * Project (107) + : : +- * BroadcastHashJoin Inner BuildRight (106) + : : :- * SortMergeJoin LeftSemi (104) + : : : :- * Sort (101) + : : : : +- Exchange (100) + : : : : +- * Filter (99) + : : : : +- * ColumnarToRow (98) + : : : : +- Scan parquet default.web_sales (97) + : : : +- * Sort (103) + : : : +- ReusedExchange (102) + : : +- ReusedExchange (105) + : +- ReusedExchange (108) + :- * HashAggregate (123) + : +- Exchange (122) + : +- * HashAggregate (121) + : +- * HashAggregate (120) + : +- ReusedExchange (119) + :- * HashAggregate (128) + : +- Exchange (127) + : +- * HashAggregate (126) + : +- * HashAggregate (125) + : +- ReusedExchange (124) + :- * HashAggregate (133) + : +- Exchange (132) + : +- * HashAggregate (131) + : +- * HashAggregate (130) + : +- ReusedExchange (129) + +- * HashAggregate (138) + +- Exchange (137) + +- * HashAggregate (136) + +- * HashAggregate (135) + +- ReusedExchange (134) (1) Scan parquet default.store_sales @@ -177,10 +174,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)) @@ -199,7 +196,7 @@ Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] Condition : isnotnull(ss_item_sk#11) -(12) ReusedExchange [Reuses operator id: 180] +(12) ReusedExchange [Reuses operator id: 177] Output [1]: [d_date_sk#14] (13) BroadcastHashJoin [codegen id : 11] @@ -248,7 +245,7 @@ Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] Condition : isnotnull(cs_item_sk#20) -(23) ReusedExchange [Reuses operator id: 180] +(23) ReusedExchange [Reuses operator id: 177] Output [1]: [d_date_sk#22] (24) BroadcastHashJoin [codegen id : 8] @@ -354,7 +351,7 @@ Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] Condition : isnotnull(ws_item_sk#35) -(46) ReusedExchange [Reuses operator id: 180] +(46) ReusedExchange [Reuses operator id: 177] Output [1]: [d_date_sk#37] (47) BroadcastHashJoin [codegen id : 16] @@ -391,663 +388,645 @@ Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id 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 -(55) HashAggregate [codegen id : 18] +(55) BroadcastExchange 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#30, class_id#31, category_id#32] - -(56) 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] - -(57) HashAggregate [codegen id : 19] -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#30, class_id#31, category_id#32] - -(58) 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] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#43] -(59) BroadcastHashJoin [codegen id : 20] +(56) 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] Join condition: None -(60) Project [codegen id : 20] -Output [1]: [i_item_sk#7 AS ss_item_sk#45] +(57) Project [codegen id : 19] +Output [1]: [i_item_sk#7 AS ss_item_sk#44] 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] -(61) Exchange -Input [1]: [ss_item_sk#45] -Arguments: hashpartitioning(ss_item_sk#45, 5), ENSURE_REQUIREMENTS, [id=#46] +(58) Exchange +Input [1]: [ss_item_sk#44] +Arguments: hashpartitioning(ss_item_sk#44, 5), ENSURE_REQUIREMENTS, [id=#45] -(62) Sort [codegen id : 21] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +(59) Sort [codegen id : 20] +Input [1]: [ss_item_sk#44] +Arguments: [ss_item_sk#44 ASC NULLS FIRST], false, 0 -(63) SortMergeJoin [codegen id : 45] +(60) SortMergeJoin [codegen id : 43] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#45] +Right keys [1]: [ss_item_sk#44] Join condition: None -(64) ReusedExchange [Reuses operator id: 175] -Output [1]: [d_date_sk#47] +(61) ReusedExchange [Reuses operator id: 172] +Output [1]: [d_date_sk#46] -(65) BroadcastHashJoin [codegen id : 45] +(62) BroadcastHashJoin [codegen id : 43] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#47] +Right keys [1]: [d_date_sk#46] Join condition: None -(66) Project [codegen id : 45] +(63) Project [codegen id : 43] 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#47] +Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#46] -(67) Scan parquet default.item -Output [4]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51] +(64) Scan parquet default.item +Output [4]: [i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(68) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51] +(65) ColumnarToRow [codegen id : 22] +Input [4]: [i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] -(69) Filter [codegen id : 23] -Input [4]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51] -Condition : isnotnull(i_item_sk#48) +(66) Filter [codegen id : 22] +Input [4]: [i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] +Condition : isnotnull(i_item_sk#47) -(70) Exchange -Input [4]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51] -Arguments: hashpartitioning(i_item_sk#48, 5), ENSURE_REQUIREMENTS, [id=#52] +(67) Exchange +Input [4]: [i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] +Arguments: hashpartitioning(i_item_sk#47, 5), ENSURE_REQUIREMENTS, [id=#51] -(71) Sort [codegen id : 24] -Input [4]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51] -Arguments: [i_item_sk#48 ASC NULLS FIRST], false, 0 +(68) Sort [codegen id : 23] +Input [4]: [i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] +Arguments: [i_item_sk#47 ASC NULLS FIRST], false, 0 -(72) ReusedExchange [Reuses operator id: 61] -Output [1]: [ss_item_sk#45] +(69) ReusedExchange [Reuses operator id: 58] +Output [1]: [ss_item_sk#44] -(73) Sort [codegen id : 43] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +(70) Sort [codegen id : 41] +Input [1]: [ss_item_sk#44] +Arguments: [ss_item_sk#44 ASC NULLS FIRST], false, 0 -(74) SortMergeJoin [codegen id : 44] -Left keys [1]: [i_item_sk#48] -Right keys [1]: [ss_item_sk#45] +(71) SortMergeJoin [codegen id : 42] +Left keys [1]: [i_item_sk#47] +Right keys [1]: [ss_item_sk#44] Join condition: None -(75) BroadcastExchange -Input [4]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#53] +(72) BroadcastExchange +Input [4]: [i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#52] -(76) BroadcastHashJoin [codegen id : 45] +(73) BroadcastHashJoin [codegen id : 43] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#48] +Right keys [1]: [i_item_sk#47] Join condition: None -(77) Project [codegen id : 45] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#49, i_class_id#50, i_category_id#51] -Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51] +(74) Project [codegen id : 43] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#48, i_class_id#49, i_category_id#50] +Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] -(78) HashAggregate [codegen id : 45] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#49, i_class_id#50, i_category_id#51] -Keys [3]: [i_brand_id#49, i_class_id#50, i_category_id#51] +(75) HashAggregate [codegen id : 43] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#48, i_class_id#49, i_category_id#50] +Keys [3]: [i_brand_id#48, i_class_id#49, i_category_id#50] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)] -Aggregate Attributes [3]: [sum#54, isEmpty#55, count#56] -Results [6]: [i_brand_id#49, i_class_id#50, i_category_id#51, sum#57, isEmpty#58, count#59] +Aggregate Attributes [3]: [sum#53, isEmpty#54, count#55] +Results [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum#56, isEmpty#57, count#58] -(79) Exchange -Input [6]: [i_brand_id#49, i_class_id#50, i_category_id#51, sum#57, isEmpty#58, count#59] -Arguments: hashpartitioning(i_brand_id#49, i_class_id#50, i_category_id#51, 5), ENSURE_REQUIREMENTS, [id=#60] +(76) Exchange +Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum#56, isEmpty#57, count#58] +Arguments: hashpartitioning(i_brand_id#48, i_class_id#49, i_category_id#50, 5), ENSURE_REQUIREMENTS, [id=#59] -(80) HashAggregate [codegen id : 46] -Input [6]: [i_brand_id#49, i_class_id#50, i_category_id#51, sum#57, isEmpty#58, count#59] -Keys [3]: [i_brand_id#49, i_class_id#50, i_category_id#51] +(77) HashAggregate [codegen id : 44] +Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum#56, isEmpty#57, count#58] +Keys [3]: [i_brand_id#48, i_class_id#49, i_category_id#50] Functions [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2))), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#61, count(1)#62] -Results [6]: [store AS channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#61 AS sales#64, count(1)#62 AS number_sales#65] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#60, count(1)#61] +Results [6]: [store AS channel#62, i_brand_id#48, i_class_id#49, i_category_id#50, sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#60 AS sales#63, count(1)#61 AS number_sales#64] -(81) Filter [codegen id : 46] -Input [6]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sales#64, number_sales#65] -Condition : (isnotnull(sales#64) AND (cast(sales#64 as decimal(32,6)) > cast(Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +(78) Filter [codegen id : 44] +Input [6]: [channel#62, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64] +Condition : (isnotnull(sales#63) AND (cast(sales#63 as decimal(32,6)) > cast(Subquery scalar-subquery#65, [id=#66] as decimal(32,6)))) -(82) Scan parquet default.catalog_sales -Output [4]: [cs_item_sk#68, cs_quantity#69, cs_list_price#70, cs_sold_date_sk#71] +(79) Scan parquet default.catalog_sales +Output [4]: [cs_item_sk#67, cs_quantity#68, cs_list_price#69, cs_sold_date_sk#70] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#71), dynamicpruningexpression(cs_sold_date_sk#71 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#70), dynamicpruningexpression(cs_sold_date_sk#70 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(83) ColumnarToRow [codegen id : 47] -Input [4]: [cs_item_sk#68, cs_quantity#69, cs_list_price#70, cs_sold_date_sk#71] +(80) ColumnarToRow [codegen id : 45] +Input [4]: [cs_item_sk#67, cs_quantity#68, cs_list_price#69, cs_sold_date_sk#70] -(84) Filter [codegen id : 47] -Input [4]: [cs_item_sk#68, cs_quantity#69, cs_list_price#70, cs_sold_date_sk#71] -Condition : isnotnull(cs_item_sk#68) +(81) Filter [codegen id : 45] +Input [4]: [cs_item_sk#67, cs_quantity#68, cs_list_price#69, cs_sold_date_sk#70] +Condition : isnotnull(cs_item_sk#67) -(85) Exchange -Input [4]: [cs_item_sk#68, cs_quantity#69, cs_list_price#70, cs_sold_date_sk#71] -Arguments: hashpartitioning(cs_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#72] +(82) Exchange +Input [4]: [cs_item_sk#67, cs_quantity#68, cs_list_price#69, cs_sold_date_sk#70] +Arguments: hashpartitioning(cs_item_sk#67, 5), ENSURE_REQUIREMENTS, [id=#71] -(86) Sort [codegen id : 48] -Input [4]: [cs_item_sk#68, cs_quantity#69, cs_list_price#70, cs_sold_date_sk#71] -Arguments: [cs_item_sk#68 ASC NULLS FIRST], false, 0 +(83) Sort [codegen id : 46] +Input [4]: [cs_item_sk#67, cs_quantity#68, cs_list_price#69, cs_sold_date_sk#70] +Arguments: [cs_item_sk#67 ASC NULLS FIRST], false, 0 -(87) ReusedExchange [Reuses operator id: 61] -Output [1]: [ss_item_sk#45] +(84) ReusedExchange [Reuses operator id: 58] +Output [1]: [ss_item_sk#44] -(88) Sort [codegen id : 67] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +(85) Sort [codegen id : 64] +Input [1]: [ss_item_sk#44] +Arguments: [ss_item_sk#44 ASC NULLS FIRST], false, 0 -(89) SortMergeJoin [codegen id : 91] -Left keys [1]: [cs_item_sk#68] -Right keys [1]: [ss_item_sk#45] +(86) SortMergeJoin [codegen id : 87] +Left keys [1]: [cs_item_sk#67] +Right keys [1]: [ss_item_sk#44] Join condition: None -(90) ReusedExchange [Reuses operator id: 175] -Output [1]: [d_date_sk#73] +(87) ReusedExchange [Reuses operator id: 172] +Output [1]: [d_date_sk#72] -(91) BroadcastHashJoin [codegen id : 91] -Left keys [1]: [cs_sold_date_sk#71] -Right keys [1]: [d_date_sk#73] +(88) BroadcastHashJoin [codegen id : 87] +Left keys [1]: [cs_sold_date_sk#70] +Right keys [1]: [d_date_sk#72] Join condition: None -(92) Project [codegen id : 91] -Output [3]: [cs_item_sk#68, cs_quantity#69, cs_list_price#70] -Input [5]: [cs_item_sk#68, cs_quantity#69, cs_list_price#70, cs_sold_date_sk#71, d_date_sk#73] +(89) Project [codegen id : 87] +Output [3]: [cs_item_sk#67, cs_quantity#68, cs_list_price#69] +Input [5]: [cs_item_sk#67, cs_quantity#68, cs_list_price#69, cs_sold_date_sk#70, d_date_sk#72] -(93) ReusedExchange [Reuses operator id: 75] -Output [4]: [i_item_sk#74, i_brand_id#75, i_class_id#76, i_category_id#77] +(90) ReusedExchange [Reuses operator id: 72] +Output [4]: [i_item_sk#73, i_brand_id#74, i_class_id#75, i_category_id#76] -(94) BroadcastHashJoin [codegen id : 91] -Left keys [1]: [cs_item_sk#68] -Right keys [1]: [i_item_sk#74] +(91) BroadcastHashJoin [codegen id : 87] +Left keys [1]: [cs_item_sk#67] +Right keys [1]: [i_item_sk#73] Join condition: None -(95) Project [codegen id : 91] -Output [5]: [cs_quantity#69, cs_list_price#70, i_brand_id#75, i_class_id#76, i_category_id#77] -Input [7]: [cs_item_sk#68, cs_quantity#69, cs_list_price#70, i_item_sk#74, i_brand_id#75, i_class_id#76, i_category_id#77] - -(96) HashAggregate [codegen id : 91] -Input [5]: [cs_quantity#69, cs_list_price#70, i_brand_id#75, i_class_id#76, i_category_id#77] -Keys [3]: [i_brand_id#75, i_class_id#76, i_category_id#77] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cs_quantity#69 as decimal(12,2))) * promote_precision(cast(cs_list_price#70 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)] -Aggregate Attributes [3]: [sum#78, isEmpty#79, count#80] -Results [6]: [i_brand_id#75, i_class_id#76, i_category_id#77, sum#81, isEmpty#82, count#83] - -(97) Exchange -Input [6]: [i_brand_id#75, i_class_id#76, i_category_id#77, sum#81, isEmpty#82, count#83] -Arguments: hashpartitioning(i_brand_id#75, i_class_id#76, i_category_id#77, 5), ENSURE_REQUIREMENTS, [id=#84] - -(98) HashAggregate [codegen id : 92] -Input [6]: [i_brand_id#75, i_class_id#76, i_category_id#77, sum#81, isEmpty#82, count#83] -Keys [3]: [i_brand_id#75, i_class_id#76, i_category_id#77] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cs_quantity#69 as decimal(12,2))) * promote_precision(cast(cs_list_price#70 as decimal(12,2)))), DecimalType(18,2))), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cs_quantity#69 as decimal(12,2))) * promote_precision(cast(cs_list_price#70 as decimal(12,2)))), DecimalType(18,2)))#85, count(1)#86] -Results [6]: [catalog AS channel#87, i_brand_id#75, i_class_id#76, i_category_id#77, sum(CheckOverflow((promote_precision(cast(cs_quantity#69 as decimal(12,2))) * promote_precision(cast(cs_list_price#70 as decimal(12,2)))), DecimalType(18,2)))#85 AS sales#88, count(1)#86 AS number_sales#89] - -(99) Filter [codegen id : 92] -Input [6]: [channel#87, i_brand_id#75, i_class_id#76, i_category_id#77, sales#88, number_sales#89] -Condition : (isnotnull(sales#88) AND (cast(sales#88 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) - -(100) Scan parquet default.web_sales -Output [4]: [ws_item_sk#90, ws_quantity#91, ws_list_price#92, ws_sold_date_sk#93] +(92) Project [codegen id : 87] +Output [5]: [cs_quantity#68, cs_list_price#69, i_brand_id#74, i_class_id#75, i_category_id#76] +Input [7]: [cs_item_sk#67, cs_quantity#68, cs_list_price#69, i_item_sk#73, i_brand_id#74, i_class_id#75, i_category_id#76] + +(93) HashAggregate [codegen id : 87] +Input [5]: [cs_quantity#68, cs_list_price#69, 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(cs_quantity#68 as decimal(12,2))) * promote_precision(cast(cs_list_price#69 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)] +Aggregate Attributes [3]: [sum#77, isEmpty#78, count#79] +Results [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#80, isEmpty#81, count#82] + +(94) Exchange +Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#80, isEmpty#81, count#82] +Arguments: hashpartitioning(i_brand_id#74, i_class_id#75, i_category_id#76, 5), ENSURE_REQUIREMENTS, [id=#83] + +(95) HashAggregate [codegen id : 88] +Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#80, isEmpty#81, count#82] +Keys [3]: [i_brand_id#74, i_class_id#75, i_category_id#76] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cs_quantity#68 as decimal(12,2))) * promote_precision(cast(cs_list_price#69 as decimal(12,2)))), DecimalType(18,2))), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cs_quantity#68 as decimal(12,2))) * promote_precision(cast(cs_list_price#69 as decimal(12,2)))), DecimalType(18,2)))#84, count(1)#85] +Results [6]: [catalog AS channel#86, i_brand_id#74, i_class_id#75, i_category_id#76, sum(CheckOverflow((promote_precision(cast(cs_quantity#68 as decimal(12,2))) * promote_precision(cast(cs_list_price#69 as decimal(12,2)))), DecimalType(18,2)))#84 AS sales#87, count(1)#85 AS number_sales#88] + +(96) Filter [codegen id : 88] +Input [6]: [channel#86, i_brand_id#74, i_class_id#75, i_category_id#76, sales#87, number_sales#88] +Condition : (isnotnull(sales#87) AND (cast(sales#87 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#65, [id=#66] as decimal(32,6)))) + +(97) Scan parquet default.web_sales +Output [4]: [ws_item_sk#89, ws_quantity#90, ws_list_price#91, ws_sold_date_sk#92] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#93), dynamicpruningexpression(ws_sold_date_sk#93 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#92), dynamicpruningexpression(ws_sold_date_sk#92 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(101) ColumnarToRow [codegen id : 93] -Input [4]: [ws_item_sk#90, ws_quantity#91, ws_list_price#92, ws_sold_date_sk#93] +(98) ColumnarToRow [codegen id : 89] +Input [4]: [ws_item_sk#89, ws_quantity#90, ws_list_price#91, ws_sold_date_sk#92] -(102) Filter [codegen id : 93] -Input [4]: [ws_item_sk#90, ws_quantity#91, ws_list_price#92, ws_sold_date_sk#93] -Condition : isnotnull(ws_item_sk#90) +(99) Filter [codegen id : 89] +Input [4]: [ws_item_sk#89, ws_quantity#90, ws_list_price#91, ws_sold_date_sk#92] +Condition : isnotnull(ws_item_sk#89) -(103) Exchange -Input [4]: [ws_item_sk#90, ws_quantity#91, ws_list_price#92, ws_sold_date_sk#93] -Arguments: hashpartitioning(ws_item_sk#90, 5), ENSURE_REQUIREMENTS, [id=#94] +(100) Exchange +Input [4]: [ws_item_sk#89, ws_quantity#90, ws_list_price#91, ws_sold_date_sk#92] +Arguments: hashpartitioning(ws_item_sk#89, 5), ENSURE_REQUIREMENTS, [id=#93] -(104) Sort [codegen id : 94] -Input [4]: [ws_item_sk#90, ws_quantity#91, ws_list_price#92, ws_sold_date_sk#93] -Arguments: [ws_item_sk#90 ASC NULLS FIRST], false, 0 +(101) Sort [codegen id : 90] +Input [4]: [ws_item_sk#89, ws_quantity#90, ws_list_price#91, ws_sold_date_sk#92] +Arguments: [ws_item_sk#89 ASC NULLS FIRST], false, 0 -(105) ReusedExchange [Reuses operator id: 61] -Output [1]: [ss_item_sk#45] +(102) ReusedExchange [Reuses operator id: 58] +Output [1]: [ss_item_sk#44] -(106) Sort [codegen id : 113] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +(103) Sort [codegen id : 108] +Input [1]: [ss_item_sk#44] +Arguments: [ss_item_sk#44 ASC NULLS FIRST], false, 0 -(107) SortMergeJoin [codegen id : 137] -Left keys [1]: [ws_item_sk#90] -Right keys [1]: [ss_item_sk#45] +(104) SortMergeJoin [codegen id : 131] +Left keys [1]: [ws_item_sk#89] +Right keys [1]: [ss_item_sk#44] Join condition: None -(108) ReusedExchange [Reuses operator id: 175] -Output [1]: [d_date_sk#95] +(105) ReusedExchange [Reuses operator id: 172] +Output [1]: [d_date_sk#94] -(109) BroadcastHashJoin [codegen id : 137] -Left keys [1]: [ws_sold_date_sk#93] -Right keys [1]: [d_date_sk#95] +(106) BroadcastHashJoin [codegen id : 131] +Left keys [1]: [ws_sold_date_sk#92] +Right keys [1]: [d_date_sk#94] Join condition: None -(110) Project [codegen id : 137] -Output [3]: [ws_item_sk#90, ws_quantity#91, ws_list_price#92] -Input [5]: [ws_item_sk#90, ws_quantity#91, ws_list_price#92, ws_sold_date_sk#93, d_date_sk#95] +(107) Project [codegen id : 131] +Output [3]: [ws_item_sk#89, ws_quantity#90, ws_list_price#91] +Input [5]: [ws_item_sk#89, ws_quantity#90, ws_list_price#91, ws_sold_date_sk#92, d_date_sk#94] -(111) ReusedExchange [Reuses operator id: 75] -Output [4]: [i_item_sk#96, i_brand_id#97, i_class_id#98, i_category_id#99] +(108) ReusedExchange [Reuses operator id: 72] +Output [4]: [i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98] -(112) BroadcastHashJoin [codegen id : 137] -Left keys [1]: [ws_item_sk#90] -Right keys [1]: [i_item_sk#96] +(109) BroadcastHashJoin [codegen id : 131] +Left keys [1]: [ws_item_sk#89] +Right keys [1]: [i_item_sk#95] Join condition: None -(113) Project [codegen id : 137] -Output [5]: [ws_quantity#91, ws_list_price#92, i_brand_id#97, i_class_id#98, i_category_id#99] -Input [7]: [ws_item_sk#90, ws_quantity#91, ws_list_price#92, i_item_sk#96, i_brand_id#97, i_class_id#98, i_category_id#99] - -(114) HashAggregate [codegen id : 137] -Input [5]: [ws_quantity#91, ws_list_price#92, i_brand_id#97, i_class_id#98, i_category_id#99] -Keys [3]: [i_brand_id#97, i_class_id#98, i_category_id#99] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(ws_quantity#91 as decimal(12,2))) * promote_precision(cast(ws_list_price#92 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)] -Aggregate Attributes [3]: [sum#100, isEmpty#101, count#102] -Results [6]: [i_brand_id#97, i_class_id#98, i_category_id#99, sum#103, isEmpty#104, count#105] - -(115) Exchange -Input [6]: [i_brand_id#97, i_class_id#98, i_category_id#99, sum#103, isEmpty#104, count#105] -Arguments: hashpartitioning(i_brand_id#97, i_class_id#98, i_category_id#99, 5), ENSURE_REQUIREMENTS, [id=#106] - -(116) HashAggregate [codegen id : 138] -Input [6]: [i_brand_id#97, i_class_id#98, i_category_id#99, sum#103, isEmpty#104, count#105] -Keys [3]: [i_brand_id#97, i_class_id#98, i_category_id#99] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(ws_quantity#91 as decimal(12,2))) * promote_precision(cast(ws_list_price#92 as decimal(12,2)))), DecimalType(18,2))), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ws_quantity#91 as decimal(12,2))) * promote_precision(cast(ws_list_price#92 as decimal(12,2)))), DecimalType(18,2)))#107, count(1)#108] -Results [6]: [web AS channel#109, i_brand_id#97, i_class_id#98, i_category_id#99, sum(CheckOverflow((promote_precision(cast(ws_quantity#91 as decimal(12,2))) * promote_precision(cast(ws_list_price#92 as decimal(12,2)))), DecimalType(18,2)))#107 AS sales#110, count(1)#108 AS number_sales#111] - -(117) Filter [codegen id : 138] -Input [6]: [channel#109, i_brand_id#97, i_class_id#98, i_category_id#99, sales#110, number_sales#111] -Condition : (isnotnull(sales#110) AND (cast(sales#110 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) - -(118) Union - -(119) HashAggregate [codegen id : 139] -Input [6]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sales#64, number_sales#65] -Keys [4]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51] -Functions [2]: [partial_sum(sales#64), partial_sum(number_sales#65)] -Aggregate Attributes [3]: [sum#112, isEmpty#113, sum#114] -Results [7]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum#115, isEmpty#116, sum#117] - -(120) Exchange -Input [7]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum#115, isEmpty#116, sum#117] -Arguments: hashpartitioning(channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, 5), ENSURE_REQUIREMENTS, [id=#118] - -(121) HashAggregate [codegen id : 140] -Input [7]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum#115, isEmpty#116, sum#117] -Keys [4]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51] -Functions [2]: [sum(sales#64), sum(number_sales#65)] -Aggregate Attributes [2]: [sum(sales#64)#119, sum(number_sales#65)#120] -Results [6]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum(sales#64)#119 AS sum_sales#121, sum(number_sales#65)#120 AS number_sales#122] - -(122) ReusedExchange [Reuses operator id: 120] -Output [7]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum#115, isEmpty#116, sum#117] - -(123) HashAggregate [codegen id : 280] -Input [7]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum#115, isEmpty#116, sum#117] -Keys [4]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51] -Functions [2]: [sum(sales#64), sum(number_sales#65)] -Aggregate Attributes [2]: [sum(sales#64)#119, sum(number_sales#65)#120] -Results [5]: [channel#63, i_brand_id#49, i_class_id#50, sum(sales#64)#119 AS sum_sales#121, sum(number_sales#65)#120 AS number_sales#122] - -(124) HashAggregate [codegen id : 280] -Input [5]: [channel#63, i_brand_id#49, i_class_id#50, sum_sales#121, number_sales#122] -Keys [3]: [channel#63, i_brand_id#49, i_class_id#50] -Functions [2]: [partial_sum(sum_sales#121), partial_sum(number_sales#122)] -Aggregate Attributes [3]: [sum#123, isEmpty#124, sum#125] -Results [6]: [channel#63, i_brand_id#49, i_class_id#50, sum#126, isEmpty#127, sum#128] - -(125) Exchange -Input [6]: [channel#63, i_brand_id#49, i_class_id#50, sum#126, isEmpty#127, sum#128] -Arguments: hashpartitioning(channel#63, i_brand_id#49, i_class_id#50, 5), ENSURE_REQUIREMENTS, [id=#129] - -(126) HashAggregate [codegen id : 281] -Input [6]: [channel#63, i_brand_id#49, i_class_id#50, sum#126, isEmpty#127, sum#128] -Keys [3]: [channel#63, i_brand_id#49, i_class_id#50] -Functions [2]: [sum(sum_sales#121), sum(number_sales#122)] -Aggregate Attributes [2]: [sum(sum_sales#121)#130, sum(number_sales#122)#131] -Results [6]: [channel#63, i_brand_id#49, i_class_id#50, null AS i_category_id#132, sum(sum_sales#121)#130 AS sum(sum_sales)#133, sum(number_sales#122)#131 AS sum(number_sales)#134] - -(127) ReusedExchange [Reuses operator id: 120] -Output [7]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum#115, isEmpty#116, sum#117] - -(128) HashAggregate [codegen id : 421] -Input [7]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum#115, isEmpty#116, sum#117] -Keys [4]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51] -Functions [2]: [sum(sales#64), sum(number_sales#65)] -Aggregate Attributes [2]: [sum(sales#64)#119, sum(number_sales#65)#120] -Results [4]: [channel#63, i_brand_id#49, sum(sales#64)#119 AS sum_sales#121, sum(number_sales#65)#120 AS number_sales#122] - -(129) HashAggregate [codegen id : 421] -Input [4]: [channel#63, i_brand_id#49, sum_sales#121, number_sales#122] -Keys [2]: [channel#63, i_brand_id#49] -Functions [2]: [partial_sum(sum_sales#121), partial_sum(number_sales#122)] -Aggregate Attributes [3]: [sum#135, isEmpty#136, sum#137] -Results [5]: [channel#63, i_brand_id#49, sum#138, isEmpty#139, sum#140] - -(130) Exchange -Input [5]: [channel#63, i_brand_id#49, sum#138, isEmpty#139, sum#140] -Arguments: hashpartitioning(channel#63, i_brand_id#49, 5), ENSURE_REQUIREMENTS, [id=#141] - -(131) HashAggregate [codegen id : 422] -Input [5]: [channel#63, i_brand_id#49, sum#138, isEmpty#139, sum#140] -Keys [2]: [channel#63, i_brand_id#49] -Functions [2]: [sum(sum_sales#121), sum(number_sales#122)] -Aggregate Attributes [2]: [sum(sum_sales#121)#142, sum(number_sales#122)#143] -Results [6]: [channel#63, i_brand_id#49, null AS i_class_id#144, null AS i_category_id#145, sum(sum_sales#121)#142 AS sum(sum_sales)#146, sum(number_sales#122)#143 AS sum(number_sales)#147] - -(132) ReusedExchange [Reuses operator id: 120] -Output [7]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum#115, isEmpty#116, sum#117] - -(133) HashAggregate [codegen id : 562] -Input [7]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum#115, isEmpty#116, sum#117] -Keys [4]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51] -Functions [2]: [sum(sales#64), sum(number_sales#65)] -Aggregate Attributes [2]: [sum(sales#64)#119, sum(number_sales#65)#120] -Results [3]: [channel#63, sum(sales#64)#119 AS sum_sales#121, sum(number_sales#65)#120 AS number_sales#122] - -(134) HashAggregate [codegen id : 562] -Input [3]: [channel#63, sum_sales#121, number_sales#122] -Keys [1]: [channel#63] -Functions [2]: [partial_sum(sum_sales#121), partial_sum(number_sales#122)] -Aggregate Attributes [3]: [sum#148, isEmpty#149, sum#150] -Results [4]: [channel#63, sum#151, isEmpty#152, sum#153] - -(135) Exchange -Input [4]: [channel#63, sum#151, isEmpty#152, sum#153] -Arguments: hashpartitioning(channel#63, 5), ENSURE_REQUIREMENTS, [id=#154] - -(136) HashAggregate [codegen id : 563] -Input [4]: [channel#63, sum#151, isEmpty#152, sum#153] -Keys [1]: [channel#63] -Functions [2]: [sum(sum_sales#121), sum(number_sales#122)] -Aggregate Attributes [2]: [sum(sum_sales#121)#155, sum(number_sales#122)#156] -Results [6]: [channel#63, null AS i_brand_id#157, null AS i_class_id#158, null AS i_category_id#159, sum(sum_sales#121)#155 AS sum(sum_sales)#160, sum(number_sales#122)#156 AS sum(number_sales)#161] - -(137) ReusedExchange [Reuses operator id: 120] -Output [7]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum#115, isEmpty#116, sum#117] - -(138) HashAggregate [codegen id : 703] -Input [7]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum#115, isEmpty#116, sum#117] -Keys [4]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51] -Functions [2]: [sum(sales#64), sum(number_sales#65)] -Aggregate Attributes [2]: [sum(sales#64)#119, sum(number_sales#65)#120] -Results [2]: [sum(sales#64)#119 AS sum_sales#121, sum(number_sales#65)#120 AS number_sales#122] - -(139) HashAggregate [codegen id : 703] -Input [2]: [sum_sales#121, number_sales#122] +(110) Project [codegen id : 131] +Output [5]: [ws_quantity#90, ws_list_price#91, i_brand_id#96, i_class_id#97, i_category_id#98] +Input [7]: [ws_item_sk#89, ws_quantity#90, ws_list_price#91, i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98] + +(111) HashAggregate [codegen id : 131] +Input [5]: [ws_quantity#90, ws_list_price#91, 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(ws_quantity#90 as decimal(12,2))) * promote_precision(cast(ws_list_price#91 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)] +Aggregate Attributes [3]: [sum#99, isEmpty#100, count#101] +Results [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#102, isEmpty#103, count#104] + +(112) Exchange +Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#102, isEmpty#103, count#104] +Arguments: hashpartitioning(i_brand_id#96, i_class_id#97, i_category_id#98, 5), ENSURE_REQUIREMENTS, [id=#105] + +(113) HashAggregate [codegen id : 132] +Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#102, isEmpty#103, count#104] +Keys [3]: [i_brand_id#96, i_class_id#97, i_category_id#98] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(ws_quantity#90 as decimal(12,2))) * promote_precision(cast(ws_list_price#91 as decimal(12,2)))), DecimalType(18,2))), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ws_quantity#90 as decimal(12,2))) * promote_precision(cast(ws_list_price#91 as decimal(12,2)))), DecimalType(18,2)))#106, count(1)#107] +Results [6]: [web AS channel#108, i_brand_id#96, i_class_id#97, i_category_id#98, sum(CheckOverflow((promote_precision(cast(ws_quantity#90 as decimal(12,2))) * promote_precision(cast(ws_list_price#91 as decimal(12,2)))), DecimalType(18,2)))#106 AS sales#109, count(1)#107 AS number_sales#110] + +(114) Filter [codegen id : 132] +Input [6]: [channel#108, i_brand_id#96, i_class_id#97, i_category_id#98, sales#109, number_sales#110] +Condition : (isnotnull(sales#109) AND (cast(sales#109 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#65, [id=#66] as decimal(32,6)))) + +(115) Union + +(116) HashAggregate [codegen id : 133] +Input [6]: [channel#62, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64] +Keys [4]: [channel#62, i_brand_id#48, i_class_id#49, i_category_id#50] +Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] +Aggregate Attributes [3]: [sum#111, isEmpty#112, sum#113] +Results [7]: [channel#62, i_brand_id#48, i_class_id#49, i_category_id#50, sum#114, isEmpty#115, sum#116] + +(117) Exchange +Input [7]: [channel#62, i_brand_id#48, i_class_id#49, i_category_id#50, sum#114, isEmpty#115, sum#116] +Arguments: hashpartitioning(channel#62, i_brand_id#48, i_class_id#49, i_category_id#50, 5), ENSURE_REQUIREMENTS, [id=#117] + +(118) HashAggregate [codegen id : 134] +Input [7]: [channel#62, i_brand_id#48, i_class_id#49, i_category_id#50, sum#114, isEmpty#115, sum#116] +Keys [4]: [channel#62, i_brand_id#48, i_class_id#49, i_category_id#50] +Functions [2]: [sum(sales#63), sum(number_sales#64)] +Aggregate Attributes [2]: [sum(sales#63)#118, sum(number_sales#64)#119] +Results [6]: [channel#62, i_brand_id#48, i_class_id#49, i_category_id#50, sum(sales#63)#118 AS sum_sales#120, sum(number_sales#64)#119 AS number_sales#121] + +(119) ReusedExchange [Reuses operator id: 117] +Output [7]: [channel#62, i_brand_id#48, i_class_id#49, i_category_id#50, sum#114, isEmpty#115, sum#116] + +(120) HashAggregate [codegen id : 268] +Input [7]: [channel#62, i_brand_id#48, i_class_id#49, i_category_id#50, sum#114, isEmpty#115, sum#116] +Keys [4]: [channel#62, i_brand_id#48, i_class_id#49, i_category_id#50] +Functions [2]: [sum(sales#63), sum(number_sales#64)] +Aggregate Attributes [2]: [sum(sales#63)#118, sum(number_sales#64)#119] +Results [5]: [channel#62, i_brand_id#48, i_class_id#49, sum(sales#63)#118 AS sum_sales#120, sum(number_sales#64)#119 AS number_sales#121] + +(121) HashAggregate [codegen id : 268] +Input [5]: [channel#62, i_brand_id#48, i_class_id#49, sum_sales#120, number_sales#121] +Keys [3]: [channel#62, i_brand_id#48, i_class_id#49] +Functions [2]: [partial_sum(sum_sales#120), partial_sum(number_sales#121)] +Aggregate Attributes [3]: [sum#122, isEmpty#123, sum#124] +Results [6]: [channel#62, i_brand_id#48, i_class_id#49, sum#125, isEmpty#126, sum#127] + +(122) Exchange +Input [6]: [channel#62, i_brand_id#48, i_class_id#49, sum#125, isEmpty#126, sum#127] +Arguments: hashpartitioning(channel#62, i_brand_id#48, i_class_id#49, 5), ENSURE_REQUIREMENTS, [id=#128] + +(123) HashAggregate [codegen id : 269] +Input [6]: [channel#62, i_brand_id#48, i_class_id#49, sum#125, isEmpty#126, sum#127] +Keys [3]: [channel#62, i_brand_id#48, i_class_id#49] +Functions [2]: [sum(sum_sales#120), sum(number_sales#121)] +Aggregate Attributes [2]: [sum(sum_sales#120)#129, sum(number_sales#121)#130] +Results [6]: [channel#62, i_brand_id#48, i_class_id#49, null AS i_category_id#131, sum(sum_sales#120)#129 AS sum(sum_sales)#132, sum(number_sales#121)#130 AS sum(number_sales)#133] + +(124) ReusedExchange [Reuses operator id: 117] +Output [7]: [channel#62, i_brand_id#48, i_class_id#49, i_category_id#50, sum#114, isEmpty#115, sum#116] + +(125) HashAggregate [codegen id : 403] +Input [7]: [channel#62, i_brand_id#48, i_class_id#49, i_category_id#50, sum#114, isEmpty#115, sum#116] +Keys [4]: [channel#62, i_brand_id#48, i_class_id#49, i_category_id#50] +Functions [2]: [sum(sales#63), sum(number_sales#64)] +Aggregate Attributes [2]: [sum(sales#63)#118, sum(number_sales#64)#119] +Results [4]: [channel#62, i_brand_id#48, sum(sales#63)#118 AS sum_sales#120, sum(number_sales#64)#119 AS number_sales#121] + +(126) HashAggregate [codegen id : 403] +Input [4]: [channel#62, i_brand_id#48, sum_sales#120, number_sales#121] +Keys [2]: [channel#62, i_brand_id#48] +Functions [2]: [partial_sum(sum_sales#120), partial_sum(number_sales#121)] +Aggregate Attributes [3]: [sum#134, isEmpty#135, sum#136] +Results [5]: [channel#62, i_brand_id#48, sum#137, isEmpty#138, sum#139] + +(127) Exchange +Input [5]: [channel#62, i_brand_id#48, sum#137, isEmpty#138, sum#139] +Arguments: hashpartitioning(channel#62, i_brand_id#48, 5), ENSURE_REQUIREMENTS, [id=#140] + +(128) HashAggregate [codegen id : 404] +Input [5]: [channel#62, i_brand_id#48, sum#137, isEmpty#138, sum#139] +Keys [2]: [channel#62, i_brand_id#48] +Functions [2]: [sum(sum_sales#120), sum(number_sales#121)] +Aggregate Attributes [2]: [sum(sum_sales#120)#141, sum(number_sales#121)#142] +Results [6]: [channel#62, i_brand_id#48, null AS i_class_id#143, null AS i_category_id#144, sum(sum_sales#120)#141 AS sum(sum_sales)#145, sum(number_sales#121)#142 AS sum(number_sales)#146] + +(129) ReusedExchange [Reuses operator id: 117] +Output [7]: [channel#62, i_brand_id#48, i_class_id#49, i_category_id#50, sum#114, isEmpty#115, sum#116] + +(130) HashAggregate [codegen id : 538] +Input [7]: [channel#62, i_brand_id#48, i_class_id#49, i_category_id#50, sum#114, isEmpty#115, sum#116] +Keys [4]: [channel#62, i_brand_id#48, i_class_id#49, i_category_id#50] +Functions [2]: [sum(sales#63), sum(number_sales#64)] +Aggregate Attributes [2]: [sum(sales#63)#118, sum(number_sales#64)#119] +Results [3]: [channel#62, sum(sales#63)#118 AS sum_sales#120, sum(number_sales#64)#119 AS number_sales#121] + +(131) HashAggregate [codegen id : 538] +Input [3]: [channel#62, sum_sales#120, number_sales#121] +Keys [1]: [channel#62] +Functions [2]: [partial_sum(sum_sales#120), partial_sum(number_sales#121)] +Aggregate Attributes [3]: [sum#147, isEmpty#148, sum#149] +Results [4]: [channel#62, sum#150, isEmpty#151, sum#152] + +(132) Exchange +Input [4]: [channel#62, sum#150, isEmpty#151, sum#152] +Arguments: hashpartitioning(channel#62, 5), ENSURE_REQUIREMENTS, [id=#153] + +(133) HashAggregate [codegen id : 539] +Input [4]: [channel#62, sum#150, isEmpty#151, sum#152] +Keys [1]: [channel#62] +Functions [2]: [sum(sum_sales#120), sum(number_sales#121)] +Aggregate Attributes [2]: [sum(sum_sales#120)#154, sum(number_sales#121)#155] +Results [6]: [channel#62, null AS i_brand_id#156, null AS i_class_id#157, null AS i_category_id#158, sum(sum_sales#120)#154 AS sum(sum_sales)#159, sum(number_sales#121)#155 AS sum(number_sales)#160] + +(134) ReusedExchange [Reuses operator id: 117] +Output [7]: [channel#62, i_brand_id#48, i_class_id#49, i_category_id#50, sum#114, isEmpty#115, sum#116] + +(135) HashAggregate [codegen id : 673] +Input [7]: [channel#62, i_brand_id#48, i_class_id#49, i_category_id#50, sum#114, isEmpty#115, sum#116] +Keys [4]: [channel#62, i_brand_id#48, i_class_id#49, i_category_id#50] +Functions [2]: [sum(sales#63), sum(number_sales#64)] +Aggregate Attributes [2]: [sum(sales#63)#118, sum(number_sales#64)#119] +Results [2]: [sum(sales#63)#118 AS sum_sales#120, sum(number_sales#64)#119 AS number_sales#121] + +(136) HashAggregate [codegen id : 673] +Input [2]: [sum_sales#120, number_sales#121] Keys: [] -Functions [2]: [partial_sum(sum_sales#121), partial_sum(number_sales#122)] -Aggregate Attributes [3]: [sum#162, isEmpty#163, sum#164] -Results [3]: [sum#165, isEmpty#166, sum#167] +Functions [2]: [partial_sum(sum_sales#120), partial_sum(number_sales#121)] +Aggregate Attributes [3]: [sum#161, isEmpty#162, sum#163] +Results [3]: [sum#164, isEmpty#165, sum#166] -(140) Exchange -Input [3]: [sum#165, isEmpty#166, sum#167] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#168] +(137) Exchange +Input [3]: [sum#164, isEmpty#165, sum#166] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#167] -(141) HashAggregate [codegen id : 704] -Input [3]: [sum#165, isEmpty#166, sum#167] +(138) HashAggregate [codegen id : 674] +Input [3]: [sum#164, isEmpty#165, sum#166] Keys: [] -Functions [2]: [sum(sum_sales#121), sum(number_sales#122)] -Aggregate Attributes [2]: [sum(sum_sales#121)#169, sum(number_sales#122)#170] -Results [6]: [null AS channel#171, null AS i_brand_id#172, null AS i_class_id#173, null AS i_category_id#174, sum(sum_sales#121)#169 AS sum(sum_sales)#175, sum(number_sales#122)#170 AS sum(number_sales)#176] +Functions [2]: [sum(sum_sales#120), sum(number_sales#121)] +Aggregate Attributes [2]: [sum(sum_sales#120)#168, sum(number_sales#121)#169] +Results [6]: [null AS channel#170, null AS i_brand_id#171, null AS i_class_id#172, null AS i_category_id#173, sum(sum_sales#120)#168 AS sum(sum_sales)#174, sum(number_sales#121)#169 AS sum(number_sales)#175] -(142) Union +(139) Union -(143) HashAggregate [codegen id : 705] -Input [6]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum_sales#121, number_sales#122] -Keys [6]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum_sales#121, number_sales#122] +(140) HashAggregate [codegen id : 675] +Input [6]: [channel#62, i_brand_id#48, i_class_id#49, i_category_id#50, sum_sales#120, number_sales#121] +Keys [6]: [channel#62, i_brand_id#48, i_class_id#49, i_category_id#50, sum_sales#120, number_sales#121] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum_sales#121, number_sales#122] +Results [6]: [channel#62, i_brand_id#48, i_class_id#49, i_category_id#50, sum_sales#120, number_sales#121] -(144) Exchange -Input [6]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum_sales#121, number_sales#122] -Arguments: hashpartitioning(channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum_sales#121, number_sales#122, 5), ENSURE_REQUIREMENTS, [id=#177] +(141) Exchange +Input [6]: [channel#62, i_brand_id#48, i_class_id#49, i_category_id#50, sum_sales#120, number_sales#121] +Arguments: hashpartitioning(channel#62, i_brand_id#48, i_class_id#49, i_category_id#50, sum_sales#120, number_sales#121, 5), ENSURE_REQUIREMENTS, [id=#176] -(145) HashAggregate [codegen id : 706] -Input [6]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum_sales#121, number_sales#122] -Keys [6]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum_sales#121, number_sales#122] +(142) HashAggregate [codegen id : 676] +Input [6]: [channel#62, i_brand_id#48, i_class_id#49, i_category_id#50, sum_sales#120, number_sales#121] +Keys [6]: [channel#62, i_brand_id#48, i_class_id#49, i_category_id#50, sum_sales#120, number_sales#121] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum_sales#121, number_sales#122] +Results [6]: [channel#62, i_brand_id#48, i_class_id#49, i_category_id#50, sum_sales#120, number_sales#121] -(146) TakeOrderedAndProject -Input [6]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum_sales#121, number_sales#122] -Arguments: 100, [channel#63 ASC NULLS FIRST, i_brand_id#49 ASC NULLS FIRST, i_class_id#50 ASC NULLS FIRST, i_category_id#51 ASC NULLS FIRST], [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum_sales#121, number_sales#122] +(143) TakeOrderedAndProject +Input [6]: [channel#62, i_brand_id#48, i_class_id#49, i_category_id#50, sum_sales#120, number_sales#121] +Arguments: 100, [channel#62 ASC NULLS FIRST, i_brand_id#48 ASC NULLS FIRST, i_class_id#49 ASC NULLS FIRST, i_category_id#50 ASC NULLS FIRST], [channel#62, i_brand_id#48, i_class_id#49, i_category_id#50, sum_sales#120, number_sales#121] ===== Subqueries ===== -Subquery:1 Hosting operator id = 81 Hosting Expression = Subquery scalar-subquery#66, [id=#67] -* HashAggregate (165) -+- Exchange (164) - +- * HashAggregate (163) - +- Union (162) - :- * Project (151) - : +- * BroadcastHashJoin Inner BuildRight (150) - : :- * ColumnarToRow (148) - : : +- Scan parquet default.store_sales (147) - : +- ReusedExchange (149) - :- * Project (156) - : +- * BroadcastHashJoin Inner BuildRight (155) - : :- * ColumnarToRow (153) - : : +- Scan parquet default.catalog_sales (152) - : +- ReusedExchange (154) - +- * Project (161) - +- * BroadcastHashJoin Inner BuildRight (160) - :- * ColumnarToRow (158) - : +- Scan parquet default.web_sales (157) - +- ReusedExchange (159) - - -(147) Scan parquet default.store_sales -Output [3]: [ss_quantity#178, ss_list_price#179, ss_sold_date_sk#180] +Subquery:1 Hosting operator id = 78 Hosting Expression = Subquery scalar-subquery#65, [id=#66] +* HashAggregate (162) ++- Exchange (161) + +- * HashAggregate (160) + +- Union (159) + :- * Project (148) + : +- * BroadcastHashJoin Inner BuildRight (147) + : :- * ColumnarToRow (145) + : : +- Scan parquet default.store_sales (144) + : +- ReusedExchange (146) + :- * Project (153) + : +- * BroadcastHashJoin Inner BuildRight (152) + : :- * ColumnarToRow (150) + : : +- Scan parquet default.catalog_sales (149) + : +- ReusedExchange (151) + +- * Project (158) + +- * BroadcastHashJoin Inner BuildRight (157) + :- * ColumnarToRow (155) + : +- Scan parquet default.web_sales (154) + +- ReusedExchange (156) + + +(144) Scan parquet default.store_sales +Output [3]: [ss_quantity#177, ss_list_price#178, ss_sold_date_sk#179] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#180), dynamicpruningexpression(ss_sold_date_sk#180 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ss_sold_date_sk#179), dynamicpruningexpression(ss_sold_date_sk#179 IN dynamicpruning#13)] ReadSchema: struct -(148) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#178, ss_list_price#179, ss_sold_date_sk#180] +(145) ColumnarToRow [codegen id : 2] +Input [3]: [ss_quantity#177, ss_list_price#178, ss_sold_date_sk#179] -(149) ReusedExchange [Reuses operator id: 180] -Output [1]: [d_date_sk#181] +(146) ReusedExchange [Reuses operator id: 177] +Output [1]: [d_date_sk#180] -(150) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#180] -Right keys [1]: [d_date_sk#181] +(147) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#179] +Right keys [1]: [d_date_sk#180] Join condition: None -(151) Project [codegen id : 2] -Output [2]: [ss_quantity#178 AS quantity#182, ss_list_price#179 AS list_price#183] -Input [4]: [ss_quantity#178, ss_list_price#179, ss_sold_date_sk#180, d_date_sk#181] +(148) Project [codegen id : 2] +Output [2]: [ss_quantity#177 AS quantity#181, ss_list_price#178 AS list_price#182] +Input [4]: [ss_quantity#177, ss_list_price#178, ss_sold_date_sk#179, d_date_sk#180] -(152) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#184, cs_list_price#185, cs_sold_date_sk#186] +(149) Scan parquet default.catalog_sales +Output [3]: [cs_quantity#183, cs_list_price#184, cs_sold_date_sk#185] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#186), dynamicpruningexpression(cs_sold_date_sk#186 IN dynamicpruning#187)] +PartitionFilters: [isnotnull(cs_sold_date_sk#185), dynamicpruningexpression(cs_sold_date_sk#185 IN dynamicpruning#186)] ReadSchema: struct -(153) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#184, cs_list_price#185, cs_sold_date_sk#186] +(150) ColumnarToRow [codegen id : 4] +Input [3]: [cs_quantity#183, cs_list_price#184, cs_sold_date_sk#185] -(154) ReusedExchange [Reuses operator id: 170] -Output [1]: [d_date_sk#188] +(151) ReusedExchange [Reuses operator id: 167] +Output [1]: [d_date_sk#187] -(155) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#186] -Right keys [1]: [d_date_sk#188] +(152) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_sold_date_sk#185] +Right keys [1]: [d_date_sk#187] Join condition: None -(156) Project [codegen id : 4] -Output [2]: [cs_quantity#184 AS quantity#189, cs_list_price#185 AS list_price#190] -Input [4]: [cs_quantity#184, cs_list_price#185, cs_sold_date_sk#186, d_date_sk#188] +(153) Project [codegen id : 4] +Output [2]: [cs_quantity#183 AS quantity#188, cs_list_price#184 AS list_price#189] +Input [4]: [cs_quantity#183, cs_list_price#184, cs_sold_date_sk#185, d_date_sk#187] -(157) Scan parquet default.web_sales -Output [3]: [ws_quantity#191, ws_list_price#192, ws_sold_date_sk#193] +(154) Scan parquet default.web_sales +Output [3]: [ws_quantity#190, ws_list_price#191, ws_sold_date_sk#192] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#193), dynamicpruningexpression(ws_sold_date_sk#193 IN dynamicpruning#187)] +PartitionFilters: [isnotnull(ws_sold_date_sk#192), dynamicpruningexpression(ws_sold_date_sk#192 IN dynamicpruning#186)] ReadSchema: struct -(158) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#191, ws_list_price#192, ws_sold_date_sk#193] +(155) ColumnarToRow [codegen id : 6] +Input [3]: [ws_quantity#190, ws_list_price#191, ws_sold_date_sk#192] -(159) ReusedExchange [Reuses operator id: 170] -Output [1]: [d_date_sk#194] +(156) ReusedExchange [Reuses operator id: 167] +Output [1]: [d_date_sk#193] -(160) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#193] -Right keys [1]: [d_date_sk#194] +(157) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#192] +Right keys [1]: [d_date_sk#193] Join condition: None -(161) Project [codegen id : 6] -Output [2]: [ws_quantity#191 AS quantity#195, ws_list_price#192 AS list_price#196] -Input [4]: [ws_quantity#191, ws_list_price#192, ws_sold_date_sk#193, d_date_sk#194] +(158) Project [codegen id : 6] +Output [2]: [ws_quantity#190 AS quantity#194, ws_list_price#191 AS list_price#195] +Input [4]: [ws_quantity#190, ws_list_price#191, ws_sold_date_sk#192, d_date_sk#193] -(162) Union +(159) Union -(163) HashAggregate [codegen id : 7] -Input [2]: [quantity#182, list_price#183] +(160) HashAggregate [codegen id : 7] +Input [2]: [quantity#181, list_price#182] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(quantity#182 as decimal(12,2))) * promote_precision(cast(list_price#183 as decimal(12,2)))), DecimalType(18,2)))] -Aggregate Attributes [2]: [sum#197, count#198] -Results [2]: [sum#199, count#200] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(quantity#181 as decimal(12,2))) * promote_precision(cast(list_price#182 as decimal(12,2)))), DecimalType(18,2)))] +Aggregate Attributes [2]: [sum#196, count#197] +Results [2]: [sum#198, count#199] -(164) Exchange -Input [2]: [sum#199, count#200] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#201] +(161) Exchange +Input [2]: [sum#198, count#199] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#200] -(165) HashAggregate [codegen id : 8] -Input [2]: [sum#199, count#200] +(162) HashAggregate [codegen id : 8] +Input [2]: [sum#198, count#199] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(quantity#182 as decimal(12,2))) * promote_precision(cast(list_price#183 as decimal(12,2)))), DecimalType(18,2)))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(quantity#182 as decimal(12,2))) * promote_precision(cast(list_price#183 as decimal(12,2)))), DecimalType(18,2)))#202] -Results [1]: [avg(CheckOverflow((promote_precision(cast(quantity#182 as decimal(12,2))) * promote_precision(cast(list_price#183 as decimal(12,2)))), DecimalType(18,2)))#202 AS average_sales#203] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(quantity#181 as decimal(12,2))) * promote_precision(cast(list_price#182 as decimal(12,2)))), DecimalType(18,2)))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(quantity#181 as decimal(12,2))) * promote_precision(cast(list_price#182 as decimal(12,2)))), DecimalType(18,2)))#201] +Results [1]: [avg(CheckOverflow((promote_precision(cast(quantity#181 as decimal(12,2))) * promote_precision(cast(list_price#182 as decimal(12,2)))), DecimalType(18,2)))#201 AS average_sales#202] -Subquery:2 Hosting operator id = 147 Hosting Expression = ss_sold_date_sk#180 IN dynamicpruning#13 +Subquery:2 Hosting operator id = 144 Hosting Expression = ss_sold_date_sk#179 IN dynamicpruning#13 -Subquery:3 Hosting operator id = 152 Hosting Expression = cs_sold_date_sk#186 IN dynamicpruning#187 -BroadcastExchange (170) -+- * Project (169) - +- * Filter (168) - +- * ColumnarToRow (167) - +- Scan parquet default.date_dim (166) +Subquery:3 Hosting operator id = 149 Hosting Expression = cs_sold_date_sk#185 IN dynamicpruning#186 +BroadcastExchange (167) ++- * Project (166) + +- * Filter (165) + +- * ColumnarToRow (164) + +- Scan parquet default.date_dim (163) -(166) Scan parquet default.date_dim -Output [2]: [d_date_sk#188, d_year#204] +(163) Scan parquet default.date_dim +Output [2]: [d_date_sk#187, d_year#203] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(167) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#188, d_year#204] +(164) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#187, d_year#203] -(168) Filter [codegen id : 1] -Input [2]: [d_date_sk#188, d_year#204] -Condition : (((isnotnull(d_year#204) AND (d_year#204 >= 1998)) AND (d_year#204 <= 2000)) AND isnotnull(d_date_sk#188)) +(165) Filter [codegen id : 1] +Input [2]: [d_date_sk#187, d_year#203] +Condition : (((isnotnull(d_year#203) AND (d_year#203 >= 1998)) AND (d_year#203 <= 2000)) AND isnotnull(d_date_sk#187)) -(169) Project [codegen id : 1] -Output [1]: [d_date_sk#188] -Input [2]: [d_date_sk#188, d_year#204] +(166) Project [codegen id : 1] +Output [1]: [d_date_sk#187] +Input [2]: [d_date_sk#187, d_year#203] -(170) BroadcastExchange -Input [1]: [d_date_sk#188] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#205] +(167) BroadcastExchange +Input [1]: [d_date_sk#187] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#204] -Subquery:4 Hosting operator id = 157 Hosting Expression = ws_sold_date_sk#193 IN dynamicpruning#187 +Subquery:4 Hosting operator id = 154 Hosting Expression = ws_sold_date_sk#192 IN dynamicpruning#186 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (175) -+- * Project (174) - +- * Filter (173) - +- * ColumnarToRow (172) - +- Scan parquet default.date_dim (171) +BroadcastExchange (172) ++- * Project (171) + +- * Filter (170) + +- * ColumnarToRow (169) + +- Scan parquet default.date_dim (168) -(171) Scan parquet default.date_dim -Output [3]: [d_date_sk#47, d_year#206, d_moy#207] +(168) Scan parquet default.date_dim +Output [3]: [d_date_sk#46, d_year#205, d_moy#206] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(172) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#47, d_year#206, d_moy#207] +(169) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#46, d_year#205, d_moy#206] -(173) Filter [codegen id : 1] -Input [3]: [d_date_sk#47, d_year#206, d_moy#207] -Condition : ((((isnotnull(d_year#206) AND isnotnull(d_moy#207)) AND (d_year#206 = 2000)) AND (d_moy#207 = 11)) AND isnotnull(d_date_sk#47)) +(170) Filter [codegen id : 1] +Input [3]: [d_date_sk#46, d_year#205, d_moy#206] +Condition : ((((isnotnull(d_year#205) AND isnotnull(d_moy#206)) AND (d_year#205 = 2000)) AND (d_moy#206 = 11)) AND isnotnull(d_date_sk#46)) -(174) Project [codegen id : 1] -Output [1]: [d_date_sk#47] -Input [3]: [d_date_sk#47, d_year#206, d_moy#207] +(171) Project [codegen id : 1] +Output [1]: [d_date_sk#46] +Input [3]: [d_date_sk#46, d_year#205, d_moy#206] -(175) BroadcastExchange -Input [1]: [d_date_sk#47] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#208] +(172) BroadcastExchange +Input [1]: [d_date_sk#46] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#207] Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (180) -+- * Project (179) - +- * Filter (178) - +- * ColumnarToRow (177) - +- Scan parquet default.date_dim (176) +BroadcastExchange (177) ++- * Project (176) + +- * Filter (175) + +- * ColumnarToRow (174) + +- Scan parquet default.date_dim (173) -(176) Scan parquet default.date_dim -Output [2]: [d_date_sk#14, d_year#209] +(173) Scan parquet default.date_dim +Output [2]: [d_date_sk#14, d_year#208] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(177) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#14, d_year#209] +(174) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#14, d_year#208] -(178) Filter [codegen id : 1] -Input [2]: [d_date_sk#14, d_year#209] -Condition : (((isnotnull(d_year#209) AND (d_year#209 >= 1999)) AND (d_year#209 <= 2001)) AND isnotnull(d_date_sk#14)) +(175) Filter [codegen id : 1] +Input [2]: [d_date_sk#14, d_year#208] +Condition : (((isnotnull(d_year#208) AND (d_year#208 >= 1999)) AND (d_year#208 <= 2001)) AND isnotnull(d_date_sk#14)) -(179) Project [codegen id : 1] +(176) Project [codegen id : 1] Output [1]: [d_date_sk#14] -Input [2]: [d_date_sk#14, d_year#209] +Input [2]: [d_date_sk#14, d_year#208] -(180) BroadcastExchange +(177) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#210] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#209] Subquery:7 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#13 Subquery:8 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#13 -Subquery:9 Hosting operator id = 99 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:9 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#65, [id=#66] -Subquery:10 Hosting operator id = 82 Hosting Expression = cs_sold_date_sk#71 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 79 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 117 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:11 Hosting operator id = 114 Hosting Expression = ReusedSubquery Subquery scalar-subquery#65, [id=#66] -Subquery:12 Hosting operator id = 100 Hosting Expression = ws_sold_date_sk#93 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#92 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt index b5378a01bfa13..856de20a40ca8 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,27 +1,27 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - WholeStageCodegen (706) + WholeStageCodegen (676) 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 (705) + WholeStageCodegen (675) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter Union - WholeStageCodegen (140) + WholeStageCodegen (134) 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 (139) + WholeStageCodegen (133) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (46) + WholeStageCodegen (44) Filter [sales] Subquery #3 WholeStageCodegen (8) HashAggregate [sum,count] [avg(CheckOverflow((promote_precision(cast(quantity as decimal(12,2))) * promote_precision(cast(list_price as decimal(12,2)))), DecimalType(18,2))),average_sales,sum,count] InputAdapter - Exchange #19 + Exchange #18 WholeStageCodegen (7) HashAggregate [quantity,list_price] [sum,count,sum,count] InputAdapter @@ -34,7 +34,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Scan parquet default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #11 + ReusedExchange [d_date_sk] #10 WholeStageCodegen (4) Project [cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] @@ -42,7 +42,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter Scan parquet default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #20 + BroadcastExchange #19 WholeStageCodegen (1) Project [d_date_sk] Filter [d_year,d_date_sk] @@ -50,7 +50,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk] #20 + ReusedExchange [d_date_sk] #19 WholeStageCodegen (6) Project [ws_quantity,ws_list_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] @@ -59,11 +59,11 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Scan parquet default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #4 InputAdapter - ReusedExchange [d_date_sk] #20 + ReusedExchange [d_date_sk] #19 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(ss_quantity as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2))),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #3 - WholeStageCodegen (45) + WholeStageCodegen (43) 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] @@ -89,11 +89,11 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - 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] @@ -102,127 +102,122 @@ 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) - HashAggregate [brand_id,class_id,category_id] + WholeStageCodegen (18) + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] InputAdapter - Exchange [brand_id,class_id,category_id] #8 - WholeStageCodegen (18) - HashAggregate [brand_id,class_id,category_id] - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - InputAdapter - WholeStageCodegen (13) - Sort [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #9 - WholeStageCodegen (12) - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #10 - WholeStageCodegen (11) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #11 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] + WholeStageCodegen (13) + Sort [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #8 + WholeStageCodegen (12) + HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #9 + 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 + BroadcastExchange #10 + WholeStageCodegen (1) + 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] #10 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (10) + SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (5) + Sort [i_brand_id,i_class_id,i_category_id] InputAdapter - ReusedExchange [d_date_sk] #11 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (10) - SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - InputAdapter - WholeStageCodegen (5) - Sort [i_brand_id,i_class_id,i_category_id] + 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 - 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] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (9) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #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.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - WholeStageCodegen (9) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (8) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #11 - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (7) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - WholeStageCodegen (17) - Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + 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 (16) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #10 InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #16 - WholeStageCodegen (16) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #11 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 InputAdapter ReusedExchange [d_date_sk] #5 InputAdapter - BroadcastExchange #17 - WholeStageCodegen (44) + BroadcastExchange #16 + WholeStageCodegen (42) SortMergeJoin [i_item_sk,ss_item_sk] InputAdapter - WholeStageCodegen (24) + WholeStageCodegen (23) Sort [i_item_sk] InputAdapter - Exchange [i_item_sk] #18 - WholeStageCodegen (23) + Exchange [i_item_sk] #17 + WholeStageCodegen (22) Filter [i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter - WholeStageCodegen (43) + WholeStageCodegen (41) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #6 - WholeStageCodegen (92) + WholeStageCodegen (88) Filter [sales] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cs_quantity as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2))),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #21 - WholeStageCodegen (91) + Exchange [i_brand_id,i_class_id,i_category_id] #20 + WholeStageCodegen (87) 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] @@ -230,32 +225,32 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [cs_sold_date_sk,d_date_sk] SortMergeJoin [cs_item_sk,ss_item_sk] InputAdapter - WholeStageCodegen (48) + WholeStageCodegen (46) Sort [cs_item_sk] InputAdapter - Exchange [cs_item_sk] #22 - WholeStageCodegen (47) + Exchange [cs_item_sk] #21 + WholeStageCodegen (45) Filter [cs_item_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter - WholeStageCodegen (67) + WholeStageCodegen (64) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #6 InputAdapter ReusedExchange [d_date_sk] #5 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #17 - WholeStageCodegen (138) + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #16 + WholeStageCodegen (132) Filter [sales] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(ws_quantity as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2))),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #23 - WholeStageCodegen (137) + Exchange [i_brand_id,i_class_id,i_category_id] #22 + WholeStageCodegen (131) 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] @@ -263,57 +258,57 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [ws_sold_date_sk,d_date_sk] SortMergeJoin [ws_item_sk,ss_item_sk] InputAdapter - WholeStageCodegen (94) + WholeStageCodegen (90) Sort [ws_item_sk] InputAdapter - Exchange [ws_item_sk] #24 - WholeStageCodegen (93) + Exchange [ws_item_sk] #23 + WholeStageCodegen (89) Filter [ws_item_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter - WholeStageCodegen (113) + WholeStageCodegen (108) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #6 InputAdapter ReusedExchange [d_date_sk] #5 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #17 - WholeStageCodegen (281) + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #16 + WholeStageCodegen (269) 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] #25 - WholeStageCodegen (280) + Exchange [channel,i_brand_id,i_class_id] #24 + WholeStageCodegen (268) 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 ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (422) + WholeStageCodegen (404) 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] #26 - WholeStageCodegen (421) + Exchange [channel,i_brand_id] #25 + WholeStageCodegen (403) 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 ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (563) + WholeStageCodegen (539) 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] #27 - WholeStageCodegen (562) + Exchange [channel] #26 + WholeStageCodegen (538) 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 ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (704) + WholeStageCodegen (674) HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter - Exchange #28 - WholeStageCodegen (703) + Exchange #27 + WholeStageCodegen (673) 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 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 5d0a71ecbf8a2..2438fa9d7eb57 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt @@ -1,131 +1,129 @@ == Physical Plan == -TakeOrderedAndProject (127) -+- * HashAggregate (126) - +- Exchange (125) - +- * HashAggregate (124) - +- Union (123) - :- * HashAggregate (102) - : +- Exchange (101) - : +- * HashAggregate (100) - : +- Union (99) - : :- * Filter (68) - : : +- * HashAggregate (67) - : : +- Exchange (66) - : : +- * HashAggregate (65) - : : +- * Project (64) - : : +- * BroadcastHashJoin Inner BuildRight (63) - : : :- * Project (61) - : : : +- * BroadcastHashJoin Inner BuildRight (60) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (53) +TakeOrderedAndProject (125) ++- * HashAggregate (124) + +- Exchange (123) + +- * HashAggregate (122) + +- Union (121) + :- * HashAggregate (100) + : +- Exchange (99) + : +- * HashAggregate (98) + : +- Union (97) + : :- * Filter (66) + : : +- * HashAggregate (65) + : : +- Exchange (64) + : : +- * HashAggregate (63) + : : +- * Project (62) + : : +- * BroadcastHashJoin Inner BuildRight (61) + : : :- * Project (59) + : : : +- * BroadcastHashJoin Inner BuildRight (58) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (51) : : : : :- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.store_sales (1) - : : : : +- BroadcastExchange (52) - : : : : +- * Project (51) - : : : : +- * BroadcastHashJoin Inner BuildRight (50) + : : : : +- BroadcastExchange (50) + : : : : +- * Project (49) + : : : : +- * BroadcastHashJoin Inner BuildRight (48) : : : : :- * Filter (6) : : : : : +- * ColumnarToRow (5) : : : : : +- Scan parquet default.item (4) - : : : : +- BroadcastExchange (49) - : : : : +- * HashAggregate (48) - : : : : +- * HashAggregate (47) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) - : : : : :- * HashAggregate (35) - : : : : : +- Exchange (34) - : : : : : +- * HashAggregate (33) - : : : : : +- * Project (32) - : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : :- * Project (29) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : : :- * Filter (9) - : : : : : : : +- * ColumnarToRow (8) - : : : : : : : +- Scan parquet default.store_sales (7) - : : : : : : +- BroadcastExchange (27) - : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : : :- * Filter (12) - : : : : : : : +- * ColumnarToRow (11) - : : : : : : : +- Scan parquet default.item (10) - : : : : : : +- BroadcastExchange (25) - : : : : : : +- * Project (24) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : : :- * 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) - : : : : : : +- ReusedExchange (22) - : : : : : +- ReusedExchange (30) - : : : : +- BroadcastExchange (45) - : : : : +- * Project (44) - : : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : : :- * Project (41) - : : : : : +- * BroadcastHashJoin Inner BuildRight (40) - : : : : : :- * Filter (38) - : : : : : : +- * ColumnarToRow (37) - : : : : : : +- Scan parquet default.web_sales (36) - : : : : : +- ReusedExchange (39) - : : : : +- ReusedExchange (42) - : : : +- BroadcastExchange (59) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (58) - : : : :- * Filter (56) - : : : : +- * ColumnarToRow (55) - : : : : +- Scan parquet default.item (54) - : : : +- ReusedExchange (57) - : : +- ReusedExchange (62) - : :- * Filter (83) - : : +- * HashAggregate (82) - : : +- Exchange (81) - : : +- * HashAggregate (80) - : : +- * Project (79) - : : +- * BroadcastHashJoin Inner BuildRight (78) - : : :- * Project (76) - : : : +- * BroadcastHashJoin Inner BuildRight (75) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (73) - : : : : :- * Filter (71) - : : : : : +- * ColumnarToRow (70) - : : : : : +- Scan parquet default.catalog_sales (69) - : : : : +- ReusedExchange (72) - : : : +- ReusedExchange (74) - : : +- ReusedExchange (77) - : +- * Filter (98) - : +- * HashAggregate (97) - : +- Exchange (96) - : +- * HashAggregate (95) - : +- * Project (94) - : +- * BroadcastHashJoin Inner BuildRight (93) - : :- * Project (91) - : : +- * BroadcastHashJoin Inner BuildRight (90) - : : :- * BroadcastHashJoin LeftSemi BuildRight (88) - : : : :- * Filter (86) - : : : : +- * ColumnarToRow (85) - : : : : +- Scan parquet default.web_sales (84) - : : : +- ReusedExchange (87) - : : +- ReusedExchange (89) - : +- ReusedExchange (92) - :- * HashAggregate (107) - : +- Exchange (106) - : +- * HashAggregate (105) - : +- * HashAggregate (104) - : +- ReusedExchange (103) - :- * HashAggregate (112) - : +- Exchange (111) - : +- * HashAggregate (110) - : +- * HashAggregate (109) - : +- ReusedExchange (108) - :- * HashAggregate (117) - : +- Exchange (116) - : +- * HashAggregate (115) - : +- * HashAggregate (114) - : +- ReusedExchange (113) - +- * HashAggregate (122) - +- Exchange (121) - +- * HashAggregate (120) - +- * HashAggregate (119) - +- ReusedExchange (118) + : : : : +- BroadcastExchange (47) + : : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) + : : : : :- * HashAggregate (35) + : : : : : +- Exchange (34) + : : : : : +- * HashAggregate (33) + : : : : : +- * Project (32) + : : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : : :- * Project (29) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : : :- * Filter (9) + : : : : : : : +- * ColumnarToRow (8) + : : : : : : : +- Scan parquet default.store_sales (7) + : : : : : : +- BroadcastExchange (27) + : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) + : : : : : : :- * Filter (12) + : : : : : : : +- * ColumnarToRow (11) + : : : : : : : +- Scan parquet default.item (10) + : : : : : : +- BroadcastExchange (25) + : : : : : : +- * Project (24) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : : :- * 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) + : : : : : : +- ReusedExchange (22) + : : : : : +- ReusedExchange (30) + : : : : +- BroadcastExchange (45) + : : : : +- * Project (44) + : : : : +- * BroadcastHashJoin Inner BuildRight (43) + : : : : :- * Project (41) + : : : : : +- * BroadcastHashJoin Inner BuildRight (40) + : : : : : :- * Filter (38) + : : : : : : +- * ColumnarToRow (37) + : : : : : : +- Scan parquet default.web_sales (36) + : : : : : +- ReusedExchange (39) + : : : : +- ReusedExchange (42) + : : : +- BroadcastExchange (57) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (56) + : : : :- * Filter (54) + : : : : +- * ColumnarToRow (53) + : : : : +- Scan parquet default.item (52) + : : : +- ReusedExchange (55) + : : +- ReusedExchange (60) + : :- * Filter (81) + : : +- * HashAggregate (80) + : : +- Exchange (79) + : : +- * HashAggregate (78) + : : +- * Project (77) + : : +- * BroadcastHashJoin Inner BuildRight (76) + : : :- * Project (74) + : : : +- * BroadcastHashJoin Inner BuildRight (73) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (71) + : : : : :- * Filter (69) + : : : : : +- * ColumnarToRow (68) + : : : : : +- Scan parquet default.catalog_sales (67) + : : : : +- ReusedExchange (70) + : : : +- ReusedExchange (72) + : : +- ReusedExchange (75) + : +- * Filter (96) + : +- * HashAggregate (95) + : +- Exchange (94) + : +- * HashAggregate (93) + : +- * Project (92) + : +- * BroadcastHashJoin Inner BuildRight (91) + : :- * Project (89) + : : +- * BroadcastHashJoin Inner BuildRight (88) + : : :- * BroadcastHashJoin LeftSemi BuildRight (86) + : : : :- * Filter (84) + : : : : +- * ColumnarToRow (83) + : : : : +- Scan parquet default.web_sales (82) + : : : +- ReusedExchange (85) + : : +- ReusedExchange (87) + : +- ReusedExchange (90) + :- * HashAggregate (105) + : +- Exchange (104) + : +- * HashAggregate (103) + : +- * HashAggregate (102) + : +- ReusedExchange (101) + :- * HashAggregate (110) + : +- Exchange (109) + : +- * HashAggregate (108) + : +- * HashAggregate (107) + : +- ReusedExchange (106) + :- * HashAggregate (115) + : +- Exchange (114) + : +- * HashAggregate (113) + : +- * HashAggregate (112) + : +- ReusedExchange (111) + +- * HashAggregate (120) + +- Exchange (119) + +- * HashAggregate (118) + +- * HashAggregate (117) + +- ReusedExchange (116) (1) Scan parquet default.store_sales @@ -228,7 +226,7 @@ Join condition: None 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) ReusedExchange [Reuses operator id: 161] +(22) ReusedExchange [Reuses operator id: 159] Output [1]: [d_date_sk#24] (23) BroadcastHashJoin [codegen id : 3] @@ -262,7 +260,7 @@ Join condition: None 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] -(30) ReusedExchange [Reuses operator id: 161] +(30) ReusedExchange [Reuses operator id: 159] Output [1]: [d_date_sk#27] (31) BroadcastHashJoin [codegen id : 6] @@ -319,7 +317,7 @@ Join condition: None 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] -(42) ReusedExchange [Reuses operator id: 161] +(42) ReusedExchange [Reuses operator id: 159] Output [1]: [d_date_sk#38] (43) BroadcastHashJoin [codegen id : 9] @@ -340,112 +338,98 @@ Left keys [6]: [coalesce(brand_id#28, 0), isnull(brand_id#28), coalesce(class_id 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 -(47) 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] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] - -(48) 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] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] - -(49) BroadcastExchange +(47) 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] -(50) BroadcastHashJoin [codegen id : 11] +(48) 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] Join condition: None -(51) Project [codegen id : 11] +(49) 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] -(52) BroadcastExchange +(50) BroadcastExchange Input [1]: [ss_item_sk#41] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] -(53) BroadcastHashJoin [codegen id : 25] +(51) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#41] Join condition: None -(54) Scan parquet default.item +(52) Scan parquet default.item Output [4]: [i_item_sk#43, i_brand_id#44, i_class_id#45, i_category_id#46] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(55) ColumnarToRow [codegen id : 23] +(53) ColumnarToRow [codegen id : 23] Input [4]: [i_item_sk#43, i_brand_id#44, i_class_id#45, i_category_id#46] -(56) Filter [codegen id : 23] +(54) Filter [codegen id : 23] Input [4]: [i_item_sk#43, i_brand_id#44, i_class_id#45, i_category_id#46] Condition : isnotnull(i_item_sk#43) -(57) ReusedExchange [Reuses operator id: 52] +(55) ReusedExchange [Reuses operator id: 50] Output [1]: [ss_item_sk#41] -(58) BroadcastHashJoin [codegen id : 23] +(56) BroadcastHashJoin [codegen id : 23] Left keys [1]: [i_item_sk#43] Right keys [1]: [ss_item_sk#41] Join condition: None -(59) BroadcastExchange +(57) BroadcastExchange Input [4]: [i_item_sk#43, i_brand_id#44, i_class_id#45, i_category_id#46] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#47] -(60) BroadcastHashJoin [codegen id : 25] +(58) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#43] Join condition: None -(61) Project [codegen id : 25] +(59) Project [codegen id : 25] Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#44, i_class_id#45, i_category_id#46] Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#43, i_brand_id#44, i_class_id#45, i_category_id#46] -(62) ReusedExchange [Reuses operator id: 156] +(60) ReusedExchange [Reuses operator id: 154] Output [1]: [d_date_sk#48] -(63) BroadcastHashJoin [codegen id : 25] +(61) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#48] Join condition: None -(64) Project [codegen id : 25] +(62) Project [codegen id : 25] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#44, i_class_id#45, i_category_id#46] Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#44, i_class_id#45, i_category_id#46, d_date_sk#48] -(65) HashAggregate [codegen id : 25] +(63) HashAggregate [codegen id : 25] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#44, i_class_id#45, i_category_id#46] Keys [3]: [i_brand_id#44, i_class_id#45, i_category_id#46] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)] Aggregate Attributes [3]: [sum#49, isEmpty#50, count#51] Results [6]: [i_brand_id#44, i_class_id#45, i_category_id#46, sum#52, isEmpty#53, count#54] -(66) Exchange +(64) Exchange Input [6]: [i_brand_id#44, i_class_id#45, i_category_id#46, sum#52, isEmpty#53, count#54] Arguments: hashpartitioning(i_brand_id#44, i_class_id#45, i_category_id#46, 5), ENSURE_REQUIREMENTS, [id=#55] -(67) HashAggregate [codegen id : 26] +(65) HashAggregate [codegen id : 26] Input [6]: [i_brand_id#44, i_class_id#45, i_category_id#46, sum#52, isEmpty#53, count#54] Keys [3]: [i_brand_id#44, i_class_id#45, i_category_id#46] Functions [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2))), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#56, count(1)#57] Results [6]: [store AS channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#56 AS sales#59, count(1)#57 AS number_sales#60] -(68) Filter [codegen id : 26] +(66) Filter [codegen id : 26] Input [6]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sales#59, number_sales#60] Condition : (isnotnull(sales#59) AND (cast(sales#59 as decimal(32,6)) > cast(Subquery scalar-subquery#61, [id=#62] as decimal(32,6)))) -(69) Scan parquet default.catalog_sales +(67) Scan parquet default.catalog_sales Output [4]: [cs_item_sk#63, cs_quantity#64, cs_list_price#65, cs_sold_date_sk#66] Batched: true Location: InMemoryFileIndex [] @@ -453,68 +437,68 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#66), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(70) ColumnarToRow [codegen id : 51] +(68) ColumnarToRow [codegen id : 51] Input [4]: [cs_item_sk#63, cs_quantity#64, cs_list_price#65, cs_sold_date_sk#66] -(71) Filter [codegen id : 51] +(69) Filter [codegen id : 51] Input [4]: [cs_item_sk#63, cs_quantity#64, cs_list_price#65, cs_sold_date_sk#66] Condition : isnotnull(cs_item_sk#63) -(72) ReusedExchange [Reuses operator id: 52] +(70) ReusedExchange [Reuses operator id: 50] Output [1]: [ss_item_sk#41] -(73) BroadcastHashJoin [codegen id : 51] +(71) BroadcastHashJoin [codegen id : 51] Left keys [1]: [cs_item_sk#63] Right keys [1]: [ss_item_sk#41] Join condition: None -(74) ReusedExchange [Reuses operator id: 59] +(72) ReusedExchange [Reuses operator id: 57] Output [4]: [i_item_sk#67, i_brand_id#68, i_class_id#69, i_category_id#70] -(75) BroadcastHashJoin [codegen id : 51] +(73) BroadcastHashJoin [codegen id : 51] Left keys [1]: [cs_item_sk#63] Right keys [1]: [i_item_sk#67] Join condition: None -(76) Project [codegen id : 51] +(74) Project [codegen id : 51] Output [6]: [cs_quantity#64, cs_list_price#65, cs_sold_date_sk#66, i_brand_id#68, i_class_id#69, i_category_id#70] Input [8]: [cs_item_sk#63, cs_quantity#64, cs_list_price#65, cs_sold_date_sk#66, i_item_sk#67, i_brand_id#68, i_class_id#69, i_category_id#70] -(77) ReusedExchange [Reuses operator id: 156] +(75) ReusedExchange [Reuses operator id: 154] Output [1]: [d_date_sk#71] -(78) BroadcastHashJoin [codegen id : 51] +(76) BroadcastHashJoin [codegen id : 51] Left keys [1]: [cs_sold_date_sk#66] Right keys [1]: [d_date_sk#71] Join condition: None -(79) Project [codegen id : 51] +(77) Project [codegen id : 51] Output [5]: [cs_quantity#64, cs_list_price#65, i_brand_id#68, i_class_id#69, i_category_id#70] Input [7]: [cs_quantity#64, cs_list_price#65, cs_sold_date_sk#66, i_brand_id#68, i_class_id#69, i_category_id#70, d_date_sk#71] -(80) HashAggregate [codegen id : 51] +(78) HashAggregate [codegen id : 51] Input [5]: [cs_quantity#64, cs_list_price#65, i_brand_id#68, i_class_id#69, i_category_id#70] Keys [3]: [i_brand_id#68, i_class_id#69, i_category_id#70] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cs_quantity#64 as decimal(12,2))) * promote_precision(cast(cs_list_price#65 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)] Aggregate Attributes [3]: [sum#72, isEmpty#73, count#74] Results [6]: [i_brand_id#68, i_class_id#69, i_category_id#70, sum#75, isEmpty#76, count#77] -(81) Exchange +(79) Exchange Input [6]: [i_brand_id#68, i_class_id#69, i_category_id#70, sum#75, isEmpty#76, count#77] Arguments: hashpartitioning(i_brand_id#68, i_class_id#69, i_category_id#70, 5), ENSURE_REQUIREMENTS, [id=#78] -(82) HashAggregate [codegen id : 52] +(80) HashAggregate [codegen id : 52] Input [6]: [i_brand_id#68, i_class_id#69, i_category_id#70, sum#75, isEmpty#76, count#77] Keys [3]: [i_brand_id#68, i_class_id#69, i_category_id#70] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cs_quantity#64 as decimal(12,2))) * promote_precision(cast(cs_list_price#65 as decimal(12,2)))), DecimalType(18,2))), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cs_quantity#64 as decimal(12,2))) * promote_precision(cast(cs_list_price#65 as decimal(12,2)))), DecimalType(18,2)))#79, count(1)#80] Results [6]: [catalog AS channel#81, i_brand_id#68, i_class_id#69, i_category_id#70, sum(CheckOverflow((promote_precision(cast(cs_quantity#64 as decimal(12,2))) * promote_precision(cast(cs_list_price#65 as decimal(12,2)))), DecimalType(18,2)))#79 AS sales#82, count(1)#80 AS number_sales#83] -(83) Filter [codegen id : 52] +(81) Filter [codegen id : 52] Input [6]: [channel#81, i_brand_id#68, i_class_id#69, i_category_id#70, sales#82, number_sales#83] Condition : (isnotnull(sales#82) AND (cast(sales#82 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#61, [id=#62] as decimal(32,6)))) -(84) Scan parquet default.web_sales +(82) Scan parquet default.web_sales Output [4]: [ws_item_sk#84, ws_quantity#85, ws_list_price#86, ws_sold_date_sk#87] Batched: true Location: InMemoryFileIndex [] @@ -522,424 +506,424 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#87), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(85) ColumnarToRow [codegen id : 77] +(83) ColumnarToRow [codegen id : 77] Input [4]: [ws_item_sk#84, ws_quantity#85, ws_list_price#86, ws_sold_date_sk#87] -(86) Filter [codegen id : 77] +(84) Filter [codegen id : 77] Input [4]: [ws_item_sk#84, ws_quantity#85, ws_list_price#86, ws_sold_date_sk#87] Condition : isnotnull(ws_item_sk#84) -(87) ReusedExchange [Reuses operator id: 52] +(85) ReusedExchange [Reuses operator id: 50] Output [1]: [ss_item_sk#41] -(88) BroadcastHashJoin [codegen id : 77] +(86) BroadcastHashJoin [codegen id : 77] Left keys [1]: [ws_item_sk#84] Right keys [1]: [ss_item_sk#41] Join condition: None -(89) ReusedExchange [Reuses operator id: 59] +(87) ReusedExchange [Reuses operator id: 57] Output [4]: [i_item_sk#88, i_brand_id#89, i_class_id#90, i_category_id#91] -(90) BroadcastHashJoin [codegen id : 77] +(88) BroadcastHashJoin [codegen id : 77] Left keys [1]: [ws_item_sk#84] Right keys [1]: [i_item_sk#88] Join condition: None -(91) Project [codegen id : 77] +(89) Project [codegen id : 77] Output [6]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#87, i_brand_id#89, i_class_id#90, i_category_id#91] Input [8]: [ws_item_sk#84, ws_quantity#85, ws_list_price#86, ws_sold_date_sk#87, i_item_sk#88, i_brand_id#89, i_class_id#90, i_category_id#91] -(92) ReusedExchange [Reuses operator id: 156] +(90) ReusedExchange [Reuses operator id: 154] Output [1]: [d_date_sk#92] -(93) BroadcastHashJoin [codegen id : 77] +(91) BroadcastHashJoin [codegen id : 77] Left keys [1]: [ws_sold_date_sk#87] Right keys [1]: [d_date_sk#92] Join condition: None -(94) Project [codegen id : 77] +(92) Project [codegen id : 77] Output [5]: [ws_quantity#85, ws_list_price#86, i_brand_id#89, i_class_id#90, i_category_id#91] Input [7]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#87, i_brand_id#89, i_class_id#90, i_category_id#91, d_date_sk#92] -(95) HashAggregate [codegen id : 77] +(93) HashAggregate [codegen id : 77] Input [5]: [ws_quantity#85, ws_list_price#86, i_brand_id#89, i_class_id#90, i_category_id#91] Keys [3]: [i_brand_id#89, i_class_id#90, i_category_id#91] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(ws_quantity#85 as decimal(12,2))) * promote_precision(cast(ws_list_price#86 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)] Aggregate Attributes [3]: [sum#93, isEmpty#94, count#95] Results [6]: [i_brand_id#89, i_class_id#90, i_category_id#91, sum#96, isEmpty#97, count#98] -(96) Exchange +(94) Exchange Input [6]: [i_brand_id#89, i_class_id#90, i_category_id#91, sum#96, isEmpty#97, count#98] Arguments: hashpartitioning(i_brand_id#89, i_class_id#90, i_category_id#91, 5), ENSURE_REQUIREMENTS, [id=#99] -(97) HashAggregate [codegen id : 78] +(95) HashAggregate [codegen id : 78] Input [6]: [i_brand_id#89, i_class_id#90, i_category_id#91, sum#96, isEmpty#97, count#98] Keys [3]: [i_brand_id#89, i_class_id#90, i_category_id#91] Functions [2]: [sum(CheckOverflow((promote_precision(cast(ws_quantity#85 as decimal(12,2))) * promote_precision(cast(ws_list_price#86 as decimal(12,2)))), DecimalType(18,2))), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ws_quantity#85 as decimal(12,2))) * promote_precision(cast(ws_list_price#86 as decimal(12,2)))), DecimalType(18,2)))#100, count(1)#101] Results [6]: [web AS channel#102, i_brand_id#89, i_class_id#90, i_category_id#91, sum(CheckOverflow((promote_precision(cast(ws_quantity#85 as decimal(12,2))) * promote_precision(cast(ws_list_price#86 as decimal(12,2)))), DecimalType(18,2)))#100 AS sales#103, count(1)#101 AS number_sales#104] -(98) Filter [codegen id : 78] +(96) Filter [codegen id : 78] Input [6]: [channel#102, i_brand_id#89, i_class_id#90, i_category_id#91, sales#103, number_sales#104] Condition : (isnotnull(sales#103) AND (cast(sales#103 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#61, [id=#62] as decimal(32,6)))) -(99) Union +(97) Union -(100) HashAggregate [codegen id : 79] +(98) HashAggregate [codegen id : 79] Input [6]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sales#59, number_sales#60] Keys [4]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46] Functions [2]: [partial_sum(sales#59), partial_sum(number_sales#60)] Aggregate Attributes [3]: [sum#105, isEmpty#106, sum#107] Results [7]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum#108, isEmpty#109, sum#110] -(101) Exchange +(99) Exchange Input [7]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum#108, isEmpty#109, sum#110] Arguments: hashpartitioning(channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, 5), ENSURE_REQUIREMENTS, [id=#111] -(102) HashAggregate [codegen id : 80] +(100) HashAggregate [codegen id : 80] Input [7]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum#108, isEmpty#109, sum#110] Keys [4]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46] Functions [2]: [sum(sales#59), sum(number_sales#60)] Aggregate Attributes [2]: [sum(sales#59)#112, sum(number_sales#60)#113] Results [6]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum(sales#59)#112 AS sum_sales#114, sum(number_sales#60)#113 AS number_sales#115] -(103) ReusedExchange [Reuses operator id: 101] +(101) ReusedExchange [Reuses operator id: 99] Output [7]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum#108, isEmpty#109, sum#110] -(104) HashAggregate [codegen id : 160] +(102) HashAggregate [codegen id : 160] Input [7]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum#108, isEmpty#109, sum#110] Keys [4]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46] Functions [2]: [sum(sales#59), sum(number_sales#60)] Aggregate Attributes [2]: [sum(sales#59)#112, sum(number_sales#60)#113] Results [5]: [channel#58, i_brand_id#44, i_class_id#45, sum(sales#59)#112 AS sum_sales#114, sum(number_sales#60)#113 AS number_sales#115] -(105) HashAggregate [codegen id : 160] +(103) HashAggregate [codegen id : 160] Input [5]: [channel#58, i_brand_id#44, i_class_id#45, sum_sales#114, number_sales#115] Keys [3]: [channel#58, i_brand_id#44, i_class_id#45] Functions [2]: [partial_sum(sum_sales#114), partial_sum(number_sales#115)] Aggregate Attributes [3]: [sum#116, isEmpty#117, sum#118] Results [6]: [channel#58, i_brand_id#44, i_class_id#45, sum#119, isEmpty#120, sum#121] -(106) Exchange +(104) Exchange Input [6]: [channel#58, i_brand_id#44, i_class_id#45, sum#119, isEmpty#120, sum#121] Arguments: hashpartitioning(channel#58, i_brand_id#44, i_class_id#45, 5), ENSURE_REQUIREMENTS, [id=#122] -(107) HashAggregate [codegen id : 161] +(105) HashAggregate [codegen id : 161] Input [6]: [channel#58, i_brand_id#44, i_class_id#45, sum#119, isEmpty#120, sum#121] Keys [3]: [channel#58, i_brand_id#44, i_class_id#45] Functions [2]: [sum(sum_sales#114), sum(number_sales#115)] Aggregate Attributes [2]: [sum(sum_sales#114)#123, sum(number_sales#115)#124] Results [6]: [channel#58, i_brand_id#44, i_class_id#45, null AS i_category_id#125, sum(sum_sales#114)#123 AS sum(sum_sales)#126, sum(number_sales#115)#124 AS sum(number_sales)#127] -(108) ReusedExchange [Reuses operator id: 101] +(106) ReusedExchange [Reuses operator id: 99] Output [7]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum#108, isEmpty#109, sum#110] -(109) HashAggregate [codegen id : 241] +(107) HashAggregate [codegen id : 241] Input [7]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum#108, isEmpty#109, sum#110] Keys [4]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46] Functions [2]: [sum(sales#59), sum(number_sales#60)] Aggregate Attributes [2]: [sum(sales#59)#112, sum(number_sales#60)#113] Results [4]: [channel#58, i_brand_id#44, sum(sales#59)#112 AS sum_sales#114, sum(number_sales#60)#113 AS number_sales#115] -(110) HashAggregate [codegen id : 241] +(108) HashAggregate [codegen id : 241] Input [4]: [channel#58, i_brand_id#44, sum_sales#114, number_sales#115] Keys [2]: [channel#58, i_brand_id#44] Functions [2]: [partial_sum(sum_sales#114), partial_sum(number_sales#115)] Aggregate Attributes [3]: [sum#128, isEmpty#129, sum#130] Results [5]: [channel#58, i_brand_id#44, sum#131, isEmpty#132, sum#133] -(111) Exchange +(109) Exchange Input [5]: [channel#58, i_brand_id#44, sum#131, isEmpty#132, sum#133] Arguments: hashpartitioning(channel#58, i_brand_id#44, 5), ENSURE_REQUIREMENTS, [id=#134] -(112) HashAggregate [codegen id : 242] +(110) HashAggregate [codegen id : 242] Input [5]: [channel#58, i_brand_id#44, sum#131, isEmpty#132, sum#133] Keys [2]: [channel#58, i_brand_id#44] Functions [2]: [sum(sum_sales#114), sum(number_sales#115)] Aggregate Attributes [2]: [sum(sum_sales#114)#135, sum(number_sales#115)#136] Results [6]: [channel#58, i_brand_id#44, null AS i_class_id#137, null AS i_category_id#138, sum(sum_sales#114)#135 AS sum(sum_sales)#139, sum(number_sales#115)#136 AS sum(number_sales)#140] -(113) ReusedExchange [Reuses operator id: 101] +(111) ReusedExchange [Reuses operator id: 99] Output [7]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum#108, isEmpty#109, sum#110] -(114) HashAggregate [codegen id : 322] +(112) HashAggregate [codegen id : 322] Input [7]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum#108, isEmpty#109, sum#110] Keys [4]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46] Functions [2]: [sum(sales#59), sum(number_sales#60)] Aggregate Attributes [2]: [sum(sales#59)#112, sum(number_sales#60)#113] Results [3]: [channel#58, sum(sales#59)#112 AS sum_sales#114, sum(number_sales#60)#113 AS number_sales#115] -(115) HashAggregate [codegen id : 322] +(113) HashAggregate [codegen id : 322] Input [3]: [channel#58, sum_sales#114, number_sales#115] Keys [1]: [channel#58] Functions [2]: [partial_sum(sum_sales#114), partial_sum(number_sales#115)] Aggregate Attributes [3]: [sum#141, isEmpty#142, sum#143] Results [4]: [channel#58, sum#144, isEmpty#145, sum#146] -(116) Exchange +(114) Exchange Input [4]: [channel#58, sum#144, isEmpty#145, sum#146] Arguments: hashpartitioning(channel#58, 5), ENSURE_REQUIREMENTS, [id=#147] -(117) HashAggregate [codegen id : 323] +(115) HashAggregate [codegen id : 323] Input [4]: [channel#58, sum#144, isEmpty#145, sum#146] Keys [1]: [channel#58] Functions [2]: [sum(sum_sales#114), sum(number_sales#115)] Aggregate Attributes [2]: [sum(sum_sales#114)#148, sum(number_sales#115)#149] Results [6]: [channel#58, null AS i_brand_id#150, null AS i_class_id#151, null AS i_category_id#152, sum(sum_sales#114)#148 AS sum(sum_sales)#153, sum(number_sales#115)#149 AS sum(number_sales)#154] -(118) ReusedExchange [Reuses operator id: 101] +(116) ReusedExchange [Reuses operator id: 99] Output [7]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum#108, isEmpty#109, sum#110] -(119) HashAggregate [codegen id : 403] +(117) HashAggregate [codegen id : 403] Input [7]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum#108, isEmpty#109, sum#110] Keys [4]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46] Functions [2]: [sum(sales#59), sum(number_sales#60)] Aggregate Attributes [2]: [sum(sales#59)#112, sum(number_sales#60)#113] Results [2]: [sum(sales#59)#112 AS sum_sales#114, sum(number_sales#60)#113 AS number_sales#115] -(120) HashAggregate [codegen id : 403] +(118) HashAggregate [codegen id : 403] Input [2]: [sum_sales#114, number_sales#115] Keys: [] Functions [2]: [partial_sum(sum_sales#114), partial_sum(number_sales#115)] Aggregate Attributes [3]: [sum#155, isEmpty#156, sum#157] Results [3]: [sum#158, isEmpty#159, sum#160] -(121) Exchange +(119) Exchange Input [3]: [sum#158, isEmpty#159, sum#160] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#161] -(122) HashAggregate [codegen id : 404] +(120) HashAggregate [codegen id : 404] Input [3]: [sum#158, isEmpty#159, sum#160] Keys: [] Functions [2]: [sum(sum_sales#114), sum(number_sales#115)] Aggregate Attributes [2]: [sum(sum_sales#114)#162, sum(number_sales#115)#163] Results [6]: [null AS channel#164, null AS i_brand_id#165, null AS i_class_id#166, null AS i_category_id#167, sum(sum_sales#114)#162 AS sum(sum_sales)#168, sum(number_sales#115)#163 AS sum(number_sales)#169] -(123) Union +(121) Union -(124) HashAggregate [codegen id : 405] +(122) HashAggregate [codegen id : 405] Input [6]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum_sales#114, number_sales#115] Keys [6]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum_sales#114, number_sales#115] Functions: [] Aggregate Attributes: [] Results [6]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum_sales#114, number_sales#115] -(125) Exchange +(123) Exchange Input [6]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum_sales#114, number_sales#115] Arguments: hashpartitioning(channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum_sales#114, number_sales#115, 5), ENSURE_REQUIREMENTS, [id=#170] -(126) HashAggregate [codegen id : 406] +(124) HashAggregate [codegen id : 406] Input [6]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum_sales#114, number_sales#115] Keys [6]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum_sales#114, number_sales#115] Functions: [] Aggregate Attributes: [] Results [6]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum_sales#114, number_sales#115] -(127) TakeOrderedAndProject +(125) TakeOrderedAndProject Input [6]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum_sales#114, number_sales#115] Arguments: 100, [channel#58 ASC NULLS FIRST, i_brand_id#44 ASC NULLS FIRST, i_class_id#45 ASC NULLS FIRST, i_category_id#46 ASC NULLS FIRST], [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum_sales#114, number_sales#115] ===== Subqueries ===== -Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#61, [id=#62] -* HashAggregate (146) -+- Exchange (145) - +- * HashAggregate (144) - +- Union (143) - :- * Project (132) - : +- * BroadcastHashJoin Inner BuildRight (131) - : :- * ColumnarToRow (129) - : : +- Scan parquet default.store_sales (128) - : +- ReusedExchange (130) - :- * Project (137) - : +- * BroadcastHashJoin Inner BuildRight (136) - : :- * ColumnarToRow (134) - : : +- Scan parquet default.catalog_sales (133) - : +- ReusedExchange (135) - +- * Project (142) - +- * BroadcastHashJoin Inner BuildRight (141) - :- * ColumnarToRow (139) - : +- Scan parquet default.web_sales (138) - +- ReusedExchange (140) - - -(128) Scan parquet default.store_sales +Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#61, [id=#62] +* HashAggregate (144) ++- Exchange (143) + +- * HashAggregate (142) + +- Union (141) + :- * Project (130) + : +- * BroadcastHashJoin Inner BuildRight (129) + : :- * ColumnarToRow (127) + : : +- Scan parquet default.store_sales (126) + : +- ReusedExchange (128) + :- * Project (135) + : +- * BroadcastHashJoin Inner BuildRight (134) + : :- * ColumnarToRow (132) + : : +- Scan parquet default.catalog_sales (131) + : +- ReusedExchange (133) + +- * Project (140) + +- * BroadcastHashJoin Inner BuildRight (139) + :- * ColumnarToRow (137) + : +- Scan parquet default.web_sales (136) + +- ReusedExchange (138) + + +(126) Scan parquet default.store_sales Output [3]: [ss_quantity#171, ss_list_price#172, ss_sold_date_sk#173] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#173), dynamicpruningexpression(ss_sold_date_sk#173 IN dynamicpruning#12)] ReadSchema: struct -(129) ColumnarToRow [codegen id : 2] +(127) ColumnarToRow [codegen id : 2] Input [3]: [ss_quantity#171, ss_list_price#172, ss_sold_date_sk#173] -(130) ReusedExchange [Reuses operator id: 161] +(128) ReusedExchange [Reuses operator id: 159] Output [1]: [d_date_sk#174] -(131) BroadcastHashJoin [codegen id : 2] +(129) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#173] Right keys [1]: [d_date_sk#174] Join condition: None -(132) Project [codegen id : 2] +(130) Project [codegen id : 2] Output [2]: [ss_quantity#171 AS quantity#175, ss_list_price#172 AS list_price#176] Input [4]: [ss_quantity#171, ss_list_price#172, ss_sold_date_sk#173, d_date_sk#174] -(133) Scan parquet default.catalog_sales +(131) Scan parquet default.catalog_sales Output [3]: [cs_quantity#177, cs_list_price#178, cs_sold_date_sk#179] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#179), dynamicpruningexpression(cs_sold_date_sk#179 IN dynamicpruning#180)] ReadSchema: struct -(134) ColumnarToRow [codegen id : 4] +(132) ColumnarToRow [codegen id : 4] Input [3]: [cs_quantity#177, cs_list_price#178, cs_sold_date_sk#179] -(135) ReusedExchange [Reuses operator id: 151] +(133) ReusedExchange [Reuses operator id: 149] Output [1]: [d_date_sk#181] -(136) BroadcastHashJoin [codegen id : 4] +(134) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#179] Right keys [1]: [d_date_sk#181] Join condition: None -(137) Project [codegen id : 4] +(135) Project [codegen id : 4] Output [2]: [cs_quantity#177 AS quantity#182, cs_list_price#178 AS list_price#183] Input [4]: [cs_quantity#177, cs_list_price#178, cs_sold_date_sk#179, d_date_sk#181] -(138) Scan parquet default.web_sales +(136) Scan parquet default.web_sales Output [3]: [ws_quantity#184, ws_list_price#185, ws_sold_date_sk#186] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#186), dynamicpruningexpression(ws_sold_date_sk#186 IN dynamicpruning#180)] ReadSchema: struct -(139) ColumnarToRow [codegen id : 6] +(137) ColumnarToRow [codegen id : 6] Input [3]: [ws_quantity#184, ws_list_price#185, ws_sold_date_sk#186] -(140) ReusedExchange [Reuses operator id: 151] +(138) ReusedExchange [Reuses operator id: 149] Output [1]: [d_date_sk#187] -(141) BroadcastHashJoin [codegen id : 6] +(139) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#186] Right keys [1]: [d_date_sk#187] Join condition: None -(142) Project [codegen id : 6] +(140) Project [codegen id : 6] Output [2]: [ws_quantity#184 AS quantity#188, ws_list_price#185 AS list_price#189] Input [4]: [ws_quantity#184, ws_list_price#185, ws_sold_date_sk#186, d_date_sk#187] -(143) Union +(141) Union -(144) HashAggregate [codegen id : 7] +(142) HashAggregate [codegen id : 7] Input [2]: [quantity#175, list_price#176] Keys: [] Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(quantity#175 as decimal(12,2))) * promote_precision(cast(list_price#176 as decimal(12,2)))), DecimalType(18,2)))] Aggregate Attributes [2]: [sum#190, count#191] Results [2]: [sum#192, count#193] -(145) Exchange +(143) Exchange Input [2]: [sum#192, count#193] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#194] -(146) HashAggregate [codegen id : 8] +(144) HashAggregate [codegen id : 8] Input [2]: [sum#192, count#193] Keys: [] Functions [1]: [avg(CheckOverflow((promote_precision(cast(quantity#175 as decimal(12,2))) * promote_precision(cast(list_price#176 as decimal(12,2)))), DecimalType(18,2)))] Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(quantity#175 as decimal(12,2))) * promote_precision(cast(list_price#176 as decimal(12,2)))), DecimalType(18,2)))#195] Results [1]: [avg(CheckOverflow((promote_precision(cast(quantity#175 as decimal(12,2))) * promote_precision(cast(list_price#176 as decimal(12,2)))), DecimalType(18,2)))#195 AS average_sales#196] -Subquery:2 Hosting operator id = 128 Hosting Expression = ss_sold_date_sk#173 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 126 Hosting Expression = ss_sold_date_sk#173 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 133 Hosting Expression = cs_sold_date_sk#179 IN dynamicpruning#180 -BroadcastExchange (151) -+- * Project (150) - +- * Filter (149) - +- * ColumnarToRow (148) - +- Scan parquet default.date_dim (147) +Subquery:3 Hosting operator id = 131 Hosting Expression = cs_sold_date_sk#179 IN dynamicpruning#180 +BroadcastExchange (149) ++- * Project (148) + +- * Filter (147) + +- * ColumnarToRow (146) + +- Scan parquet default.date_dim (145) -(147) Scan parquet default.date_dim +(145) Scan parquet default.date_dim Output [2]: [d_date_sk#181, d_year#197] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(148) ColumnarToRow [codegen id : 1] +(146) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#181, d_year#197] -(149) Filter [codegen id : 1] +(147) Filter [codegen id : 1] Input [2]: [d_date_sk#181, d_year#197] Condition : (((isnotnull(d_year#197) AND (d_year#197 >= 1998)) AND (d_year#197 <= 2000)) AND isnotnull(d_date_sk#181)) -(150) Project [codegen id : 1] +(148) Project [codegen id : 1] Output [1]: [d_date_sk#181] Input [2]: [d_date_sk#181, d_year#197] -(151) BroadcastExchange +(149) BroadcastExchange Input [1]: [d_date_sk#181] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#198] -Subquery:4 Hosting operator id = 138 Hosting Expression = ws_sold_date_sk#186 IN dynamicpruning#180 +Subquery:4 Hosting operator id = 136 Hosting Expression = ws_sold_date_sk#186 IN dynamicpruning#180 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (156) -+- * Project (155) - +- * Filter (154) - +- * ColumnarToRow (153) - +- Scan parquet default.date_dim (152) +BroadcastExchange (154) ++- * Project (153) + +- * Filter (152) + +- * ColumnarToRow (151) + +- Scan parquet default.date_dim (150) -(152) Scan parquet default.date_dim +(150) Scan parquet default.date_dim Output [3]: [d_date_sk#48, d_year#199, d_moy#200] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(153) ColumnarToRow [codegen id : 1] +(151) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#48, d_year#199, d_moy#200] -(154) Filter [codegen id : 1] +(152) Filter [codegen id : 1] Input [3]: [d_date_sk#48, d_year#199, d_moy#200] Condition : ((((isnotnull(d_year#199) AND isnotnull(d_moy#200)) AND (d_year#199 = 2000)) AND (d_moy#200 = 11)) AND isnotnull(d_date_sk#48)) -(155) Project [codegen id : 1] +(153) Project [codegen id : 1] Output [1]: [d_date_sk#48] Input [3]: [d_date_sk#48, d_year#199, d_moy#200] -(156) BroadcastExchange +(154) BroadcastExchange Input [1]: [d_date_sk#48] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#201] Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (161) -+- * Project (160) - +- * Filter (159) - +- * ColumnarToRow (158) - +- Scan parquet default.date_dim (157) +BroadcastExchange (159) ++- * Project (158) + +- * Filter (157) + +- * ColumnarToRow (156) + +- Scan parquet default.date_dim (155) -(157) Scan parquet default.date_dim +(155) Scan parquet default.date_dim Output [2]: [d_date_sk#27, d_year#202] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(158) ColumnarToRow [codegen id : 1] +(156) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#27, d_year#202] -(159) Filter [codegen id : 1] +(157) Filter [codegen id : 1] Input [2]: [d_date_sk#27, d_year#202] Condition : (((isnotnull(d_year#202) AND (d_year#202 >= 1999)) AND (d_year#202 <= 2001)) AND isnotnull(d_date_sk#27)) -(160) Project [codegen id : 1] +(158) Project [codegen id : 1] Output [1]: [d_date_sk#27] Input [2]: [d_date_sk#27, d_year#202] -(161) BroadcastExchange +(159) BroadcastExchange Input [1]: [d_date_sk#27] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#203] @@ -947,12 +931,12 @@ Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN d Subquery:8 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 83 Hosting Expression = ReusedSubquery Subquery scalar-subquery#61, [id=#62] +Subquery:9 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#61, [id=#62] -Subquery:10 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#66 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#66 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 98 Hosting Expression = ReusedSubquery Subquery scalar-subquery#61, [id=#62] +Subquery:11 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#61, [id=#62] -Subquery:12 Hosting operator id = 84 Hosting Expression = ws_sold_date_sk#87 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 82 Hosting Expression = ws_sold_date_sk#87 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt index f800a80a4e636..086c36864ebdb 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 @@ -94,77 +94,75 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter BroadcastExchange #6 WholeStageCodegen (10) - HashAggregate [brand_id,class_id,category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] HashAggregate [brand_id,class_id,category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - 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 (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (1) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) + InputAdapter + Exchange [brand_id,class_id,category_id] #7 + WholeStageCodegen (6) + HashAggregate [brand_id,class_id,category_id] Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] + 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.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + BroadcastExchange #9 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (3) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (1) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [d_date_sk] #8 InputAdapter ReusedExchange [d_date_sk] #8 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + InputAdapter + ReusedExchange [d_date_sk] #8 InputAdapter BroadcastExchange #13 WholeStageCodegen (23)