diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index ad4769ff8e31a..2d6de2a11c6b8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -234,6 +234,10 @@ abstract class Optimizer(catalogManager: CatalogManager) // this batch. Batch("Early Filter and Projection Push-Down", Once, earlyScanPushDownRules: _*), Batch("Update CTE Relation Stats", Once, UpdateCTERelationStats), + // Must run after "Early Filter and Projection Push-Down" because it relies on + // accurate stats (e.g., DSv2 relations only report stats after V2ScanRelationPushDown). + Batch("Push Down Join Through Union", Once, + PushDownJoinThroughUnion), // Since join costs in AQP can change between multiple runs, there is no reason that we have an // idempotence enforcement on this batch. We thus make it FixedPoint(1) instead of Once. Batch("Join Reorder", FixedPoint(1), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushDownJoinThroughUnion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushDownJoinThroughUnion.scala new file mode 100644 index 0000000000000..f11184f74b83d --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushDownJoinThroughUnion.scala @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import org.apache.spark.SparkException +import org.apache.spark.sql.catalyst.analysis.DeduplicateRelations +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.TreePattern.{JOIN, UNION} + +/** + * Pushes down `Join` through `Union` when the right side of the join is small enough + * to broadcast. + * + * This rule transforms the pattern: + * {{{ + * Join(Union(c1, c2, ..., cN), right, joinType, cond) + * }}} + * into: + * {{{ + * Union(Join(c1, right, joinType, cond1), Join(c2, right, joinType, cond2), ...) + * }}} + * + * where each `condK` has the Union output attributes rewritten to the corresponding child's + * output attributes. + * + * This is beneficial when the right side is small enough to broadcast, because it avoids + * shuffling the (potentially very large) Union result before the Join. Instead, each Union + * branch joins independently with the broadcasted right side. + * + * Applicable join types: Inner, LeftOuter. + */ +object PushDownJoinThroughUnion + extends Rule[LogicalPlan] + with JoinSelectionHelper { + + def apply(plan: LogicalPlan): LogicalPlan = plan.transformUpWithPruning( + _.containsAllPatterns(JOIN, UNION), ruleId) { + + case join @ Join(u: Union, right, joinType, joinCond, hint) + if (joinType == Inner || joinType == LeftOuter) && + canPlanAsBroadcastHashJoin(join, conf) && + // Exclude right subtrees containing subqueries, as DeduplicateRelations + // may not correctly handle correlated references when cloning. + !right.exists(_.expressions.exists(SubqueryExpression.hasSubquery)) => + + val unionHeadOutput = u.children.head.output + val newChildren = u.children.zipWithIndex.map { case (child, idx) => + val newRight = if (idx == 0) right else dedupRight(right) + val leftRewrites = AttributeMap(unionHeadOutput.zip(child.output)) + val rightRewrites = if (idx == 0) { + AttributeMap.empty[Attribute] + } else { + AttributeMap(right.output.zip(newRight.output)) + } + val newCond = joinCond.map(_.transform { + case a: Attribute if leftRewrites.contains(a) => leftRewrites(a) + case a: Attribute if rightRewrites.contains(a) => rightRewrites(a) + }) + Join(child, newRight, joinType, newCond, hint) + } + u.withNewChildren(newChildren) + } + + /** + * Creates a copy of `plan` with fresh ExprIds on all output attributes, + * using the same "fake self-join + DeduplicateRelations" pattern as InlineCTE. + */ + private def dedupRight(plan: LogicalPlan): LogicalPlan = { + DeduplicateRelations( + Join(plan, plan, Inner, None, JoinHint.NONE) + ) match { + case Join(_, deduped, _, _, _) => deduped + case other => + throw SparkException.internalError( + s"Unexpected plan shape after DeduplicateRelations: ${other.getClass.getName}") + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala index 1e718c02f5ea5..4ed918328a16b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala @@ -157,6 +157,7 @@ object RuleIdCollection { "org.apache.spark.sql.catalyst.optimizer.Optimizer$OptimizeSubqueries" :: "org.apache.spark.sql.catalyst.optimizer.PropagateEmptyRelation" :: "org.apache.spark.sql.catalyst.optimizer.PruneFilters" :: + "org.apache.spark.sql.catalyst.optimizer.PushDownJoinThroughUnion" :: "org.apache.spark.sql.catalyst.optimizer.PushDownLeftSemiAntiJoin" :: "org.apache.spark.sql.catalyst.optimizer.PushExtraPredicateThroughJoin" :: "org.apache.spark.sql.catalyst.optimizer.PushFoldableIntoBranches" :: diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PushDownJoinThroughUnionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PushDownJoinThroughUnionSuite.scala new file mode 100644 index 0000000000000..a24c2bc46dbd7 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PushDownJoinThroughUnionSuite.scala @@ -0,0 +1,259 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.expressions.Explode +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, Union} +import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.IntegerType + +class PushDownJoinThroughUnionSuite extends PlanTest { + + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = Batch("PushDownJoinThroughUnion", FixedPoint(10), + PushDownJoinThroughUnion) :: Nil + } + + val testRelation1 = LocalRelation($"a".int, $"b".int) + val testRelation2 = LocalRelation($"c".int, $"d".int) + val testRelation3 = LocalRelation($"e".int, $"f".int) + val testRelation4 = LocalRelation($"g".int, $"h".int) + + test("Push down Inner Join through Union when right side is small") { + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1000") { + val union = Union(testRelation1, testRelation2) + val query = union.join(testRelation3, Inner, Some($"a" === $"e")) + val optimized = Optimize.execute(query.analyze) + + val expected = Union( + testRelation1.join(testRelation3, Inner, Some($"a" === $"e")), + testRelation2.join(testRelation3, Inner, Some($"c" === $"e")) + ).analyze + + comparePlans(optimized, expected) + } + } + + test("Push down Left Outer Join through Union when right side is small") { + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1000") { + val union = Union(testRelation1, testRelation2) + val query = union.join(testRelation3, LeftOuter, Some($"a" === $"e")) + val optimized = Optimize.execute(query.analyze) + + val expected = Union( + testRelation1.join(testRelation3, LeftOuter, Some($"a" === $"e")), + testRelation2.join(testRelation3, LeftOuter, Some($"c" === $"e")) + ).analyze + + comparePlans(optimized, expected) + } + } + + test("Do not push down when right side is too large (broadcast disabled)") { + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val union = Union(testRelation1, testRelation2) + val query = union.join(testRelation3, Inner, Some($"a" === $"e")) + val optimized = Optimize.execute(query.analyze) + + comparePlans(optimized, query.analyze) + } + } + + test("Correctly rewrite attributes in join condition") { + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1000") { + val union = Union(testRelation1, testRelation2) + val query = union.join(testRelation3, Inner, Some($"a" === $"e" && $"b" > 10)) + val optimized = Optimize.execute(query.analyze) + + val expected = Union( + testRelation1.join(testRelation3, Inner, Some($"a" === $"e" && $"b" > 10)), + testRelation2.join(testRelation3, Inner, Some($"c" === $"e" && $"d" > 10)) + ).analyze + + comparePlans(optimized, expected) + } + } + + test("Push down Inner Join through 3-way Union (TPC-DS pattern)") { + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1000") { + val union = Union(Seq(testRelation1, testRelation2, testRelation4)) + val query = union.join(testRelation3, Inner, Some($"a" === $"e")) + val optimized = Optimize.execute(query.analyze) + + val expected = Union(Seq( + testRelation1.join(testRelation3, Inner, Some($"a" === $"e")), + testRelation2.join(testRelation3, Inner, Some($"c" === $"e")), + testRelation4.join(testRelation3, Inner, Some($"g" === $"e")) + )).analyze + + comparePlans(optimized, expected) + } + } + + test("Do not push down unsupported join types") { + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1000") { + val union = Union(testRelation1, testRelation2) + Seq(RightOuter, FullOuter, LeftSemi, LeftAnti).foreach { joinType => + val query = union.join(testRelation3, joinType, Some($"a" === $"e")) + val optimized = Optimize.execute(query.analyze) + comparePlans(optimized, query.analyze) + } + } + } + + test("Do not push down Cross Join (no join condition)") { + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1000") { + val union = Union(testRelation1, testRelation2) + val query = union.join(testRelation3, Inner, None) + val optimized = Optimize.execute(query.analyze) + + comparePlans(optimized, query.analyze) + } + } + + test("Do not push down when Union is on the right side") { + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1000") { + val union = Union(testRelation1, testRelation2) + val query = testRelation3.join(union, Inner, Some($"e" === $"a")) + val optimized = Optimize.execute(query.analyze) + + comparePlans(optimized, query.analyze) + } + } + + test("Push down when right side is a complex subplan") { + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1000") { + val complexRight = testRelation3 + .where($"f" > 0) + .select($"e", ($"f" + 1).as("f_plus_1")) + val union = Union(testRelation1, testRelation2) + val query = union.join(complexRight, Inner, Some($"a" === $"e")) + val optimized = Optimize.execute(query.analyze) + + // Verify the optimization was applied (Union should be the root) + assert(optimized.isInstanceOf[Union]) + // Verify no duplicate ExprIds across Union children's top-level output. + // Each branch should have independent ExprIds for the right side. + val childOutputs = optimized.asInstanceOf[Union].children.map(_.output) + for (i <- childOutputs.indices; j <- (i + 1) until childOutputs.length) { + val ids_i = childOutputs(i).map(_.exprId).toSet + val ids_j = childOutputs(j).map(_.exprId).toSet + assert(ids_i.intersect(ids_j).isEmpty, + s"Union children $i and $j share ExprIds: ${ids_i.intersect(ids_j)}") + } + } + } + + test("Push down when right side contains Generate (Explode)") { + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1000") { + val arrayRelation = LocalRelation($"k".int, $"arr".array(IntegerType)) + val rightWithGenerate = arrayRelation + .generate(Explode($"arr"), outputNames = Seq("exploded_val")) + .select($"k", $"exploded_val") + val union = Union(testRelation1, testRelation2) + val query = union.join(rightWithGenerate, Inner, Some($"a" === $"k")) + val optimized = Optimize.execute(query.analyze) + + // Verify the optimization was applied + assert(optimized.isInstanceOf[Union]) + // Verify no duplicate ExprIds across Union children's output + val childOutputs = optimized.asInstanceOf[Union].children.map(_.output) + for (i <- childOutputs.indices; j <- (i + 1) until childOutputs.length) { + val ids_i = childOutputs(i).map(_.exprId).toSet + val ids_j = childOutputs(j).map(_.exprId).toSet + assert(ids_i.intersect(ids_j).isEmpty, + s"Union children $i and $j share ExprIds: ${ids_i.intersect(ids_j)}") + } + } + } + + test("Push down when right side contains SubqueryAlias") { + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1000") { + val rightWithAlias = testRelation3.subquery("dim") + val union = Union(testRelation1, testRelation2) + val query = union.join(rightWithAlias, Inner, Some($"a" === $"e")) + val optimized = Optimize.execute(query.analyze) + + // Verify the optimization was applied + assert(optimized.isInstanceOf[Union]) + // Verify no duplicate ExprIds across Union children's output + val childOutputs = optimized.asInstanceOf[Union].children.map(_.output) + for (i <- childOutputs.indices; j <- (i + 1) until childOutputs.length) { + val ids_i = childOutputs(i).map(_.exprId).toSet + val ids_j = childOutputs(j).map(_.exprId).toSet + assert(ids_i.intersect(ids_j).isEmpty, + s"Union children $i and $j share ExprIds: ${ids_i.intersect(ids_j)}") + } + } + } + + test("Push down when right side contains Project with Alias") { + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1000") { + val rightWithAlias = testRelation3 + .select($"e", ($"f" + 1).as("f_plus_1")) + val union = Union(testRelation1, testRelation2) + val query = union.join(rightWithAlias, Inner, Some($"a" === $"e")) + val optimized = Optimize.execute(query.analyze) + + assert(optimized.isInstanceOf[Union]) + val childOutputs = optimized.asInstanceOf[Union].children.map(_.output) + for (i <- childOutputs.indices; j <- (i + 1) until childOutputs.length) { + val ids_i = childOutputs(i).map(_.exprId).toSet + val ids_j = childOutputs(j).map(_.exprId).toSet + assert(ids_i.intersect(ids_j).isEmpty, + s"Union children $i and $j share ExprIds: ${ids_i.intersect(ids_j)}") + } + } + } + + test("Push down when right side contains Aggregate") { + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1000") { + val rightWithAgg = testRelation3 + .groupBy($"e")(count($"f").as("cnt"), $"e") + val union = Union(testRelation1, testRelation2) + val query = union.join(rightWithAgg, Inner, Some($"a" === $"e")) + val optimized = Optimize.execute(query.analyze) + + assert(optimized.isInstanceOf[Union]) + val childOutputs = optimized.asInstanceOf[Union].children.map(_.output) + for (i <- childOutputs.indices; j <- (i + 1) until childOutputs.length) { + val ids_i = childOutputs(i).map(_.exprId).toSet + val ids_j = childOutputs(j).map(_.exprId).toSet + assert(ids_i.intersect(ids_j).isEmpty, + s"Union children $i and $j share ExprIds: ${ids_i.intersect(ids_j)}") + } + } + } +} diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/explain.txt index 15064b764679d..b0f272919e6c9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/explain.txt @@ -1,55 +1,61 @@ == Physical Plan == -* Sort (51) -+- Exchange (50) - +- * Project (49) - +- * BroadcastHashJoin Inner BuildRight (48) - :- * Project (23) - : +- * BroadcastHashJoin Inner BuildRight (22) - : :- * HashAggregate (16) - : : +- Exchange (15) - : : +- * HashAggregate (14) - : : +- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- Union (7) - : : : :- * Project (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : : +- * Project (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet spark_catalog.default.catalog_sales (4) - : : +- BroadcastExchange (11) - : : +- * Filter (10) - : : +- * ColumnarToRow (9) - : : +- Scan parquet spark_catalog.default.date_dim (8) - : +- BroadcastExchange (21) - : +- * Project (20) - : +- * Filter (19) - : +- * ColumnarToRow (18) - : +- Scan parquet spark_catalog.default.date_dim (17) - +- BroadcastExchange (47) - +- * Project (46) - +- * BroadcastHashJoin Inner BuildRight (45) - :- * HashAggregate (39) - : +- Exchange (38) - : +- * HashAggregate (37) - : +- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- Union (30) - : : :- * Project (26) - : : : +- * ColumnarToRow (25) - : : : +- Scan parquet spark_catalog.default.web_sales (24) - : : +- * Project (29) - : : +- * ColumnarToRow (28) - : : +- Scan parquet spark_catalog.default.catalog_sales (27) - : +- BroadcastExchange (34) - : +- * Filter (33) - : +- * ColumnarToRow (32) - : +- Scan parquet spark_catalog.default.date_dim (31) - +- BroadcastExchange (44) - +- * Project (43) - +- * Filter (42) - +- * ColumnarToRow (41) - +- Scan parquet spark_catalog.default.date_dim (40) +* Sort (57) ++- Exchange (56) + +- * Project (55) + +- * BroadcastHashJoin Inner BuildRight (54) + :- * Project (26) + : +- * BroadcastHashJoin Inner BuildRight (25) + : :- * HashAggregate (19) + : : +- Exchange (18) + : : +- * HashAggregate (17) + : : +- Union (16) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * Project (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet spark_catalog.default.web_sales (1) + : : : +- BroadcastExchange (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet spark_catalog.default.date_dim (4) + : : +- * Project (15) + : : +- * BroadcastHashJoin Inner BuildRight (14) + : : :- * Project (12) + : : : +- * ColumnarToRow (11) + : : : +- Scan parquet spark_catalog.default.catalog_sales (10) + : : +- ReusedExchange (13) + : +- BroadcastExchange (24) + : +- * Project (23) + : +- * Filter (22) + : +- * ColumnarToRow (21) + : +- Scan parquet spark_catalog.default.date_dim (20) + +- BroadcastExchange (53) + +- * Project (52) + +- * BroadcastHashJoin Inner BuildRight (51) + :- * HashAggregate (45) + : +- Exchange (44) + : +- * HashAggregate (43) + : +- Union (42) + : :- * Project (35) + : : +- * BroadcastHashJoin Inner BuildRight (34) + : : :- * Project (29) + : : : +- * ColumnarToRow (28) + : : : +- Scan parquet spark_catalog.default.web_sales (27) + : : +- BroadcastExchange (33) + : : +- * Filter (32) + : : +- * ColumnarToRow (31) + : : +- Scan parquet spark_catalog.default.date_dim (30) + : +- * Project (41) + : +- * BroadcastHashJoin Inner BuildRight (40) + : :- * Project (38) + : : +- * ColumnarToRow (37) + : : +- Scan parquet spark_catalog.default.catalog_sales (36) + : +- ReusedExchange (39) + +- BroadcastExchange (50) + +- * Project (49) + +- * Filter (48) + +- * ColumnarToRow (47) + +- Scan parquet spark_catalog.default.date_dim (46) (1) Scan parquet spark_catalog.default.web_sales @@ -59,329 +65,355 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#2)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 1] +(2) ColumnarToRow [codegen id : 2] Input [2]: [ws_ext_sales_price#1, ws_sold_date_sk#2] -(3) Project [codegen id : 1] +(3) Project [codegen id : 2] Output [2]: [ws_sold_date_sk#2 AS sold_date_sk#3, ws_ext_sales_price#1 AS sales_price#4] Input [2]: [ws_ext_sales_price#1, ws_sold_date_sk#2] -(4) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ext_sales_price#5, cs_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#6)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 2] -Input [2]: [cs_ext_sales_price#5, cs_sold_date_sk#6] - -(6) Project [codegen id : 2] -Output [2]: [cs_sold_date_sk#6 AS sold_date_sk#7, cs_ext_sales_price#5 AS sales_price#8] -Input [2]: [cs_ext_sales_price#5, cs_sold_date_sk#6] - -(7) Union - -(8) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] +(4) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#5, d_week_seq#6, d_day_name#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_week_seq)] ReadSchema: struct -(9) ColumnarToRow [codegen id : 3] -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] +(5) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#5, d_week_seq#6, d_day_name#7] -(10) Filter [codegen id : 3] -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] -Condition : ((isnotnull(d_date_sk#9) AND isnotnull(d_week_seq#10)) AND might_contain(Subquery scalar-subquery#12, [id=#1], xxhash64(d_week_seq#10, 42))) +(6) Filter [codegen id : 1] +Input [3]: [d_date_sk#5, d_week_seq#6, d_day_name#7] +Condition : ((isnotnull(d_date_sk#5) AND isnotnull(d_week_seq#6)) AND might_contain(Subquery scalar-subquery#8, [id=#1], xxhash64(d_week_seq#6, 42))) -(11) BroadcastExchange -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] +(7) BroadcastExchange +Input [3]: [d_date_sk#5, d_week_seq#6, d_day_name#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(12) BroadcastHashJoin [codegen id : 4] +(8) BroadcastHashJoin [codegen id : 2] Left keys [1]: [sold_date_sk#3] -Right keys [1]: [d_date_sk#9] +Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(13) Project [codegen id : 4] -Output [3]: [sales_price#4, d_week_seq#10, d_day_name#11] -Input [5]: [sold_date_sk#3, sales_price#4, d_date_sk#9, d_week_seq#10, d_day_name#11] - -(14) HashAggregate [codegen id : 4] -Input [3]: [sales_price#4, d_week_seq#10, d_day_name#11] -Keys [1]: [d_week_seq#10] -Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))] -Aggregate Attributes [7]: [sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Results [8]: [d_week_seq#10, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25, sum#26] - -(15) Exchange -Input [8]: [d_week_seq#10, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25, sum#26] -Arguments: hashpartitioning(d_week_seq#10, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(16) HashAggregate [codegen id : 12] -Input [8]: [d_week_seq#10, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25, sum#26] -Keys [1]: [d_week_seq#10] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END))#27, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END))#29, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END))#30, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END))#31, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END))#32, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))#33] -Results [8]: [d_week_seq#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END))#27,17,2) AS sun_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END))#28,17,2) AS mon_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END))#29,17,2) AS tue_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END))#30,17,2) AS wed_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END))#31,17,2) AS thu_sales#38, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END))#32,17,2) AS fri_sales#39, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))#33,17,2) AS sat_sales#40] - -(17) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#41, d_year#42] +(9) Project [codegen id : 2] +Output [3]: [sales_price#4, d_week_seq#6, d_day_name#7] +Input [5]: [sold_date_sk#3, sales_price#4, d_date_sk#5, d_week_seq#6, d_day_name#7] + +(10) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_ext_sales_price#9, cs_sold_date_sk#10] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#10)] +ReadSchema: struct + +(11) ColumnarToRow [codegen id : 4] +Input [2]: [cs_ext_sales_price#9, cs_sold_date_sk#10] + +(12) Project [codegen id : 4] +Output [2]: [cs_sold_date_sk#10 AS sold_date_sk#11, cs_ext_sales_price#9 AS sales_price#12] +Input [2]: [cs_ext_sales_price#9, cs_sold_date_sk#10] + +(13) ReusedExchange [Reuses operator id: 7] +Output [3]: [d_date_sk#13, d_week_seq#14, d_day_name#15] + +(14) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [sold_date_sk#11] +Right keys [1]: [d_date_sk#13] +Join type: Inner +Join condition: None + +(15) Project [codegen id : 4] +Output [3]: [sales_price#12, d_week_seq#14, d_day_name#15] +Input [5]: [sold_date_sk#11, sales_price#12, d_date_sk#13, d_week_seq#14, d_day_name#15] + +(16) Union + +(17) HashAggregate [codegen id : 5] +Input [3]: [sales_price#4, d_week_seq#6, d_day_name#7] +Keys [1]: [d_week_seq#6] +Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN sales_price#4 END))] +Aggregate Attributes [7]: [sum#16, sum#17, sum#18, sum#19, sum#20, sum#21, sum#22] +Results [8]: [d_week_seq#6, sum#23, sum#24, sum#25, sum#26, sum#27, sum#28, sum#29] + +(18) Exchange +Input [8]: [d_week_seq#6, sum#23, sum#24, sum#25, sum#26, sum#27, sum#28, sum#29] +Arguments: hashpartitioning(d_week_seq#6, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(19) HashAggregate [codegen id : 14] +Input [8]: [d_week_seq#6, sum#23, sum#24, sum#25, sum#26, sum#27, sum#28, sum#29] +Keys [1]: [d_week_seq#6] +Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN sales_price#4 END))] +Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN sales_price#4 END))#30, sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN sales_price#4 END))#31, sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN sales_price#4 END))#32, sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN sales_price#4 END))#33, sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN sales_price#4 END))#34, sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN sales_price#4 END))#35, sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN sales_price#4 END))#36] +Results [8]: [d_week_seq#6, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN sales_price#4 END))#30,17,2) AS sun_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN sales_price#4 END))#31,17,2) AS mon_sales#38, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN sales_price#4 END))#32,17,2) AS tue_sales#39, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN sales_price#4 END))#33,17,2) AS wed_sales#40, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN sales_price#4 END))#34,17,2) AS thu_sales#41, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN sales_price#4 END))#35,17,2) AS fri_sales#42, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN sales_price#4 END))#36,17,2) AS sat_sales#43] + +(20) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_week_seq#44, d_year#45] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_week_seq)] ReadSchema: struct -(18) ColumnarToRow [codegen id : 5] -Input [2]: [d_week_seq#41, d_year#42] +(21) ColumnarToRow [codegen id : 6] +Input [2]: [d_week_seq#44, d_year#45] -(19) Filter [codegen id : 5] -Input [2]: [d_week_seq#41, d_year#42] -Condition : ((isnotnull(d_year#42) AND (d_year#42 = 2001)) AND isnotnull(d_week_seq#41)) +(22) Filter [codegen id : 6] +Input [2]: [d_week_seq#44, d_year#45] +Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2001)) AND isnotnull(d_week_seq#44)) -(20) Project [codegen id : 5] -Output [1]: [d_week_seq#41] -Input [2]: [d_week_seq#41, d_year#42] +(23) Project [codegen id : 6] +Output [1]: [d_week_seq#44] +Input [2]: [d_week_seq#44, d_year#45] -(21) BroadcastExchange -Input [1]: [d_week_seq#41] +(24) BroadcastExchange +Input [1]: [d_week_seq#44] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(22) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [d_week_seq#10] -Right keys [1]: [d_week_seq#41] +(25) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [d_week_seq#6] +Right keys [1]: [d_week_seq#44] Join type: Inner Join condition: None -(23) Project [codegen id : 12] -Output [8]: [d_week_seq#10 AS d_week_seq1#43, sun_sales#34 AS sun_sales1#44, mon_sales#35 AS mon_sales1#45, tue_sales#36 AS tue_sales1#46, wed_sales#37 AS wed_sales1#47, thu_sales#38 AS thu_sales1#48, fri_sales#39 AS fri_sales1#49, sat_sales#40 AS sat_sales1#50] -Input [9]: [d_week_seq#10, sun_sales#34, mon_sales#35, tue_sales#36, wed_sales#37, thu_sales#38, fri_sales#39, sat_sales#40, d_week_seq#41] +(26) Project [codegen id : 14] +Output [8]: [d_week_seq#6 AS d_week_seq1#46, sun_sales#37 AS sun_sales1#47, mon_sales#38 AS mon_sales1#48, tue_sales#39 AS tue_sales1#49, wed_sales#40 AS wed_sales1#50, thu_sales#41 AS thu_sales1#51, fri_sales#42 AS fri_sales1#52, sat_sales#43 AS sat_sales1#53] +Input [9]: [d_week_seq#6, sun_sales#37, mon_sales#38, tue_sales#39, wed_sales#40, thu_sales#41, fri_sales#42, sat_sales#43, d_week_seq#44] -(24) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_ext_sales_price#51, ws_sold_date_sk#52] +(27) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_ext_sales_price#54, ws_sold_date_sk#55] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#52)] +PartitionFilters: [isnotnull(ws_sold_date_sk#55)] ReadSchema: struct -(25) ColumnarToRow [codegen id : 6] -Input [2]: [ws_ext_sales_price#51, ws_sold_date_sk#52] +(28) ColumnarToRow [codegen id : 8] +Input [2]: [ws_ext_sales_price#54, ws_sold_date_sk#55] -(26) Project [codegen id : 6] -Output [2]: [ws_sold_date_sk#52 AS sold_date_sk#53, ws_ext_sales_price#51 AS sales_price#54] -Input [2]: [ws_ext_sales_price#51, ws_sold_date_sk#52] +(29) Project [codegen id : 8] +Output [2]: [ws_sold_date_sk#55 AS sold_date_sk#56, ws_ext_sales_price#54 AS sales_price#57] +Input [2]: [ws_ext_sales_price#54, ws_sold_date_sk#55] -(27) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ext_sales_price#55, cs_sold_date_sk#56] +(30) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#58, d_week_seq#59, d_day_name#60] Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#56)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_week_seq)] +ReadSchema: struct + +(31) ColumnarToRow [codegen id : 7] +Input [3]: [d_date_sk#58, d_week_seq#59, d_day_name#60] -(28) ColumnarToRow [codegen id : 7] -Input [2]: [cs_ext_sales_price#55, cs_sold_date_sk#56] +(32) Filter [codegen id : 7] +Input [3]: [d_date_sk#58, d_week_seq#59, d_day_name#60] +Condition : ((isnotnull(d_date_sk#58) AND isnotnull(d_week_seq#59)) AND might_contain(Subquery scalar-subquery#61, [id=#5], xxhash64(d_week_seq#59, 42))) -(29) Project [codegen id : 7] -Output [2]: [cs_sold_date_sk#56 AS sold_date_sk#57, cs_ext_sales_price#55 AS sales_price#58] -Input [2]: [cs_ext_sales_price#55, cs_sold_date_sk#56] +(33) BroadcastExchange +Input [3]: [d_date_sk#58, d_week_seq#59, d_day_name#60] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] + +(34) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [sold_date_sk#56] +Right keys [1]: [d_date_sk#58] +Join type: Inner +Join condition: None -(30) Union +(35) Project [codegen id : 8] +Output [3]: [sales_price#57, d_week_seq#59, d_day_name#60] +Input [5]: [sold_date_sk#56, sales_price#57, d_date_sk#58, d_week_seq#59, d_day_name#60] -(31) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#59, d_week_seq#60, d_day_name#61] +(36) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_ext_sales_price#62, cs_sold_date_sk#63] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_week_seq)] -ReadSchema: struct +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#63)] +ReadSchema: struct -(32) ColumnarToRow [codegen id : 8] -Input [3]: [d_date_sk#59, d_week_seq#60, d_day_name#61] +(37) ColumnarToRow [codegen id : 10] +Input [2]: [cs_ext_sales_price#62, cs_sold_date_sk#63] -(33) Filter [codegen id : 8] -Input [3]: [d_date_sk#59, d_week_seq#60, d_day_name#61] -Condition : ((isnotnull(d_date_sk#59) AND isnotnull(d_week_seq#60)) AND might_contain(Subquery scalar-subquery#62, [id=#5], xxhash64(d_week_seq#60, 42))) +(38) Project [codegen id : 10] +Output [2]: [cs_sold_date_sk#63 AS sold_date_sk#64, cs_ext_sales_price#62 AS sales_price#65] +Input [2]: [cs_ext_sales_price#62, cs_sold_date_sk#63] -(34) BroadcastExchange -Input [3]: [d_date_sk#59, d_week_seq#60, d_day_name#61] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +(39) ReusedExchange [Reuses operator id: 33] +Output [3]: [d_date_sk#66, d_week_seq#67, d_day_name#68] -(35) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [sold_date_sk#53] -Right keys [1]: [d_date_sk#59] +(40) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [sold_date_sk#64] +Right keys [1]: [d_date_sk#66] Join type: Inner Join condition: None -(36) Project [codegen id : 9] -Output [3]: [sales_price#54, d_week_seq#60, d_day_name#61] -Input [5]: [sold_date_sk#53, sales_price#54, d_date_sk#59, d_week_seq#60, d_day_name#61] - -(37) HashAggregate [codegen id : 9] -Input [3]: [sales_price#54, d_week_seq#60, d_day_name#61] -Keys [1]: [d_week_seq#60] -Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#61 = Sunday ) THEN sales_price#54 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#61 = Monday ) THEN sales_price#54 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#61 = Tuesday ) THEN sales_price#54 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#61 = Wednesday) THEN sales_price#54 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#61 = Thursday ) THEN sales_price#54 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#61 = Friday ) THEN sales_price#54 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#61 = Saturday ) THEN sales_price#54 END))] -Aggregate Attributes [7]: [sum#63, sum#64, sum#65, sum#66, sum#67, sum#68, sum#69] -Results [8]: [d_week_seq#60, sum#70, sum#71, sum#72, sum#73, sum#74, sum#75, sum#76] - -(38) Exchange -Input [8]: [d_week_seq#60, sum#70, sum#71, sum#72, sum#73, sum#74, sum#75, sum#76] -Arguments: hashpartitioning(d_week_seq#60, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(39) HashAggregate [codegen id : 11] -Input [8]: [d_week_seq#60, sum#70, sum#71, sum#72, sum#73, sum#74, sum#75, sum#76] -Keys [1]: [d_week_seq#60] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#61 = Sunday ) THEN sales_price#54 END)), sum(UnscaledValue(CASE WHEN (d_day_name#61 = Monday ) THEN sales_price#54 END)), sum(UnscaledValue(CASE WHEN (d_day_name#61 = Tuesday ) THEN sales_price#54 END)), sum(UnscaledValue(CASE WHEN (d_day_name#61 = Wednesday) THEN sales_price#54 END)), sum(UnscaledValue(CASE WHEN (d_day_name#61 = Thursday ) THEN sales_price#54 END)), sum(UnscaledValue(CASE WHEN (d_day_name#61 = Friday ) THEN sales_price#54 END)), sum(UnscaledValue(CASE WHEN (d_day_name#61 = Saturday ) THEN sales_price#54 END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#61 = Sunday ) THEN sales_price#54 END))#27, sum(UnscaledValue(CASE WHEN (d_day_name#61 = Monday ) THEN sales_price#54 END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#61 = Tuesday ) THEN sales_price#54 END))#29, sum(UnscaledValue(CASE WHEN (d_day_name#61 = Wednesday) THEN sales_price#54 END))#30, sum(UnscaledValue(CASE WHEN (d_day_name#61 = Thursday ) THEN sales_price#54 END))#31, sum(UnscaledValue(CASE WHEN (d_day_name#61 = Friday ) THEN sales_price#54 END))#32, sum(UnscaledValue(CASE WHEN (d_day_name#61 = Saturday ) THEN sales_price#54 END))#33] -Results [8]: [d_week_seq#60, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#61 = Sunday ) THEN sales_price#54 END))#27,17,2) AS sun_sales#77, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#61 = Monday ) THEN sales_price#54 END))#28,17,2) AS mon_sales#78, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#61 = Tuesday ) THEN sales_price#54 END))#29,17,2) AS tue_sales#79, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#61 = Wednesday) THEN sales_price#54 END))#30,17,2) AS wed_sales#80, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#61 = Thursday ) THEN sales_price#54 END))#31,17,2) AS thu_sales#81, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#61 = Friday ) THEN sales_price#54 END))#32,17,2) AS fri_sales#82, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#61 = Saturday ) THEN sales_price#54 END))#33,17,2) AS sat_sales#83] - -(40) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#84, d_year#85] +(41) Project [codegen id : 10] +Output [3]: [sales_price#65, d_week_seq#67, d_day_name#68] +Input [5]: [sold_date_sk#64, sales_price#65, d_date_sk#66, d_week_seq#67, d_day_name#68] + +(42) Union + +(43) HashAggregate [codegen id : 11] +Input [3]: [sales_price#57, d_week_seq#59, d_day_name#60] +Keys [1]: [d_week_seq#59] +Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#60 = Sunday ) THEN sales_price#57 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#60 = Monday ) THEN sales_price#57 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#60 = Tuesday ) THEN sales_price#57 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#60 = Wednesday) THEN sales_price#57 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#60 = Thursday ) THEN sales_price#57 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#60 = Friday ) THEN sales_price#57 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#60 = Saturday ) THEN sales_price#57 END))] +Aggregate Attributes [7]: [sum#69, sum#70, sum#71, sum#72, sum#73, sum#74, sum#75] +Results [8]: [d_week_seq#59, sum#76, sum#77, sum#78, sum#79, sum#80, sum#81, sum#82] + +(44) Exchange +Input [8]: [d_week_seq#59, sum#76, sum#77, sum#78, sum#79, sum#80, sum#81, sum#82] +Arguments: hashpartitioning(d_week_seq#59, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(45) HashAggregate [codegen id : 13] +Input [8]: [d_week_seq#59, sum#76, sum#77, sum#78, sum#79, sum#80, sum#81, sum#82] +Keys [1]: [d_week_seq#59] +Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#60 = Sunday ) THEN sales_price#57 END)), sum(UnscaledValue(CASE WHEN (d_day_name#60 = Monday ) THEN sales_price#57 END)), sum(UnscaledValue(CASE WHEN (d_day_name#60 = Tuesday ) THEN sales_price#57 END)), sum(UnscaledValue(CASE WHEN (d_day_name#60 = Wednesday) THEN sales_price#57 END)), sum(UnscaledValue(CASE WHEN (d_day_name#60 = Thursday ) THEN sales_price#57 END)), sum(UnscaledValue(CASE WHEN (d_day_name#60 = Friday ) THEN sales_price#57 END)), sum(UnscaledValue(CASE WHEN (d_day_name#60 = Saturday ) THEN sales_price#57 END))] +Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#60 = Sunday ) THEN sales_price#57 END))#30, sum(UnscaledValue(CASE WHEN (d_day_name#60 = Monday ) THEN sales_price#57 END))#31, sum(UnscaledValue(CASE WHEN (d_day_name#60 = Tuesday ) THEN sales_price#57 END))#32, sum(UnscaledValue(CASE WHEN (d_day_name#60 = Wednesday) THEN sales_price#57 END))#33, sum(UnscaledValue(CASE WHEN (d_day_name#60 = Thursday ) THEN sales_price#57 END))#34, sum(UnscaledValue(CASE WHEN (d_day_name#60 = Friday ) THEN sales_price#57 END))#35, sum(UnscaledValue(CASE WHEN (d_day_name#60 = Saturday ) THEN sales_price#57 END))#36] +Results [8]: [d_week_seq#59, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#60 = Sunday ) THEN sales_price#57 END))#30,17,2) AS sun_sales#83, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#60 = Monday ) THEN sales_price#57 END))#31,17,2) AS mon_sales#84, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#60 = Tuesday ) THEN sales_price#57 END))#32,17,2) AS tue_sales#85, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#60 = Wednesday) THEN sales_price#57 END))#33,17,2) AS wed_sales#86, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#60 = Thursday ) THEN sales_price#57 END))#34,17,2) AS thu_sales#87, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#60 = Friday ) THEN sales_price#57 END))#35,17,2) AS fri_sales#88, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#60 = Saturday ) THEN sales_price#57 END))#36,17,2) AS sat_sales#89] + +(46) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_week_seq#90, d_year#91] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_week_seq)] ReadSchema: struct -(41) ColumnarToRow [codegen id : 10] -Input [2]: [d_week_seq#84, d_year#85] +(47) ColumnarToRow [codegen id : 12] +Input [2]: [d_week_seq#90, d_year#91] -(42) Filter [codegen id : 10] -Input [2]: [d_week_seq#84, d_year#85] -Condition : ((isnotnull(d_year#85) AND (d_year#85 = 2002)) AND isnotnull(d_week_seq#84)) +(48) Filter [codegen id : 12] +Input [2]: [d_week_seq#90, d_year#91] +Condition : ((isnotnull(d_year#91) AND (d_year#91 = 2002)) AND isnotnull(d_week_seq#90)) -(43) Project [codegen id : 10] -Output [1]: [d_week_seq#84] -Input [2]: [d_week_seq#84, d_year#85] +(49) Project [codegen id : 12] +Output [1]: [d_week_seq#90] +Input [2]: [d_week_seq#90, d_year#91] -(44) BroadcastExchange -Input [1]: [d_week_seq#84] +(50) BroadcastExchange +Input [1]: [d_week_seq#90] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -(45) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [d_week_seq#60] -Right keys [1]: [d_week_seq#84] +(51) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [d_week_seq#59] +Right keys [1]: [d_week_seq#90] Join type: Inner Join condition: None -(46) Project [codegen id : 11] -Output [8]: [d_week_seq#60 AS d_week_seq2#86, sun_sales#77 AS sun_sales2#87, mon_sales#78 AS mon_sales2#88, tue_sales#79 AS tue_sales2#89, wed_sales#80 AS wed_sales2#90, thu_sales#81 AS thu_sales2#91, fri_sales#82 AS fri_sales2#92, sat_sales#83 AS sat_sales2#93] -Input [9]: [d_week_seq#60, sun_sales#77, mon_sales#78, tue_sales#79, wed_sales#80, thu_sales#81, fri_sales#82, sat_sales#83, d_week_seq#84] +(52) Project [codegen id : 13] +Output [8]: [d_week_seq#59 AS d_week_seq2#92, sun_sales#83 AS sun_sales2#93, mon_sales#84 AS mon_sales2#94, tue_sales#85 AS tue_sales2#95, wed_sales#86 AS wed_sales2#96, thu_sales#87 AS thu_sales2#97, fri_sales#88 AS fri_sales2#98, sat_sales#89 AS sat_sales2#99] +Input [9]: [d_week_seq#59, sun_sales#83, mon_sales#84, tue_sales#85, wed_sales#86, thu_sales#87, fri_sales#88, sat_sales#89, d_week_seq#90] -(47) BroadcastExchange -Input [8]: [d_week_seq2#86, sun_sales2#87, mon_sales2#88, tue_sales2#89, wed_sales2#90, thu_sales2#91, fri_sales2#92, sat_sales2#93] +(53) BroadcastExchange +Input [8]: [d_week_seq2#92, sun_sales2#93, mon_sales2#94, tue_sales2#95, wed_sales2#96, thu_sales2#97, fri_sales2#98, sat_sales2#99] Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [plan_id=9] -(48) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [d_week_seq1#43] -Right keys [1]: [(d_week_seq2#86 - 53)] +(54) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [d_week_seq1#46] +Right keys [1]: [(d_week_seq2#92 - 53)] Join type: Inner Join condition: None -(49) Project [codegen id : 12] -Output [8]: [d_week_seq1#43, round((sun_sales1#44 / sun_sales2#87), 2) AS round((sun_sales1 / sun_sales2), 2)#94, round((mon_sales1#45 / mon_sales2#88), 2) AS round((mon_sales1 / mon_sales2), 2)#95, round((tue_sales1#46 / tue_sales2#89), 2) AS round((tue_sales1 / tue_sales2), 2)#96, round((wed_sales1#47 / wed_sales2#90), 2) AS round((wed_sales1 / wed_sales2), 2)#97, round((thu_sales1#48 / thu_sales2#91), 2) AS round((thu_sales1 / thu_sales2), 2)#98, round((fri_sales1#49 / fri_sales2#92), 2) AS round((fri_sales1 / fri_sales2), 2)#99, round((sat_sales1#50 / sat_sales2#93), 2) AS round((sat_sales1 / sat_sales2), 2)#100] -Input [16]: [d_week_seq1#43, sun_sales1#44, mon_sales1#45, tue_sales1#46, wed_sales1#47, thu_sales1#48, fri_sales1#49, sat_sales1#50, d_week_seq2#86, sun_sales2#87, mon_sales2#88, tue_sales2#89, wed_sales2#90, thu_sales2#91, fri_sales2#92, sat_sales2#93] +(55) Project [codegen id : 14] +Output [8]: [d_week_seq1#46, round((sun_sales1#47 / sun_sales2#93), 2) AS round((sun_sales1 / sun_sales2), 2)#100, round((mon_sales1#48 / mon_sales2#94), 2) AS round((mon_sales1 / mon_sales2), 2)#101, round((tue_sales1#49 / tue_sales2#95), 2) AS round((tue_sales1 / tue_sales2), 2)#102, round((wed_sales1#50 / wed_sales2#96), 2) AS round((wed_sales1 / wed_sales2), 2)#103, round((thu_sales1#51 / thu_sales2#97), 2) AS round((thu_sales1 / thu_sales2), 2)#104, round((fri_sales1#52 / fri_sales2#98), 2) AS round((fri_sales1 / fri_sales2), 2)#105, round((sat_sales1#53 / sat_sales2#99), 2) AS round((sat_sales1 / sat_sales2), 2)#106] +Input [16]: [d_week_seq1#46, sun_sales1#47, mon_sales1#48, tue_sales1#49, wed_sales1#50, thu_sales1#51, fri_sales1#52, sat_sales1#53, d_week_seq2#92, sun_sales2#93, mon_sales2#94, tue_sales2#95, wed_sales2#96, thu_sales2#97, fri_sales2#98, sat_sales2#99] -(50) Exchange -Input [8]: [d_week_seq1#43, round((sun_sales1 / sun_sales2), 2)#94, round((mon_sales1 / mon_sales2), 2)#95, round((tue_sales1 / tue_sales2), 2)#96, round((wed_sales1 / wed_sales2), 2)#97, round((thu_sales1 / thu_sales2), 2)#98, round((fri_sales1 / fri_sales2), 2)#99, round((sat_sales1 / sat_sales2), 2)#100] -Arguments: rangepartitioning(d_week_seq1#43 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=10] +(56) Exchange +Input [8]: [d_week_seq1#46, round((sun_sales1 / sun_sales2), 2)#100, round((mon_sales1 / mon_sales2), 2)#101, round((tue_sales1 / tue_sales2), 2)#102, round((wed_sales1 / wed_sales2), 2)#103, round((thu_sales1 / thu_sales2), 2)#104, round((fri_sales1 / fri_sales2), 2)#105, round((sat_sales1 / sat_sales2), 2)#106] +Arguments: rangepartitioning(d_week_seq1#46 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(51) Sort [codegen id : 13] -Input [8]: [d_week_seq1#43, round((sun_sales1 / sun_sales2), 2)#94, round((mon_sales1 / mon_sales2), 2)#95, round((tue_sales1 / tue_sales2), 2)#96, round((wed_sales1 / wed_sales2), 2)#97, round((thu_sales1 / thu_sales2), 2)#98, round((fri_sales1 / fri_sales2), 2)#99, round((sat_sales1 / sat_sales2), 2)#100] -Arguments: [d_week_seq1#43 ASC NULLS FIRST], true, 0 +(57) Sort [codegen id : 15] +Input [8]: [d_week_seq1#46, round((sun_sales1 / sun_sales2), 2)#100, round((mon_sales1 / mon_sales2), 2)#101, round((tue_sales1 / tue_sales2), 2)#102, round((wed_sales1 / wed_sales2), 2)#103, round((thu_sales1 / thu_sales2), 2)#104, round((fri_sales1 / fri_sales2), 2)#105, round((sat_sales1 / sat_sales2), 2)#106] +Arguments: [d_week_seq1#46 ASC NULLS FIRST], true, 0 ===== Subqueries ===== -Subquery:1 Hosting operator id = 10 Hosting Expression = Subquery scalar-subquery#12, [id=#1] -ObjectHashAggregate (58) -+- Exchange (57) - +- ObjectHashAggregate (56) - +- * Project (55) - +- * Filter (54) - +- * ColumnarToRow (53) - +- Scan parquet spark_catalog.default.date_dim (52) +Subquery:1 Hosting operator id = 6 Hosting Expression = Subquery scalar-subquery#8, [id=#1] +ObjectHashAggregate (64) ++- Exchange (63) + +- ObjectHashAggregate (62) + +- * Project (61) + +- * Filter (60) + +- * ColumnarToRow (59) + +- Scan parquet spark_catalog.default.date_dim (58) -(52) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#41, d_year#42] +(58) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_week_seq#44, d_year#45] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_week_seq)] ReadSchema: struct -(53) ColumnarToRow [codegen id : 1] -Input [2]: [d_week_seq#41, d_year#42] +(59) ColumnarToRow [codegen id : 1] +Input [2]: [d_week_seq#44, d_year#45] -(54) Filter [codegen id : 1] -Input [2]: [d_week_seq#41, d_year#42] -Condition : ((isnotnull(d_year#42) AND (d_year#42 = 2001)) AND isnotnull(d_week_seq#41)) +(60) Filter [codegen id : 1] +Input [2]: [d_week_seq#44, d_year#45] +Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2001)) AND isnotnull(d_week_seq#44)) -(55) Project [codegen id : 1] -Output [1]: [d_week_seq#41] -Input [2]: [d_week_seq#41, d_year#42] +(61) Project [codegen id : 1] +Output [1]: [d_week_seq#44] +Input [2]: [d_week_seq#44, d_year#45] -(56) ObjectHashAggregate -Input [1]: [d_week_seq#41] +(62) ObjectHashAggregate +Input [1]: [d_week_seq#44] Keys: [] -Functions [1]: [partial_bloom_filter_agg(xxhash64(d_week_seq#41, 42), 362, 9656, 0, 0)] -Aggregate Attributes [1]: [buf#101] -Results [1]: [buf#102] +Functions [1]: [partial_bloom_filter_agg(xxhash64(d_week_seq#44, 42), 362, 9656, 0, 0)] +Aggregate Attributes [1]: [buf#107] +Results [1]: [buf#108] -(57) Exchange -Input [1]: [buf#102] +(63) Exchange +Input [1]: [buf#108] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=11] -(58) ObjectHashAggregate -Input [1]: [buf#102] +(64) ObjectHashAggregate +Input [1]: [buf#108] Keys: [] -Functions [1]: [bloom_filter_agg(xxhash64(d_week_seq#41, 42), 362, 9656, 0, 0)] -Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(d_week_seq#41, 42), 362, 9656, 0, 0)#103] -Results [1]: [bloom_filter_agg(xxhash64(d_week_seq#41, 42), 362, 9656, 0, 0)#103 AS bloomFilter#104] - -Subquery:2 Hosting operator id = 33 Hosting Expression = Subquery scalar-subquery#62, [id=#5] -ObjectHashAggregate (65) -+- Exchange (64) - +- ObjectHashAggregate (63) - +- * Project (62) - +- * Filter (61) - +- * ColumnarToRow (60) - +- Scan parquet spark_catalog.default.date_dim (59) - - -(59) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#84, d_year#85] +Functions [1]: [bloom_filter_agg(xxhash64(d_week_seq#44, 42), 362, 9656, 0, 0)] +Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(d_week_seq#44, 42), 362, 9656, 0, 0)#109] +Results [1]: [bloom_filter_agg(xxhash64(d_week_seq#44, 42), 362, 9656, 0, 0)#109 AS bloomFilter#110] + +Subquery:2 Hosting operator id = 32 Hosting Expression = Subquery scalar-subquery#61, [id=#5] +ObjectHashAggregate (71) ++- Exchange (70) + +- ObjectHashAggregate (69) + +- * Project (68) + +- * Filter (67) + +- * ColumnarToRow (66) + +- Scan parquet spark_catalog.default.date_dim (65) + + +(65) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_week_seq#90, d_year#91] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_week_seq)] ReadSchema: struct -(60) ColumnarToRow [codegen id : 1] -Input [2]: [d_week_seq#84, d_year#85] +(66) ColumnarToRow [codegen id : 1] +Input [2]: [d_week_seq#90, d_year#91] -(61) Filter [codegen id : 1] -Input [2]: [d_week_seq#84, d_year#85] -Condition : ((isnotnull(d_year#85) AND (d_year#85 = 2002)) AND isnotnull(d_week_seq#84)) +(67) Filter [codegen id : 1] +Input [2]: [d_week_seq#90, d_year#91] +Condition : ((isnotnull(d_year#91) AND (d_year#91 = 2002)) AND isnotnull(d_week_seq#90)) -(62) Project [codegen id : 1] -Output [1]: [d_week_seq#84] -Input [2]: [d_week_seq#84, d_year#85] +(68) Project [codegen id : 1] +Output [1]: [d_week_seq#90] +Input [2]: [d_week_seq#90, d_year#91] -(63) ObjectHashAggregate -Input [1]: [d_week_seq#84] +(69) ObjectHashAggregate +Input [1]: [d_week_seq#90] Keys: [] -Functions [1]: [partial_bloom_filter_agg(xxhash64(d_week_seq#84, 42), 362, 9656, 0, 0)] -Aggregate Attributes [1]: [buf#105] -Results [1]: [buf#106] +Functions [1]: [partial_bloom_filter_agg(xxhash64(d_week_seq#90, 42), 362, 9656, 0, 0)] +Aggregate Attributes [1]: [buf#111] +Results [1]: [buf#112] -(64) Exchange -Input [1]: [buf#106] +(70) Exchange +Input [1]: [buf#112] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] -(65) ObjectHashAggregate -Input [1]: [buf#106] +(71) ObjectHashAggregate +Input [1]: [buf#112] Keys: [] -Functions [1]: [bloom_filter_agg(xxhash64(d_week_seq#84, 42), 362, 9656, 0, 0)] -Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(d_week_seq#84, 42), 362, 9656, 0, 0)#107] -Results [1]: [bloom_filter_agg(xxhash64(d_week_seq#84, 42), 362, 9656, 0, 0)#107 AS bloomFilter#108] +Functions [1]: [bloom_filter_agg(xxhash64(d_week_seq#90, 42), 362, 9656, 0, 0)] +Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(d_week_seq#90, 42), 362, 9656, 0, 0)#113] +Results [1]: [bloom_filter_agg(xxhash64(d_week_seq#90, 42), 362, 9656, 0, 0)#113 AS bloomFilter#114] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/simplified.txt index 4fb858b42521a..e4a8fb4c2f156 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (13) +WholeStageCodegen (15) Sort [d_week_seq1] InputAdapter Exchange [d_week_seq1] #1 - WholeStageCodegen (12) + WholeStageCodegen (14) Project [d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] BroadcastHashJoin [d_week_seq1,d_week_seq2] Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] @@ -10,42 +10,46 @@ WholeStageCodegen (13) HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] InputAdapter Exchange [d_week_seq] #2 - WholeStageCodegen (4) + WholeStageCodegen (5) HashAggregate [d_week_seq,d_day_name,sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,d_week_seq,d_day_name] - BroadcastHashJoin [sold_date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (1) + InputAdapter + Union + WholeStageCodegen (2) + Project [sales_price,d_week_seq,d_day_name] + BroadcastHashJoin [sold_date_sk,d_date_sk] Project [ws_sold_date_sk,ws_ext_sales_price] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_sales [ws_ext_sales_price,ws_sold_date_sk] - WholeStageCodegen (2) + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Filter [d_date_sk,d_week_seq] + Subquery #1 + ObjectHashAggregate [buf] [bloom_filter_agg(xxhash64(d_week_seq, 42), 362, 9656, 0, 0),bloomFilter,buf] + Exchange #4 + ObjectHashAggregate [d_week_seq] [buf,buf] + WholeStageCodegen (1) + Project [d_week_seq] + Filter [d_year,d_week_seq] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_week_seq,d_year] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] + WholeStageCodegen (4) + Project [sales_price,d_week_seq,d_day_name] + BroadcastHashJoin [sold_date_sk,d_date_sk] Project [cs_sold_date_sk,cs_ext_sales_price] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (3) - Filter [d_date_sk,d_week_seq] - Subquery #1 - ObjectHashAggregate [buf] [bloom_filter_agg(xxhash64(d_week_seq, 42), 362, 9656, 0, 0),bloomFilter,buf] - Exchange #4 - ObjectHashAggregate [d_week_seq] [buf,buf] - WholeStageCodegen (1) - Project [d_week_seq] - Filter [d_year,d_week_seq] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_week_seq,d_year] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] + InputAdapter + ReusedExchange [d_date_sk,d_week_seq,d_day_name] #3 InputAdapter BroadcastExchange #5 - WholeStageCodegen (5) + WholeStageCodegen (6) Project [d_week_seq] Filter [d_year,d_week_seq] ColumnarToRow @@ -53,48 +57,52 @@ WholeStageCodegen (13) Scan parquet spark_catalog.default.date_dim [d_week_seq,d_year] InputAdapter BroadcastExchange #6 - WholeStageCodegen (11) + WholeStageCodegen (13) Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] InputAdapter Exchange [d_week_seq] #7 - WholeStageCodegen (9) + WholeStageCodegen (11) HashAggregate [d_week_seq,d_day_name,sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,d_week_seq,d_day_name] - BroadcastHashJoin [sold_date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (6) + InputAdapter + Union + WholeStageCodegen (8) + Project [sales_price,d_week_seq,d_day_name] + BroadcastHashJoin [sold_date_sk,d_date_sk] Project [ws_sold_date_sk,ws_ext_sales_price] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_sales [ws_ext_sales_price,ws_sold_date_sk] - WholeStageCodegen (7) + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (7) + Filter [d_date_sk,d_week_seq] + Subquery #2 + ObjectHashAggregate [buf] [bloom_filter_agg(xxhash64(d_week_seq, 42), 362, 9656, 0, 0),bloomFilter,buf] + Exchange #9 + ObjectHashAggregate [d_week_seq] [buf,buf] + WholeStageCodegen (1) + Project [d_week_seq] + Filter [d_year,d_week_seq] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_week_seq,d_year] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] + WholeStageCodegen (10) + Project [sales_price,d_week_seq,d_day_name] + BroadcastHashJoin [sold_date_sk,d_date_sk] Project [cs_sold_date_sk,cs_ext_sales_price] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (8) - Filter [d_date_sk,d_week_seq] - Subquery #2 - ObjectHashAggregate [buf] [bloom_filter_agg(xxhash64(d_week_seq, 42), 362, 9656, 0, 0),bloomFilter,buf] - Exchange #9 - ObjectHashAggregate [d_week_seq] [buf,buf] - WholeStageCodegen (1) - Project [d_week_seq] - Filter [d_year,d_week_seq] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_week_seq,d_year] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] + InputAdapter + ReusedExchange [d_date_sk,d_week_seq,d_day_name] #8 InputAdapter BroadcastExchange #10 - WholeStageCodegen (10) + WholeStageCodegen (12) Project [d_week_seq] Filter [d_year,d_week_seq] ColumnarToRow diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt index 8587ac85ae8c2..f6d37eca78b43 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt @@ -1,41 +1,44 @@ == Physical Plan == -* Sort (37) -+- Exchange (36) - +- * Project (35) - +- * BroadcastHashJoin Inner BuildRight (34) - :- * Project (23) - : +- * BroadcastHashJoin Inner BuildRight (22) - : :- * HashAggregate (16) - : : +- Exchange (15) - : : +- * HashAggregate (14) - : : +- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- Union (7) - : : : :- * Project (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : : +- * Project (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet spark_catalog.default.catalog_sales (4) - : : +- BroadcastExchange (11) - : : +- * Filter (10) - : : +- * ColumnarToRow (9) - : : +- Scan parquet spark_catalog.default.date_dim (8) - : +- BroadcastExchange (21) - : +- * Project (20) - : +- * Filter (19) - : +- * ColumnarToRow (18) - : +- Scan parquet spark_catalog.default.date_dim (17) - +- BroadcastExchange (33) - +- * Project (32) - +- * BroadcastHashJoin Inner BuildRight (31) - :- * HashAggregate (25) - : +- ReusedExchange (24) - +- BroadcastExchange (30) - +- * Project (29) - +- * Filter (28) - +- * ColumnarToRow (27) - +- Scan parquet spark_catalog.default.date_dim (26) +* Sort (40) ++- Exchange (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (26) + : +- * BroadcastHashJoin Inner BuildRight (25) + : :- * HashAggregate (19) + : : +- Exchange (18) + : : +- * HashAggregate (17) + : : +- Union (16) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * Project (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet spark_catalog.default.web_sales (1) + : : : +- BroadcastExchange (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet spark_catalog.default.date_dim (4) + : : +- * Project (15) + : : +- * BroadcastHashJoin Inner BuildRight (14) + : : :- * Project (12) + : : : +- * ColumnarToRow (11) + : : : +- Scan parquet spark_catalog.default.catalog_sales (10) + : : +- ReusedExchange (13) + : +- BroadcastExchange (24) + : +- * Project (23) + : +- * Filter (22) + : +- * ColumnarToRow (21) + : +- Scan parquet spark_catalog.default.date_dim (20) + +- BroadcastExchange (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * HashAggregate (28) + : +- ReusedExchange (27) + +- BroadcastExchange (33) + +- * Project (32) + +- * Filter (31) + +- * ColumnarToRow (30) + +- Scan parquet spark_catalog.default.date_dim (29) (1) Scan parquet spark_catalog.default.web_sales @@ -45,168 +48,181 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#2)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 1] +(2) ColumnarToRow [codegen id : 2] Input [2]: [ws_ext_sales_price#1, ws_sold_date_sk#2] -(3) Project [codegen id : 1] +(3) Project [codegen id : 2] Output [2]: [ws_sold_date_sk#2 AS sold_date_sk#3, ws_ext_sales_price#1 AS sales_price#4] Input [2]: [ws_ext_sales_price#1, ws_sold_date_sk#2] -(4) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ext_sales_price#5, cs_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#6)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 2] -Input [2]: [cs_ext_sales_price#5, cs_sold_date_sk#6] - -(6) Project [codegen id : 2] -Output [2]: [cs_sold_date_sk#6 AS sold_date_sk#7, cs_ext_sales_price#5 AS sales_price#8] -Input [2]: [cs_ext_sales_price#5, cs_sold_date_sk#6] - -(7) Union - -(8) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] +(4) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#5, d_week_seq#6, d_day_name#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_week_seq)] ReadSchema: struct -(9) ColumnarToRow [codegen id : 3] -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] +(5) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#5, d_week_seq#6, d_day_name#7] -(10) Filter [codegen id : 3] -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] -Condition : (isnotnull(d_date_sk#9) AND isnotnull(d_week_seq#10)) +(6) Filter [codegen id : 1] +Input [3]: [d_date_sk#5, d_week_seq#6, d_day_name#7] +Condition : (isnotnull(d_date_sk#5) AND isnotnull(d_week_seq#6)) -(11) BroadcastExchange -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] +(7) BroadcastExchange +Input [3]: [d_date_sk#5, d_week_seq#6, d_day_name#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(8) BroadcastHashJoin [codegen id : 2] Left keys [1]: [sold_date_sk#3] -Right keys [1]: [d_date_sk#9] +Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(13) Project [codegen id : 4] -Output [3]: [sales_price#4, d_week_seq#10, d_day_name#11] -Input [5]: [sold_date_sk#3, sales_price#4, d_date_sk#9, d_week_seq#10, d_day_name#11] - -(14) HashAggregate [codegen id : 4] -Input [3]: [sales_price#4, d_week_seq#10, d_day_name#11] -Keys [1]: [d_week_seq#10] -Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))] -Aggregate Attributes [7]: [sum#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18] -Results [8]: [d_week_seq#10, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25] - -(15) Exchange -Input [8]: [d_week_seq#10, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25] -Arguments: hashpartitioning(d_week_seq#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] - -(16) HashAggregate [codegen id : 12] -Input [8]: [d_week_seq#10, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25] -Keys [1]: [d_week_seq#10] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END))#27, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END))#29, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END))#30, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END))#31, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))#32] -Results [8]: [d_week_seq#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END))#26,17,2) AS sun_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END))#27,17,2) AS mon_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END))#28,17,2) AS tue_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END))#29,17,2) AS wed_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END))#30,17,2) AS thu_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END))#31,17,2) AS fri_sales#38, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))#32,17,2) AS sat_sales#39] - -(17) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#40, d_year#41] +(9) Project [codegen id : 2] +Output [3]: [sales_price#4, d_week_seq#6, d_day_name#7] +Input [5]: [sold_date_sk#3, sales_price#4, d_date_sk#5, d_week_seq#6, d_day_name#7] + +(10) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_ext_sales_price#8, cs_sold_date_sk#9] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#9)] +ReadSchema: struct + +(11) ColumnarToRow [codegen id : 4] +Input [2]: [cs_ext_sales_price#8, cs_sold_date_sk#9] + +(12) Project [codegen id : 4] +Output [2]: [cs_sold_date_sk#9 AS sold_date_sk#10, cs_ext_sales_price#8 AS sales_price#11] +Input [2]: [cs_ext_sales_price#8, cs_sold_date_sk#9] + +(13) ReusedExchange [Reuses operator id: 7] +Output [3]: [d_date_sk#12, d_week_seq#13, d_day_name#14] + +(14) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [sold_date_sk#10] +Right keys [1]: [d_date_sk#12] +Join type: Inner +Join condition: None + +(15) Project [codegen id : 4] +Output [3]: [sales_price#11, d_week_seq#13, d_day_name#14] +Input [5]: [sold_date_sk#10, sales_price#11, d_date_sk#12, d_week_seq#13, d_day_name#14] + +(16) Union + +(17) HashAggregate [codegen id : 5] +Input [3]: [sales_price#4, d_week_seq#6, d_day_name#7] +Keys [1]: [d_week_seq#6] +Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN sales_price#4 END))] +Aggregate Attributes [7]: [sum#15, sum#16, sum#17, sum#18, sum#19, sum#20, sum#21] +Results [8]: [d_week_seq#6, sum#22, sum#23, sum#24, sum#25, sum#26, sum#27, sum#28] + +(18) Exchange +Input [8]: [d_week_seq#6, sum#22, sum#23, sum#24, sum#25, sum#26, sum#27, sum#28] +Arguments: hashpartitioning(d_week_seq#6, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(19) HashAggregate [codegen id : 14] +Input [8]: [d_week_seq#6, sum#22, sum#23, sum#24, sum#25, sum#26, sum#27, sum#28] +Keys [1]: [d_week_seq#6] +Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN sales_price#4 END))] +Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN sales_price#4 END))#29, sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN sales_price#4 END))#30, sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN sales_price#4 END))#31, sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN sales_price#4 END))#32, sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN sales_price#4 END))#33, sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN sales_price#4 END))#34, sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN sales_price#4 END))#35] +Results [8]: [d_week_seq#6, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN sales_price#4 END))#29,17,2) AS sun_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN sales_price#4 END))#30,17,2) AS mon_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN sales_price#4 END))#31,17,2) AS tue_sales#38, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN sales_price#4 END))#32,17,2) AS wed_sales#39, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN sales_price#4 END))#33,17,2) AS thu_sales#40, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN sales_price#4 END))#34,17,2) AS fri_sales#41, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN sales_price#4 END))#35,17,2) AS sat_sales#42] + +(20) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_week_seq#43, d_year#44] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_week_seq)] ReadSchema: struct -(18) ColumnarToRow [codegen id : 5] -Input [2]: [d_week_seq#40, d_year#41] +(21) ColumnarToRow [codegen id : 6] +Input [2]: [d_week_seq#43, d_year#44] -(19) Filter [codegen id : 5] -Input [2]: [d_week_seq#40, d_year#41] -Condition : ((isnotnull(d_year#41) AND (d_year#41 = 2001)) AND isnotnull(d_week_seq#40)) +(22) Filter [codegen id : 6] +Input [2]: [d_week_seq#43, d_year#44] +Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2001)) AND isnotnull(d_week_seq#43)) -(20) Project [codegen id : 5] -Output [1]: [d_week_seq#40] -Input [2]: [d_week_seq#40, d_year#41] +(23) Project [codegen id : 6] +Output [1]: [d_week_seq#43] +Input [2]: [d_week_seq#43, d_year#44] -(21) BroadcastExchange -Input [1]: [d_week_seq#40] +(24) BroadcastExchange +Input [1]: [d_week_seq#43] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(22) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [d_week_seq#10] -Right keys [1]: [d_week_seq#40] +(25) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [d_week_seq#6] +Right keys [1]: [d_week_seq#43] Join type: Inner Join condition: None -(23) Project [codegen id : 12] -Output [8]: [d_week_seq#10 AS d_week_seq1#42, sun_sales#33 AS sun_sales1#43, mon_sales#34 AS mon_sales1#44, tue_sales#35 AS tue_sales1#45, wed_sales#36 AS wed_sales1#46, thu_sales#37 AS thu_sales1#47, fri_sales#38 AS fri_sales1#48, sat_sales#39 AS sat_sales1#49] -Input [9]: [d_week_seq#10, sun_sales#33, mon_sales#34, tue_sales#35, wed_sales#36, thu_sales#37, fri_sales#38, sat_sales#39, d_week_seq#40] +(26) Project [codegen id : 14] +Output [8]: [d_week_seq#6 AS d_week_seq1#45, sun_sales#36 AS sun_sales1#46, mon_sales#37 AS mon_sales1#47, tue_sales#38 AS tue_sales1#48, wed_sales#39 AS wed_sales1#49, thu_sales#40 AS thu_sales1#50, fri_sales#41 AS fri_sales1#51, sat_sales#42 AS sat_sales1#52] +Input [9]: [d_week_seq#6, sun_sales#36, mon_sales#37, tue_sales#38, wed_sales#39, thu_sales#40, fri_sales#41, sat_sales#42, d_week_seq#43] -(24) ReusedExchange [Reuses operator id: 15] -Output [8]: [d_week_seq#50, sum#51, sum#52, sum#53, sum#54, sum#55, sum#56, sum#57] +(27) ReusedExchange [Reuses operator id: 18] +Output [8]: [d_week_seq#53, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59, sum#60] -(25) HashAggregate [codegen id : 11] -Input [8]: [d_week_seq#50, sum#51, sum#52, sum#53, sum#54, sum#55, sum#56, sum#57] -Keys [1]: [d_week_seq#50] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#58 = Sunday ) THEN sales_price#59 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Monday ) THEN sales_price#59 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Tuesday ) THEN sales_price#59 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Wednesday) THEN sales_price#59 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Thursday ) THEN sales_price#59 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Friday ) THEN sales_price#59 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Saturday ) THEN sales_price#59 END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#58 = Sunday ) THEN sales_price#59 END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Monday ) THEN sales_price#59 END))#27, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Tuesday ) THEN sales_price#59 END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Wednesday) THEN sales_price#59 END))#29, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Thursday ) THEN sales_price#59 END))#30, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Friday ) THEN sales_price#59 END))#31, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Saturday ) THEN sales_price#59 END))#32] -Results [8]: [d_week_seq#50, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Sunday ) THEN sales_price#59 END))#26,17,2) AS sun_sales#60, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Monday ) THEN sales_price#59 END))#27,17,2) AS mon_sales#61, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Tuesday ) THEN sales_price#59 END))#28,17,2) AS tue_sales#62, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Wednesday) THEN sales_price#59 END))#29,17,2) AS wed_sales#63, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Thursday ) THEN sales_price#59 END))#30,17,2) AS thu_sales#64, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Friday ) THEN sales_price#59 END))#31,17,2) AS fri_sales#65, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Saturday ) THEN sales_price#59 END))#32,17,2) AS sat_sales#66] +(28) HashAggregate [codegen id : 13] +Input [8]: [d_week_seq#53, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59, sum#60] +Keys [1]: [d_week_seq#53] +Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#61 = Sunday ) THEN sales_price#62 END)), sum(UnscaledValue(CASE WHEN (d_day_name#61 = Monday ) THEN sales_price#62 END)), sum(UnscaledValue(CASE WHEN (d_day_name#61 = Tuesday ) THEN sales_price#62 END)), sum(UnscaledValue(CASE WHEN (d_day_name#61 = Wednesday) THEN sales_price#62 END)), sum(UnscaledValue(CASE WHEN (d_day_name#61 = Thursday ) THEN sales_price#62 END)), sum(UnscaledValue(CASE WHEN (d_day_name#61 = Friday ) THEN sales_price#62 END)), sum(UnscaledValue(CASE WHEN (d_day_name#61 = Saturday ) THEN sales_price#62 END))] +Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#61 = Sunday ) THEN sales_price#62 END))#29, sum(UnscaledValue(CASE WHEN (d_day_name#61 = Monday ) THEN sales_price#62 END))#30, sum(UnscaledValue(CASE WHEN (d_day_name#61 = Tuesday ) THEN sales_price#62 END))#31, sum(UnscaledValue(CASE WHEN (d_day_name#61 = Wednesday) THEN sales_price#62 END))#32, sum(UnscaledValue(CASE WHEN (d_day_name#61 = Thursday ) THEN sales_price#62 END))#33, sum(UnscaledValue(CASE WHEN (d_day_name#61 = Friday ) THEN sales_price#62 END))#34, sum(UnscaledValue(CASE WHEN (d_day_name#61 = Saturday ) THEN sales_price#62 END))#35] +Results [8]: [d_week_seq#53, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#61 = Sunday ) THEN sales_price#62 END))#29,17,2) AS sun_sales#63, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#61 = Monday ) THEN sales_price#62 END))#30,17,2) AS mon_sales#64, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#61 = Tuesday ) THEN sales_price#62 END))#31,17,2) AS tue_sales#65, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#61 = Wednesday) THEN sales_price#62 END))#32,17,2) AS wed_sales#66, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#61 = Thursday ) THEN sales_price#62 END))#33,17,2) AS thu_sales#67, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#61 = Friday ) THEN sales_price#62 END))#34,17,2) AS fri_sales#68, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#61 = Saturday ) THEN sales_price#62 END))#35,17,2) AS sat_sales#69] -(26) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#67, d_year#68] +(29) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_week_seq#70, d_year#71] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_week_seq)] ReadSchema: struct -(27) ColumnarToRow [codegen id : 10] -Input [2]: [d_week_seq#67, d_year#68] +(30) ColumnarToRow [codegen id : 12] +Input [2]: [d_week_seq#70, d_year#71] -(28) Filter [codegen id : 10] -Input [2]: [d_week_seq#67, d_year#68] -Condition : ((isnotnull(d_year#68) AND (d_year#68 = 2002)) AND isnotnull(d_week_seq#67)) +(31) Filter [codegen id : 12] +Input [2]: [d_week_seq#70, d_year#71] +Condition : ((isnotnull(d_year#71) AND (d_year#71 = 2002)) AND isnotnull(d_week_seq#70)) -(29) Project [codegen id : 10] -Output [1]: [d_week_seq#67] -Input [2]: [d_week_seq#67, d_year#68] +(32) Project [codegen id : 12] +Output [1]: [d_week_seq#70] +Input [2]: [d_week_seq#70, d_year#71] -(30) BroadcastExchange -Input [1]: [d_week_seq#67] +(33) BroadcastExchange +Input [1]: [d_week_seq#70] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(31) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [d_week_seq#50] -Right keys [1]: [d_week_seq#67] +(34) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [d_week_seq#53] +Right keys [1]: [d_week_seq#70] Join type: Inner Join condition: None -(32) Project [codegen id : 11] -Output [8]: [d_week_seq#50 AS d_week_seq2#69, sun_sales#60 AS sun_sales2#70, mon_sales#61 AS mon_sales2#71, tue_sales#62 AS tue_sales2#72, wed_sales#63 AS wed_sales2#73, thu_sales#64 AS thu_sales2#74, fri_sales#65 AS fri_sales2#75, sat_sales#66 AS sat_sales2#76] -Input [9]: [d_week_seq#50, sun_sales#60, mon_sales#61, tue_sales#62, wed_sales#63, thu_sales#64, fri_sales#65, sat_sales#66, d_week_seq#67] +(35) Project [codegen id : 13] +Output [8]: [d_week_seq#53 AS d_week_seq2#72, sun_sales#63 AS sun_sales2#73, mon_sales#64 AS mon_sales2#74, tue_sales#65 AS tue_sales2#75, wed_sales#66 AS wed_sales2#76, thu_sales#67 AS thu_sales2#77, fri_sales#68 AS fri_sales2#78, sat_sales#69 AS sat_sales2#79] +Input [9]: [d_week_seq#53, sun_sales#63, mon_sales#64, tue_sales#65, wed_sales#66, thu_sales#67, fri_sales#68, sat_sales#69, d_week_seq#70] -(33) BroadcastExchange -Input [8]: [d_week_seq2#69, sun_sales2#70, mon_sales2#71, tue_sales2#72, wed_sales2#73, thu_sales2#74, fri_sales2#75, sat_sales2#76] +(36) BroadcastExchange +Input [8]: [d_week_seq2#72, sun_sales2#73, mon_sales2#74, tue_sales2#75, wed_sales2#76, thu_sales2#77, fri_sales2#78, sat_sales2#79] Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [plan_id=5] -(34) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [d_week_seq1#42] -Right keys [1]: [(d_week_seq2#69 - 53)] +(37) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [d_week_seq1#45] +Right keys [1]: [(d_week_seq2#72 - 53)] Join type: Inner Join condition: None -(35) Project [codegen id : 12] -Output [8]: [d_week_seq1#42, round((sun_sales1#43 / sun_sales2#70), 2) AS round((sun_sales1 / sun_sales2), 2)#77, round((mon_sales1#44 / mon_sales2#71), 2) AS round((mon_sales1 / mon_sales2), 2)#78, round((tue_sales1#45 / tue_sales2#72), 2) AS round((tue_sales1 / tue_sales2), 2)#79, round((wed_sales1#46 / wed_sales2#73), 2) AS round((wed_sales1 / wed_sales2), 2)#80, round((thu_sales1#47 / thu_sales2#74), 2) AS round((thu_sales1 / thu_sales2), 2)#81, round((fri_sales1#48 / fri_sales2#75), 2) AS round((fri_sales1 / fri_sales2), 2)#82, round((sat_sales1#49 / sat_sales2#76), 2) AS round((sat_sales1 / sat_sales2), 2)#83] -Input [16]: [d_week_seq1#42, sun_sales1#43, mon_sales1#44, tue_sales1#45, wed_sales1#46, thu_sales1#47, fri_sales1#48, sat_sales1#49, d_week_seq2#69, sun_sales2#70, mon_sales2#71, tue_sales2#72, wed_sales2#73, thu_sales2#74, fri_sales2#75, sat_sales2#76] +(38) Project [codegen id : 14] +Output [8]: [d_week_seq1#45, round((sun_sales1#46 / sun_sales2#73), 2) AS round((sun_sales1 / sun_sales2), 2)#80, round((mon_sales1#47 / mon_sales2#74), 2) AS round((mon_sales1 / mon_sales2), 2)#81, round((tue_sales1#48 / tue_sales2#75), 2) AS round((tue_sales1 / tue_sales2), 2)#82, round((wed_sales1#49 / wed_sales2#76), 2) AS round((wed_sales1 / wed_sales2), 2)#83, round((thu_sales1#50 / thu_sales2#77), 2) AS round((thu_sales1 / thu_sales2), 2)#84, round((fri_sales1#51 / fri_sales2#78), 2) AS round((fri_sales1 / fri_sales2), 2)#85, round((sat_sales1#52 / sat_sales2#79), 2) AS round((sat_sales1 / sat_sales2), 2)#86] +Input [16]: [d_week_seq1#45, sun_sales1#46, mon_sales1#47, tue_sales1#48, wed_sales1#49, thu_sales1#50, fri_sales1#51, sat_sales1#52, d_week_seq2#72, sun_sales2#73, mon_sales2#74, tue_sales2#75, wed_sales2#76, thu_sales2#77, fri_sales2#78, sat_sales2#79] -(36) Exchange -Input [8]: [d_week_seq1#42, round((sun_sales1 / sun_sales2), 2)#77, round((mon_sales1 / mon_sales2), 2)#78, round((tue_sales1 / tue_sales2), 2)#79, round((wed_sales1 / wed_sales2), 2)#80, round((thu_sales1 / thu_sales2), 2)#81, round((fri_sales1 / fri_sales2), 2)#82, round((sat_sales1 / sat_sales2), 2)#83] -Arguments: rangepartitioning(d_week_seq1#42 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(39) Exchange +Input [8]: [d_week_seq1#45, round((sun_sales1 / sun_sales2), 2)#80, round((mon_sales1 / mon_sales2), 2)#81, round((tue_sales1 / tue_sales2), 2)#82, round((wed_sales1 / wed_sales2), 2)#83, round((thu_sales1 / thu_sales2), 2)#84, round((fri_sales1 / fri_sales2), 2)#85, round((sat_sales1 / sat_sales2), 2)#86] +Arguments: rangepartitioning(d_week_seq1#45 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(37) Sort [codegen id : 13] -Input [8]: [d_week_seq1#42, round((sun_sales1 / sun_sales2), 2)#77, round((mon_sales1 / mon_sales2), 2)#78, round((tue_sales1 / tue_sales2), 2)#79, round((wed_sales1 / wed_sales2), 2)#80, round((thu_sales1 / thu_sales2), 2)#81, round((fri_sales1 / fri_sales2), 2)#82, round((sat_sales1 / sat_sales2), 2)#83] -Arguments: [d_week_seq1#42 ASC NULLS FIRST], true, 0 +(40) Sort [codegen id : 15] +Input [8]: [d_week_seq1#45, round((sun_sales1 / sun_sales2), 2)#80, round((mon_sales1 / mon_sales2), 2)#81, round((tue_sales1 / tue_sales2), 2)#82, round((wed_sales1 / wed_sales2), 2)#83, round((thu_sales1 / thu_sales2), 2)#84, round((fri_sales1 / fri_sales2), 2)#85, round((sat_sales1 / sat_sales2), 2)#86] +Arguments: [d_week_seq1#45 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt index 1b34ac798b9cb..8112fa31feb0b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (13) +WholeStageCodegen (15) Sort [d_week_seq1] InputAdapter Exchange [d_week_seq1] #1 - WholeStageCodegen (12) + WholeStageCodegen (14) Project [d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] BroadcastHashJoin [d_week_seq1,d_week_seq2] Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] @@ -10,32 +10,36 @@ WholeStageCodegen (13) HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] InputAdapter Exchange [d_week_seq] #2 - WholeStageCodegen (4) + WholeStageCodegen (5) HashAggregate [d_week_seq,d_day_name,sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,d_week_seq,d_day_name] - BroadcastHashJoin [sold_date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (1) + InputAdapter + Union + WholeStageCodegen (2) + Project [sales_price,d_week_seq,d_day_name] + BroadcastHashJoin [sold_date_sk,d_date_sk] Project [ws_sold_date_sk,ws_ext_sales_price] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_sales [ws_ext_sales_price,ws_sold_date_sk] - WholeStageCodegen (2) + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Filter [d_date_sk,d_week_seq] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] + WholeStageCodegen (4) + Project [sales_price,d_week_seq,d_day_name] + BroadcastHashJoin [sold_date_sk,d_date_sk] Project [cs_sold_date_sk,cs_ext_sales_price] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (3) - Filter [d_date_sk,d_week_seq] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] + InputAdapter + ReusedExchange [d_date_sk,d_week_seq,d_day_name] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (5) + WholeStageCodegen (6) Project [d_week_seq] Filter [d_year,d_week_seq] ColumnarToRow @@ -43,7 +47,7 @@ WholeStageCodegen (13) Scan parquet spark_catalog.default.date_dim [d_week_seq,d_year] InputAdapter BroadcastExchange #5 - WholeStageCodegen (11) + WholeStageCodegen (13) Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] @@ -51,7 +55,7 @@ WholeStageCodegen (13) ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 InputAdapter BroadcastExchange #6 - WholeStageCodegen (10) + WholeStageCodegen (12) Project [d_week_seq] Filter [d_year,d_week_seq] ColumnarToRow diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt index f6b3cfe81e786..619446d2bbcee 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt @@ -1,81 +1,90 @@ == Physical Plan == -TakeOrderedAndProject (77) -+- * HashAggregate (76) - +- Exchange (75) - +- * HashAggregate (74) - +- * Expand (73) - +- Union (72) - :- * HashAggregate (21) - : +- Exchange (20) - : +- * HashAggregate (19) - : +- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- Union (9) - : : : :- * Project (4) - : : : : +- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- * Project (8) - : : : +- * Filter (7) - : : : +- * ColumnarToRow (6) - : : : +- Scan parquet spark_catalog.default.store_returns (5) - : : +- ReusedExchange (10) - : +- BroadcastExchange (16) - : +- * Filter (15) - : +- * ColumnarToRow (14) - : +- Scan parquet spark_catalog.default.store (13) - :- * HashAggregate (42) - : +- Exchange (41) - : +- * HashAggregate (40) - : +- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (33) - : : +- * BroadcastHashJoin Inner BuildRight (32) - : : :- Union (30) - : : : :- * Project (25) - : : : : +- * Filter (24) - : : : : +- * ColumnarToRow (23) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (22) - : : : +- * Project (29) - : : : +- * Filter (28) - : : : +- * ColumnarToRow (27) - : : : +- Scan parquet spark_catalog.default.catalog_returns (26) - : : +- ReusedExchange (31) - : +- BroadcastExchange (37) - : +- * Filter (36) - : +- * ColumnarToRow (35) - : +- Scan parquet spark_catalog.default.catalog_page (34) - +- * HashAggregate (71) - +- Exchange (70) - +- * HashAggregate (69) - +- * Project (68) - +- * BroadcastHashJoin Inner BuildRight (67) - :- * Project (62) - : +- * BroadcastHashJoin Inner BuildRight (61) - : :- Union (59) - : : :- * Project (46) - : : : +- * Filter (45) - : : : +- * ColumnarToRow (44) - : : : +- Scan parquet spark_catalog.default.web_sales (43) - : : +- * Project (58) - : : +- * SortMergeJoin Inner (57) - : : :- * Sort (50) - : : : +- Exchange (49) - : : : +- * ColumnarToRow (48) - : : : +- Scan parquet spark_catalog.default.web_returns (47) - : : +- * Sort (56) - : : +- Exchange (55) - : : +- * Project (54) - : : +- * Filter (53) - : : +- * ColumnarToRow (52) - : : +- Scan parquet spark_catalog.default.web_sales (51) - : +- ReusedExchange (60) - +- BroadcastExchange (66) - +- * Filter (65) - +- * ColumnarToRow (64) - +- Scan parquet spark_catalog.default.web_site (63) +TakeOrderedAndProject (86) ++- * HashAggregate (85) + +- Exchange (84) + +- * HashAggregate (83) + +- * Expand (82) + +- Union (81) + :- * HashAggregate (24) + : +- Exchange (23) + : +- * HashAggregate (22) + : +- * Project (21) + : +- * BroadcastHashJoin Inner BuildRight (20) + : :- Union (15) + : : :- * Project (7) + : : : +- * BroadcastHashJoin Inner BuildRight (6) + : : : :- * Project (4) + : : : : +- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (5) + : : +- * Project (14) + : : +- * BroadcastHashJoin Inner BuildRight (13) + : : :- * Project (11) + : : : +- * Filter (10) + : : : +- * ColumnarToRow (9) + : : : +- Scan parquet spark_catalog.default.store_returns (8) + : : +- ReusedExchange (12) + : +- BroadcastExchange (19) + : +- * Filter (18) + : +- * ColumnarToRow (17) + : +- Scan parquet spark_catalog.default.store (16) + :- * HashAggregate (48) + : +- Exchange (47) + : +- * HashAggregate (46) + : +- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) + : :- Union (39) + : : :- * Project (31) + : : : +- * BroadcastHashJoin Inner BuildRight (30) + : : : :- * Project (28) + : : : : +- * Filter (27) + : : : : +- * ColumnarToRow (26) + : : : : +- Scan parquet spark_catalog.default.catalog_sales (25) + : : : +- ReusedExchange (29) + : : +- * Project (38) + : : +- * BroadcastHashJoin Inner BuildRight (37) + : : :- * Project (35) + : : : +- * Filter (34) + : : : +- * ColumnarToRow (33) + : : : +- Scan parquet spark_catalog.default.catalog_returns (32) + : : +- ReusedExchange (36) + : +- BroadcastExchange (43) + : +- * Filter (42) + : +- * ColumnarToRow (41) + : +- Scan parquet spark_catalog.default.catalog_page (40) + +- * HashAggregate (80) + +- Exchange (79) + +- * HashAggregate (78) + +- * Project (77) + +- * BroadcastHashJoin Inner BuildRight (76) + :- Union (71) + : :- * Project (55) + : : +- * BroadcastHashJoin Inner BuildRight (54) + : : :- * Project (52) + : : : +- * Filter (51) + : : : +- * ColumnarToRow (50) + : : : +- Scan parquet spark_catalog.default.web_sales (49) + : : +- ReusedExchange (53) + : +- * Project (70) + : +- * BroadcastHashJoin Inner BuildRight (69) + : :- * Project (67) + : : +- * SortMergeJoin Inner (66) + : : :- * Sort (59) + : : : +- Exchange (58) + : : : +- * ColumnarToRow (57) + : : : +- Scan parquet spark_catalog.default.web_returns (56) + : : +- * Sort (65) + : : +- Exchange (64) + : : +- * Project (63) + : : +- * Filter (62) + : : +- * ColumnarToRow (61) + : : +- Scan parquet spark_catalog.default.web_sales (60) + : +- ReusedExchange (68) + +- BroadcastExchange (75) + +- * Filter (74) + +- * ColumnarToRow (73) + +- Scan parquet spark_catalog.default.web_site (72) (1) Scan parquet spark_catalog.default.store_sales @@ -86,398 +95,437 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 1] +(2) ColumnarToRow [codegen id : 2] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 1] +(3) Filter [codegen id : 2] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] Condition : isnotnull(ss_store_sk#1) -(4) Project [codegen id : 1] +(4) Project [codegen id : 2] Output [6]: [ss_store_sk#1 AS store_sk#6, ss_sold_date_sk#4 AS date_sk#7, ss_ext_sales_price#2 AS sales_price#8, ss_net_profit#3 AS profit#9, 0.00 AS return_amt#10, 0.00 AS net_loss#11] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -(5) Scan parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] +(5) ReusedExchange [Reuses operator id: 91] +Output [1]: [d_date_sk#12] + +(6) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [date_sk#7] +Right keys [1]: [d_date_sk#12] +Join type: Inner +Join condition: None + +(7) Project [codegen id : 2] +Output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] +Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#12] + +(8) Scan parquet spark_catalog.default.store_returns +Output [4]: [sr_store_sk#13, sr_return_amt#14, sr_net_loss#15, sr_returned_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#15), dynamicpruningexpression(sr_returned_date_sk#15 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(sr_returned_date_sk#16), dynamicpruningexpression(sr_returned_date_sk#16 IN dynamicpruning#5)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(6) ColumnarToRow [codegen id : 2] -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] +(9) ColumnarToRow [codegen id : 4] +Input [4]: [sr_store_sk#13, sr_return_amt#14, sr_net_loss#15, sr_returned_date_sk#16] -(7) Filter [codegen id : 2] -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Condition : isnotnull(sr_store_sk#12) +(10) Filter [codegen id : 4] +Input [4]: [sr_store_sk#13, sr_return_amt#14, sr_net_loss#15, sr_returned_date_sk#16] +Condition : isnotnull(sr_store_sk#13) -(8) Project [codegen id : 2] -Output [6]: [sr_store_sk#12 AS store_sk#16, sr_returned_date_sk#15 AS date_sk#17, 0.00 AS sales_price#18, 0.00 AS profit#19, sr_return_amt#13 AS return_amt#20, sr_net_loss#14 AS net_loss#21] -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] +(11) Project [codegen id : 4] +Output [6]: [sr_store_sk#13 AS store_sk#17, sr_returned_date_sk#16 AS date_sk#18, 0.00 AS sales_price#19, 0.00 AS profit#20, sr_return_amt#14 AS return_amt#21, sr_net_loss#15 AS net_loss#22] +Input [4]: [sr_store_sk#13, sr_return_amt#14, sr_net_loss#15, sr_returned_date_sk#16] -(9) Union +(12) ReusedExchange [Reuses operator id: 91] +Output [1]: [d_date_sk#23] -(10) ReusedExchange [Reuses operator id: 82] -Output [1]: [d_date_sk#22] - -(11) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [date_sk#7] -Right keys [1]: [d_date_sk#22] +(13) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [date_sk#18] +Right keys [1]: [d_date_sk#23] Join type: Inner Join condition: None -(12) Project [codegen id : 5] -Output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] -Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] +(14) Project [codegen id : 4] +Output [5]: [store_sk#17, sales_price#19, profit#20, return_amt#21, net_loss#22] +Input [7]: [store_sk#17, date_sk#18, sales_price#19, profit#20, return_amt#21, net_loss#22, d_date_sk#23] -(13) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#23, s_store_id#24] +(15) Union + +(16) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#24, s_store_id#25] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(14) ColumnarToRow [codegen id : 4] -Input [2]: [s_store_sk#23, s_store_id#24] +(17) ColumnarToRow [codegen id : 5] +Input [2]: [s_store_sk#24, s_store_id#25] -(15) Filter [codegen id : 4] -Input [2]: [s_store_sk#23, s_store_id#24] -Condition : isnotnull(s_store_sk#23) +(18) Filter [codegen id : 5] +Input [2]: [s_store_sk#24, s_store_id#25] +Condition : isnotnull(s_store_sk#24) -(16) BroadcastExchange -Input [2]: [s_store_sk#23, s_store_id#24] +(19) BroadcastExchange +Input [2]: [s_store_sk#24, s_store_id#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(17) BroadcastHashJoin [codegen id : 5] +(20) BroadcastHashJoin [codegen id : 6] Left keys [1]: [store_sk#6] -Right keys [1]: [s_store_sk#23] +Right keys [1]: [s_store_sk#24] Join type: Inner Join condition: None -(18) Project [codegen id : 5] -Output [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#24] -Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#23, s_store_id#24] +(21) Project [codegen id : 6] +Output [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] +Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#24, s_store_id#25] -(19) HashAggregate [codegen id : 5] -Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#24] -Keys [1]: [s_store_id#24] +(22) HashAggregate [codegen id : 6] +Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] +Keys [1]: [s_store_id#25] Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum#25, sum#26, sum#27, sum#28] -Results [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] +Aggregate Attributes [4]: [sum#26, sum#27, sum#28, sum#29] +Results [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] -(20) Exchange -Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] -Arguments: hashpartitioning(s_store_id#24, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(23) Exchange +Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] +Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(21) HashAggregate [codegen id : 6] -Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] -Keys [1]: [s_store_id#24] +(24) HashAggregate [codegen id : 7] +Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] +Keys [1]: [s_store_id#25] Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#33, sum(UnscaledValue(return_amt#10))#34, sum(UnscaledValue(profit#9))#35, sum(UnscaledValue(net_loss#11))#36] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#8))#33,17,2) AS sales#37, MakeDecimal(sum(UnscaledValue(return_amt#10))#34,17,2) AS returns#38, (MakeDecimal(sum(UnscaledValue(profit#9))#35,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#36,17,2)) AS profit#39, store channel AS channel#40, concat(store, s_store_id#24) AS id#41] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#34, sum(UnscaledValue(return_amt#10))#35, sum(UnscaledValue(profit#9))#36, sum(UnscaledValue(net_loss#11))#37] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#8))#34,17,2) AS sales#38, MakeDecimal(sum(UnscaledValue(return_amt#10))#35,17,2) AS returns#39, (MakeDecimal(sum(UnscaledValue(profit#9))#36,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#37,17,2)) AS profit#40, store channel AS channel#41, concat(store, s_store_id#25) AS id#42] -(22) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +(25) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#45), dynamicpruningexpression(cs_sold_date_sk#45 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#46), dynamicpruningexpression(cs_sold_date_sk#46 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct -(23) ColumnarToRow [codegen id : 7] -Input [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +(26) ColumnarToRow [codegen id : 9] +Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] + +(27) Filter [codegen id : 9] +Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] +Condition : isnotnull(cs_catalog_page_sk#43) -(24) Filter [codegen id : 7] -Input [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Condition : isnotnull(cs_catalog_page_sk#42) +(28) Project [codegen id : 9] +Output [6]: [cs_catalog_page_sk#43 AS page_sk#47, cs_sold_date_sk#46 AS date_sk#48, cs_ext_sales_price#44 AS sales_price#49, cs_net_profit#45 AS profit#50, 0.00 AS return_amt#51, 0.00 AS net_loss#52] +Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] -(25) Project [codegen id : 7] -Output [6]: [cs_catalog_page_sk#42 AS page_sk#46, cs_sold_date_sk#45 AS date_sk#47, cs_ext_sales_price#43 AS sales_price#48, cs_net_profit#44 AS profit#49, 0.00 AS return_amt#50, 0.00 AS net_loss#51] -Input [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +(29) ReusedExchange [Reuses operator id: 91] +Output [1]: [d_date_sk#53] -(26) Scan parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#52, cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] +(30) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [date_sk#48] +Right keys [1]: [d_date_sk#53] +Join type: Inner +Join condition: None + +(31) Project [codegen id : 9] +Output [5]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52] +Input [7]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52, d_date_sk#53] + +(32) Scan parquet spark_catalog.default.catalog_returns +Output [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#55), dynamicpruningexpression(cr_returned_date_sk#55 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cr_returned_date_sk#57), dynamicpruningexpression(cr_returned_date_sk#57 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct -(27) ColumnarToRow [codegen id : 8] -Input [4]: [cr_catalog_page_sk#52, cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] +(33) ColumnarToRow [codegen id : 11] +Input [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] -(28) Filter [codegen id : 8] -Input [4]: [cr_catalog_page_sk#52, cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] -Condition : isnotnull(cr_catalog_page_sk#52) +(34) Filter [codegen id : 11] +Input [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] +Condition : isnotnull(cr_catalog_page_sk#54) -(29) Project [codegen id : 8] -Output [6]: [cr_catalog_page_sk#52 AS page_sk#56, cr_returned_date_sk#55 AS date_sk#57, 0.00 AS sales_price#58, 0.00 AS profit#59, cr_return_amount#53 AS return_amt#60, cr_net_loss#54 AS net_loss#61] -Input [4]: [cr_catalog_page_sk#52, cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] +(35) Project [codegen id : 11] +Output [6]: [cr_catalog_page_sk#54 AS page_sk#58, cr_returned_date_sk#57 AS date_sk#59, 0.00 AS sales_price#60, 0.00 AS profit#61, cr_return_amount#55 AS return_amt#62, cr_net_loss#56 AS net_loss#63] +Input [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] -(30) Union +(36) ReusedExchange [Reuses operator id: 91] +Output [1]: [d_date_sk#64] -(31) ReusedExchange [Reuses operator id: 82] -Output [1]: [d_date_sk#62] - -(32) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [date_sk#47] -Right keys [1]: [d_date_sk#62] +(37) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [date_sk#59] +Right keys [1]: [d_date_sk#64] Join type: Inner Join condition: None -(33) Project [codegen id : 11] -Output [5]: [page_sk#46, sales_price#48, profit#49, return_amt#50, net_loss#51] -Input [7]: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51, d_date_sk#62] +(38) Project [codegen id : 11] +Output [5]: [page_sk#58, sales_price#60, profit#61, return_amt#62, net_loss#63] +Input [7]: [page_sk#58, date_sk#59, sales_price#60, profit#61, return_amt#62, net_loss#63, d_date_sk#64] + +(39) Union -(34) Scan parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#63, cp_catalog_page_id#64] +(40) Scan parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(35) ColumnarToRow [codegen id : 10] -Input [2]: [cp_catalog_page_sk#63, cp_catalog_page_id#64] +(41) ColumnarToRow [codegen id : 12] +Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] -(36) Filter [codegen id : 10] -Input [2]: [cp_catalog_page_sk#63, cp_catalog_page_id#64] -Condition : isnotnull(cp_catalog_page_sk#63) +(42) Filter [codegen id : 12] +Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] +Condition : isnotnull(cp_catalog_page_sk#65) -(37) BroadcastExchange -Input [2]: [cp_catalog_page_sk#63, cp_catalog_page_id#64] +(43) BroadcastExchange +Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(38) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [page_sk#46] -Right keys [1]: [cp_catalog_page_sk#63] +(44) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [page_sk#47] +Right keys [1]: [cp_catalog_page_sk#65] Join type: Inner Join condition: None -(39) Project [codegen id : 11] -Output [5]: [sales_price#48, profit#49, return_amt#50, net_loss#51, cp_catalog_page_id#64] -Input [7]: [page_sk#46, sales_price#48, profit#49, return_amt#50, net_loss#51, cp_catalog_page_sk#63, cp_catalog_page_id#64] - -(40) HashAggregate [codegen id : 11] -Input [5]: [sales_price#48, profit#49, return_amt#50, net_loss#51, cp_catalog_page_id#64] -Keys [1]: [cp_catalog_page_id#64] -Functions [4]: [partial_sum(UnscaledValue(sales_price#48)), partial_sum(UnscaledValue(return_amt#50)), partial_sum(UnscaledValue(profit#49)), partial_sum(UnscaledValue(net_loss#51))] -Aggregate Attributes [4]: [sum#65, sum#66, sum#67, sum#68] -Results [5]: [cp_catalog_page_id#64, sum#69, sum#70, sum#71, sum#72] - -(41) Exchange -Input [5]: [cp_catalog_page_id#64, sum#69, sum#70, sum#71, sum#72] -Arguments: hashpartitioning(cp_catalog_page_id#64, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(42) HashAggregate [codegen id : 12] -Input [5]: [cp_catalog_page_id#64, sum#69, sum#70, sum#71, sum#72] -Keys [1]: [cp_catalog_page_id#64] -Functions [4]: [sum(UnscaledValue(sales_price#48)), sum(UnscaledValue(return_amt#50)), sum(UnscaledValue(profit#49)), sum(UnscaledValue(net_loss#51))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#48))#73, sum(UnscaledValue(return_amt#50))#74, sum(UnscaledValue(profit#49))#75, sum(UnscaledValue(net_loss#51))#76] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#48))#73,17,2) AS sales#77, MakeDecimal(sum(UnscaledValue(return_amt#50))#74,17,2) AS returns#78, (MakeDecimal(sum(UnscaledValue(profit#49))#75,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#51))#76,17,2)) AS profit#79, catalog channel AS channel#80, concat(catalog_page, cp_catalog_page_id#64) AS id#81] - -(43) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#82, ws_ext_sales_price#83, ws_net_profit#84, ws_sold_date_sk#85] +(45) Project [codegen id : 13] +Output [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#66] +Input [7]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_sk#65, cp_catalog_page_id#66] + +(46) HashAggregate [codegen id : 13] +Input [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#66] +Keys [1]: [cp_catalog_page_id#66] +Functions [4]: [partial_sum(UnscaledValue(sales_price#49)), partial_sum(UnscaledValue(return_amt#51)), partial_sum(UnscaledValue(profit#50)), partial_sum(UnscaledValue(net_loss#52))] +Aggregate Attributes [4]: [sum#67, sum#68, sum#69, sum#70] +Results [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] + +(47) Exchange +Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] +Arguments: hashpartitioning(cp_catalog_page_id#66, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(48) HashAggregate [codegen id : 14] +Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] +Keys [1]: [cp_catalog_page_id#66] +Functions [4]: [sum(UnscaledValue(sales_price#49)), sum(UnscaledValue(return_amt#51)), sum(UnscaledValue(profit#50)), sum(UnscaledValue(net_loss#52))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#49))#75, sum(UnscaledValue(return_amt#51))#76, sum(UnscaledValue(profit#50))#77, sum(UnscaledValue(net_loss#52))#78] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#49))#75,17,2) AS sales#79, MakeDecimal(sum(UnscaledValue(return_amt#51))#76,17,2) AS returns#80, (MakeDecimal(sum(UnscaledValue(profit#50))#77,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#52))#78,17,2)) AS profit#81, catalog channel AS channel#82, concat(catalog_page, cp_catalog_page_id#66) AS id#83] + +(49) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#85), dynamicpruningexpression(ws_sold_date_sk#85 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#87), dynamicpruningexpression(ws_sold_date_sk#87 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(44) ColumnarToRow [codegen id : 13] -Input [4]: [ws_web_site_sk#82, ws_ext_sales_price#83, ws_net_profit#84, ws_sold_date_sk#85] +(50) ColumnarToRow [codegen id : 16] +Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] + +(51) Filter [codegen id : 16] +Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] +Condition : isnotnull(ws_web_site_sk#84) -(45) Filter [codegen id : 13] -Input [4]: [ws_web_site_sk#82, ws_ext_sales_price#83, ws_net_profit#84, ws_sold_date_sk#85] -Condition : isnotnull(ws_web_site_sk#82) +(52) Project [codegen id : 16] +Output [6]: [ws_web_site_sk#84 AS wsr_web_site_sk#88, ws_sold_date_sk#87 AS date_sk#89, ws_ext_sales_price#85 AS sales_price#90, ws_net_profit#86 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] +Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] -(46) Project [codegen id : 13] -Output [6]: [ws_web_site_sk#82 AS wsr_web_site_sk#86, ws_sold_date_sk#85 AS date_sk#87, ws_ext_sales_price#83 AS sales_price#88, ws_net_profit#84 AS profit#89, 0.00 AS return_amt#90, 0.00 AS net_loss#91] -Input [4]: [ws_web_site_sk#82, ws_ext_sales_price#83, ws_net_profit#84, ws_sold_date_sk#85] +(53) ReusedExchange [Reuses operator id: 91] +Output [1]: [d_date_sk#94] -(47) Scan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96] +(54) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [date_sk#89] +Right keys [1]: [d_date_sk#94] +Join type: Inner +Join condition: None + +(55) Project [codegen id : 16] +Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] +Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#94] + +(56) Scan parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#95, wr_order_number#96, wr_return_amt#97, wr_net_loss#98, wr_returned_date_sk#99] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#96), dynamicpruningexpression(wr_returned_date_sk#96 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(wr_returned_date_sk#99), dynamicpruningexpression(wr_returned_date_sk#99 IN dynamicpruning#5)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 14] -Input [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96] +(57) ColumnarToRow [codegen id : 17] +Input [5]: [wr_item_sk#95, wr_order_number#96, wr_return_amt#97, wr_net_loss#98, wr_returned_date_sk#99] -(49) Exchange -Input [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96] -Arguments: hashpartitioning(wr_item_sk#92, wr_order_number#93, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(58) Exchange +Input [5]: [wr_item_sk#95, wr_order_number#96, wr_return_amt#97, wr_net_loss#98, wr_returned_date_sk#99] +Arguments: hashpartitioning(wr_item_sk#95, wr_order_number#96, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(50) Sort [codegen id : 15] -Input [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96] -Arguments: [wr_item_sk#92 ASC NULLS FIRST, wr_order_number#93 ASC NULLS FIRST], false, 0 +(59) Sort [codegen id : 18] +Input [5]: [wr_item_sk#95, wr_order_number#96, wr_return_amt#97, wr_net_loss#98, wr_returned_date_sk#99] +Arguments: [wr_item_sk#95 ASC NULLS FIRST, wr_order_number#96 ASC NULLS FIRST], false, 0 -(51) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] +(60) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102, ws_sold_date_sk#103] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(52) ColumnarToRow [codegen id : 16] -Input [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] +(61) ColumnarToRow [codegen id : 19] +Input [4]: [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102, ws_sold_date_sk#103] -(53) Filter [codegen id : 16] -Input [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] -Condition : ((isnotnull(ws_item_sk#97) AND isnotnull(ws_order_number#99)) AND isnotnull(ws_web_site_sk#98)) +(62) Filter [codegen id : 19] +Input [4]: [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102, ws_sold_date_sk#103] +Condition : ((isnotnull(ws_item_sk#100) AND isnotnull(ws_order_number#102)) AND isnotnull(ws_web_site_sk#101)) -(54) Project [codegen id : 16] -Output [3]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99] -Input [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] +(63) Project [codegen id : 19] +Output [3]: [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102] +Input [4]: [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102, ws_sold_date_sk#103] -(55) Exchange -Input [3]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99] -Arguments: hashpartitioning(ws_item_sk#97, ws_order_number#99, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(64) Exchange +Input [3]: [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102] +Arguments: hashpartitioning(ws_item_sk#100, ws_order_number#102, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(56) Sort [codegen id : 17] -Input [3]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99] -Arguments: [ws_item_sk#97 ASC NULLS FIRST, ws_order_number#99 ASC NULLS FIRST], false, 0 +(65) Sort [codegen id : 20] +Input [3]: [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102] +Arguments: [ws_item_sk#100 ASC NULLS FIRST, ws_order_number#102 ASC NULLS FIRST], false, 0 -(57) SortMergeJoin [codegen id : 18] -Left keys [2]: [wr_item_sk#92, wr_order_number#93] -Right keys [2]: [ws_item_sk#97, ws_order_number#99] +(66) SortMergeJoin [codegen id : 22] +Left keys [2]: [wr_item_sk#95, wr_order_number#96] +Right keys [2]: [ws_item_sk#100, ws_order_number#102] Join type: Inner Join condition: None -(58) Project [codegen id : 18] -Output [6]: [ws_web_site_sk#98 AS wsr_web_site_sk#101, wr_returned_date_sk#96 AS date_sk#102, 0.00 AS sales_price#103, 0.00 AS profit#104, wr_return_amt#94 AS return_amt#105, wr_net_loss#95 AS net_loss#106] -Input [8]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96, ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99] - -(59) Union +(67) Project [codegen id : 22] +Output [6]: [ws_web_site_sk#101 AS wsr_web_site_sk#104, wr_returned_date_sk#99 AS date_sk#105, 0.00 AS sales_price#106, 0.00 AS profit#107, wr_return_amt#97 AS return_amt#108, wr_net_loss#98 AS net_loss#109] +Input [8]: [wr_item_sk#95, wr_order_number#96, wr_return_amt#97, wr_net_loss#98, wr_returned_date_sk#99, ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102] -(60) ReusedExchange [Reuses operator id: 82] -Output [1]: [d_date_sk#107] +(68) ReusedExchange [Reuses operator id: 91] +Output [1]: [d_date_sk#110] -(61) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [date_sk#87] -Right keys [1]: [d_date_sk#107] +(69) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [date_sk#105] +Right keys [1]: [d_date_sk#110] Join type: Inner Join condition: None -(62) Project [codegen id : 21] -Output [5]: [wsr_web_site_sk#86, sales_price#88, profit#89, return_amt#90, net_loss#91] -Input [7]: [wsr_web_site_sk#86, date_sk#87, sales_price#88, profit#89, return_amt#90, net_loss#91, d_date_sk#107] +(70) Project [codegen id : 22] +Output [5]: [wsr_web_site_sk#104, sales_price#106, profit#107, return_amt#108, net_loss#109] +Input [7]: [wsr_web_site_sk#104, date_sk#105, sales_price#106, profit#107, return_amt#108, net_loss#109, d_date_sk#110] + +(71) Union -(63) Scan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#108, web_site_id#109] +(72) Scan parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#111, web_site_id#112] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(64) ColumnarToRow [codegen id : 20] -Input [2]: [web_site_sk#108, web_site_id#109] +(73) ColumnarToRow [codegen id : 23] +Input [2]: [web_site_sk#111, web_site_id#112] -(65) Filter [codegen id : 20] -Input [2]: [web_site_sk#108, web_site_id#109] -Condition : isnotnull(web_site_sk#108) +(74) Filter [codegen id : 23] +Input [2]: [web_site_sk#111, web_site_id#112] +Condition : isnotnull(web_site_sk#111) -(66) BroadcastExchange -Input [2]: [web_site_sk#108, web_site_id#109] +(75) BroadcastExchange +Input [2]: [web_site_sk#111, web_site_id#112] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(67) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [wsr_web_site_sk#86] -Right keys [1]: [web_site_sk#108] +(76) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [wsr_web_site_sk#88] +Right keys [1]: [web_site_sk#111] Join type: Inner Join condition: None -(68) Project [codegen id : 21] -Output [5]: [sales_price#88, profit#89, return_amt#90, net_loss#91, web_site_id#109] -Input [7]: [wsr_web_site_sk#86, sales_price#88, profit#89, return_amt#90, net_loss#91, web_site_sk#108, web_site_id#109] - -(69) HashAggregate [codegen id : 21] -Input [5]: [sales_price#88, profit#89, return_amt#90, net_loss#91, web_site_id#109] -Keys [1]: [web_site_id#109] -Functions [4]: [partial_sum(UnscaledValue(sales_price#88)), partial_sum(UnscaledValue(return_amt#90)), partial_sum(UnscaledValue(profit#89)), partial_sum(UnscaledValue(net_loss#91))] -Aggregate Attributes [4]: [sum#110, sum#111, sum#112, sum#113] -Results [5]: [web_site_id#109, sum#114, sum#115, sum#116, sum#117] - -(70) Exchange -Input [5]: [web_site_id#109, sum#114, sum#115, sum#116, sum#117] -Arguments: hashpartitioning(web_site_id#109, 5), ENSURE_REQUIREMENTS, [plan_id=8] - -(71) HashAggregate [codegen id : 22] -Input [5]: [web_site_id#109, sum#114, sum#115, sum#116, sum#117] -Keys [1]: [web_site_id#109] -Functions [4]: [sum(UnscaledValue(sales_price#88)), sum(UnscaledValue(return_amt#90)), sum(UnscaledValue(profit#89)), sum(UnscaledValue(net_loss#91))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#88))#118, sum(UnscaledValue(return_amt#90))#119, sum(UnscaledValue(profit#89))#120, sum(UnscaledValue(net_loss#91))#121] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#88))#118,17,2) AS sales#122, MakeDecimal(sum(UnscaledValue(return_amt#90))#119,17,2) AS returns#123, (MakeDecimal(sum(UnscaledValue(profit#89))#120,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#91))#121,17,2)) AS profit#124, web channel AS channel#125, concat(web_site, web_site_id#109) AS id#126] - -(72) Union - -(73) Expand [codegen id : 23] -Input [5]: [sales#37, returns#38, profit#39, channel#40, id#41] -Arguments: [[sales#37, returns#38, profit#39, channel#40, id#41, 0], [sales#37, returns#38, profit#39, channel#40, null, 1], [sales#37, returns#38, profit#39, null, null, 3]], [sales#37, returns#38, profit#39, channel#127, id#128, spark_grouping_id#129] - -(74) HashAggregate [codegen id : 23] -Input [6]: [sales#37, returns#38, profit#39, channel#127, id#128, spark_grouping_id#129] -Keys [3]: [channel#127, id#128, spark_grouping_id#129] -Functions [3]: [partial_sum(sales#37), partial_sum(returns#38), partial_sum(profit#39)] -Aggregate Attributes [6]: [sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135] -Results [9]: [channel#127, id#128, spark_grouping_id#129, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] - -(75) Exchange -Input [9]: [channel#127, id#128, spark_grouping_id#129, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] -Arguments: hashpartitioning(channel#127, id#128, spark_grouping_id#129, 5), ENSURE_REQUIREMENTS, [plan_id=9] - -(76) HashAggregate [codegen id : 24] -Input [9]: [channel#127, id#128, spark_grouping_id#129, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] -Keys [3]: [channel#127, id#128, spark_grouping_id#129] -Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] -Aggregate Attributes [3]: [sum(sales#37)#142, sum(returns#38)#143, sum(profit#39)#144] -Results [5]: [channel#127, id#128, sum(sales#37)#142 AS sales#145, sum(returns#38)#143 AS returns#146, sum(profit#39)#144 AS profit#147] - -(77) TakeOrderedAndProject -Input [5]: [channel#127, id#128, sales#145, returns#146, profit#147] -Arguments: 100, [channel#127 ASC NULLS FIRST, id#128 ASC NULLS FIRST], [channel#127, id#128, sales#145, returns#146, profit#147] +(77) Project [codegen id : 24] +Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#112] +Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#111, web_site_id#112] + +(78) HashAggregate [codegen id : 24] +Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#112] +Keys [1]: [web_site_id#112] +Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] +Aggregate Attributes [4]: [sum#113, sum#114, sum#115, sum#116] +Results [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] + +(79) Exchange +Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] +Arguments: hashpartitioning(web_site_id#112, 5), ENSURE_REQUIREMENTS, [plan_id=8] + +(80) HashAggregate [codegen id : 25] +Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] +Keys [1]: [web_site_id#112] +Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#90))#121, sum(UnscaledValue(return_amt#92))#122, sum(UnscaledValue(profit#91))#123, sum(UnscaledValue(net_loss#93))#124] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#90))#121,17,2) AS sales#125, MakeDecimal(sum(UnscaledValue(return_amt#92))#122,17,2) AS returns#126, (MakeDecimal(sum(UnscaledValue(profit#91))#123,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#93))#124,17,2)) AS profit#127, web channel AS channel#128, concat(web_site, web_site_id#112) AS id#129] + +(81) Union + +(82) Expand [codegen id : 26] +Input [5]: [sales#38, returns#39, profit#40, channel#41, id#42] +Arguments: [[sales#38, returns#39, profit#40, channel#41, id#42, 0], [sales#38, returns#39, profit#40, channel#41, null, 1], [sales#38, returns#39, profit#40, null, null, 3]], [sales#38, returns#39, profit#40, channel#130, id#131, spark_grouping_id#132] + +(83) HashAggregate [codegen id : 26] +Input [6]: [sales#38, returns#39, profit#40, channel#130, id#131, spark_grouping_id#132] +Keys [3]: [channel#130, id#131, spark_grouping_id#132] +Functions [3]: [partial_sum(sales#38), partial_sum(returns#39), partial_sum(profit#40)] +Aggregate Attributes [6]: [sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] +Results [9]: [channel#130, id#131, spark_grouping_id#132, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] + +(84) Exchange +Input [9]: [channel#130, id#131, spark_grouping_id#132, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] +Arguments: hashpartitioning(channel#130, id#131, spark_grouping_id#132, 5), ENSURE_REQUIREMENTS, [plan_id=9] + +(85) HashAggregate [codegen id : 27] +Input [9]: [channel#130, id#131, spark_grouping_id#132, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] +Keys [3]: [channel#130, id#131, spark_grouping_id#132] +Functions [3]: [sum(sales#38), sum(returns#39), sum(profit#40)] +Aggregate Attributes [3]: [sum(sales#38)#145, sum(returns#39)#146, sum(profit#40)#147] +Results [5]: [channel#130, id#131, sum(sales#38)#145 AS sales#148, sum(returns#39)#146 AS returns#149, sum(profit#40)#147 AS profit#150] + +(86) TakeOrderedAndProject +Input [5]: [channel#130, id#131, sales#148, returns#149, profit#150] +Arguments: 100, [channel#130 ASC NULLS FIRST, id#131 ASC NULLS FIRST], [channel#130, id#131, sales#148, returns#149, profit#150] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (82) -+- * Project (81) - +- * Filter (80) - +- * ColumnarToRow (79) - +- Scan parquet spark_catalog.default.date_dim (78) +BroadcastExchange (91) ++- * Project (90) + +- * Filter (89) + +- * ColumnarToRow (88) + +- Scan parquet spark_catalog.default.date_dim (87) -(78) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#148] +(87) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#12, d_date#151] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] ReadSchema: struct -(79) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#22, d_date#148] +(88) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#12, d_date#151] -(80) Filter [codegen id : 1] -Input [2]: [d_date_sk#22, d_date#148] -Condition : (((isnotnull(d_date#148) AND (d_date#148 >= 2000-08-23)) AND (d_date#148 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) +(89) Filter [codegen id : 1] +Input [2]: [d_date_sk#12, d_date#151] +Condition : (((isnotnull(d_date#151) AND (d_date#151 >= 2000-08-23)) AND (d_date#151 <= 2000-09-06)) AND isnotnull(d_date_sk#12)) -(81) Project [codegen id : 1] -Output [1]: [d_date_sk#22] -Input [2]: [d_date_sk#22, d_date#148] +(90) Project [codegen id : 1] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_date#151] -(82) BroadcastExchange -Input [1]: [d_date_sk#22] +(91) BroadcastExchange +Input [1]: [d_date_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] -Subquery:2 Hosting operator id = 5 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 8 Hosting Expression = sr_returned_date_sk#16 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 25 Hosting Expression = cs_sold_date_sk#46 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#55 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 32 Hosting Expression = cr_returned_date_sk#57 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#85 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 49 Hosting Expression = ws_sold_date_sk#87 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 47 Hosting Expression = wr_returned_date_sk#96 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 56 Hosting Expression = wr_returned_date_sk#99 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt index fbf4ddf7b10cf..e532c10a654fb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt @@ -1,26 +1,26 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (24) + WholeStageCodegen (27) HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (23) + WholeStageCodegen (26) HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] Expand [sales,returns,profit,channel,id] InputAdapter Union - WholeStageCodegen (6) + WholeStageCodegen (7) HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] InputAdapter Exchange [s_store_id] #2 - WholeStageCodegen (5) + WholeStageCodegen (6) HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] Project [sales_price,profit,return_amt,net_loss,s_store_id] BroadcastHashJoin [store_sk,s_store_sk] - Project [store_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (1) + InputAdapter + Union + WholeStageCodegen (2) + Project [store_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] Project [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] Filter [ss_store_sk] ColumnarToRow @@ -34,105 +34,117 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - WholeStageCodegen (2) + InputAdapter + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (4) + Project [store_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] Project [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] Filter [sr_store_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) + WholeStageCodegen (5) Filter [s_store_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store [s_store_sk,s_store_id] - WholeStageCodegen (12) + WholeStageCodegen (14) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] InputAdapter Exchange [cp_catalog_page_id] #5 - WholeStageCodegen (11) + WholeStageCodegen (13) HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] BroadcastHashJoin [page_sk,cp_catalog_page_sk] - Project [page_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (7) + InputAdapter + Union + WholeStageCodegen (9) + Project [page_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] Project [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] Filter [cs_catalog_page_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (8) + InputAdapter + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (11) + Project [page_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] Project [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] Filter [cr_catalog_page_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #6 - WholeStageCodegen (10) + WholeStageCodegen (12) Filter [cp_catalog_page_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (22) + WholeStageCodegen (25) HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] InputAdapter Exchange [web_site_id] #7 - WholeStageCodegen (21) + WholeStageCodegen (24) HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] Project [sales_price,profit,return_amt,net_loss,web_site_id] BroadcastHashJoin [wsr_web_site_sk,web_site_sk] - Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (13) + InputAdapter + Union + WholeStageCodegen (16) + Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] Project [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] Filter [ws_web_site_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (18) + InputAdapter + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (22) + Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] Project [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] SortMergeJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] InputAdapter - WholeStageCodegen (15) + WholeStageCodegen (18) Sort [wr_item_sk,wr_order_number] InputAdapter Exchange [wr_item_sk,wr_order_number] #8 - WholeStageCodegen (14) + WholeStageCodegen (17) ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter - WholeStageCodegen (17) + WholeStageCodegen (20) Sort [ws_item_sk,ws_order_number] InputAdapter Exchange [ws_item_sk,ws_order_number] #9 - WholeStageCodegen (16) + WholeStageCodegen (19) Project [ws_item_sk,ws_web_site_sk,ws_order_number] Filter [ws_item_sk,ws_order_number,ws_web_site_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #10 - WholeStageCodegen (20) + WholeStageCodegen (23) Filter [web_site_sk] ColumnarToRow InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt index 313959456c809..e3a9b35896559 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt @@ -1,78 +1,87 @@ == Physical Plan == -TakeOrderedAndProject (74) -+- * HashAggregate (73) - +- Exchange (72) - +- * HashAggregate (71) - +- * Expand (70) - +- Union (69) - :- * HashAggregate (21) - : +- Exchange (20) - : +- * HashAggregate (19) - : +- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- Union (9) - : : : :- * Project (4) - : : : : +- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- * Project (8) - : : : +- * Filter (7) - : : : +- * ColumnarToRow (6) - : : : +- Scan parquet spark_catalog.default.store_returns (5) - : : +- ReusedExchange (10) - : +- BroadcastExchange (16) - : +- * Filter (15) - : +- * ColumnarToRow (14) - : +- Scan parquet spark_catalog.default.store (13) - :- * HashAggregate (42) - : +- Exchange (41) - : +- * HashAggregate (40) - : +- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (33) - : : +- * BroadcastHashJoin Inner BuildRight (32) - : : :- Union (30) - : : : :- * Project (25) - : : : : +- * Filter (24) - : : : : +- * ColumnarToRow (23) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (22) - : : : +- * Project (29) - : : : +- * Filter (28) - : : : +- * ColumnarToRow (27) - : : : +- Scan parquet spark_catalog.default.catalog_returns (26) - : : +- ReusedExchange (31) - : +- BroadcastExchange (37) - : +- * Filter (36) - : +- * ColumnarToRow (35) - : +- Scan parquet spark_catalog.default.catalog_page (34) - +- * HashAggregate (68) - +- Exchange (67) - +- * HashAggregate (66) - +- * Project (65) - +- * BroadcastHashJoin Inner BuildRight (64) - :- * Project (59) - : +- * BroadcastHashJoin Inner BuildRight (58) - : :- Union (56) - : : :- * Project (46) - : : : +- * Filter (45) - : : : +- * ColumnarToRow (44) - : : : +- Scan parquet spark_catalog.default.web_sales (43) - : : +- * Project (55) - : : +- * BroadcastHashJoin Inner BuildLeft (54) - : : :- BroadcastExchange (49) - : : : +- * ColumnarToRow (48) - : : : +- Scan parquet spark_catalog.default.web_returns (47) - : : +- * Project (53) - : : +- * Filter (52) - : : +- * ColumnarToRow (51) - : : +- Scan parquet spark_catalog.default.web_sales (50) - : +- ReusedExchange (57) - +- BroadcastExchange (63) - +- * Filter (62) - +- * ColumnarToRow (61) - +- Scan parquet spark_catalog.default.web_site (60) +TakeOrderedAndProject (83) ++- * HashAggregate (82) + +- Exchange (81) + +- * HashAggregate (80) + +- * Expand (79) + +- Union (78) + :- * HashAggregate (24) + : +- Exchange (23) + : +- * HashAggregate (22) + : +- * Project (21) + : +- * BroadcastHashJoin Inner BuildRight (20) + : :- Union (15) + : : :- * Project (7) + : : : +- * BroadcastHashJoin Inner BuildRight (6) + : : : :- * Project (4) + : : : : +- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (5) + : : +- * Project (14) + : : +- * BroadcastHashJoin Inner BuildRight (13) + : : :- * Project (11) + : : : +- * Filter (10) + : : : +- * ColumnarToRow (9) + : : : +- Scan parquet spark_catalog.default.store_returns (8) + : : +- ReusedExchange (12) + : +- BroadcastExchange (19) + : +- * Filter (18) + : +- * ColumnarToRow (17) + : +- Scan parquet spark_catalog.default.store (16) + :- * HashAggregate (48) + : +- Exchange (47) + : +- * HashAggregate (46) + : +- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) + : :- Union (39) + : : :- * Project (31) + : : : +- * BroadcastHashJoin Inner BuildRight (30) + : : : :- * Project (28) + : : : : +- * Filter (27) + : : : : +- * ColumnarToRow (26) + : : : : +- Scan parquet spark_catalog.default.catalog_sales (25) + : : : +- ReusedExchange (29) + : : +- * Project (38) + : : +- * BroadcastHashJoin Inner BuildRight (37) + : : :- * Project (35) + : : : +- * Filter (34) + : : : +- * ColumnarToRow (33) + : : : +- Scan parquet spark_catalog.default.catalog_returns (32) + : : +- ReusedExchange (36) + : +- BroadcastExchange (43) + : +- * Filter (42) + : +- * ColumnarToRow (41) + : +- Scan parquet spark_catalog.default.catalog_page (40) + +- * HashAggregate (77) + +- Exchange (76) + +- * HashAggregate (75) + +- * Project (74) + +- * BroadcastHashJoin Inner BuildRight (73) + :- Union (68) + : :- * Project (55) + : : +- * BroadcastHashJoin Inner BuildRight (54) + : : :- * Project (52) + : : : +- * Filter (51) + : : : +- * ColumnarToRow (50) + : : : +- Scan parquet spark_catalog.default.web_sales (49) + : : +- ReusedExchange (53) + : +- * Project (67) + : +- * BroadcastHashJoin Inner BuildRight (66) + : :- * Project (64) + : : +- * BroadcastHashJoin Inner BuildLeft (63) + : : :- BroadcastExchange (58) + : : : +- * ColumnarToRow (57) + : : : +- Scan parquet spark_catalog.default.web_returns (56) + : : +- * Project (62) + : : +- * Filter (61) + : : +- * ColumnarToRow (60) + : : +- Scan parquet spark_catalog.default.web_sales (59) + : +- ReusedExchange (65) + +- BroadcastExchange (72) + +- * Filter (71) + +- * ColumnarToRow (70) + +- Scan parquet spark_catalog.default.web_site (69) (1) Scan parquet spark_catalog.default.store_sales @@ -83,386 +92,425 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 1] +(2) ColumnarToRow [codegen id : 2] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 1] +(3) Filter [codegen id : 2] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] Condition : isnotnull(ss_store_sk#1) -(4) Project [codegen id : 1] +(4) Project [codegen id : 2] Output [6]: [ss_store_sk#1 AS store_sk#6, ss_sold_date_sk#4 AS date_sk#7, ss_ext_sales_price#2 AS sales_price#8, ss_net_profit#3 AS profit#9, 0.00 AS return_amt#10, 0.00 AS net_loss#11] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -(5) Scan parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] +(5) ReusedExchange [Reuses operator id: 88] +Output [1]: [d_date_sk#12] + +(6) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [date_sk#7] +Right keys [1]: [d_date_sk#12] +Join type: Inner +Join condition: None + +(7) Project [codegen id : 2] +Output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] +Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#12] + +(8) Scan parquet spark_catalog.default.store_returns +Output [4]: [sr_store_sk#13, sr_return_amt#14, sr_net_loss#15, sr_returned_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#15), dynamicpruningexpression(sr_returned_date_sk#15 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(sr_returned_date_sk#16), dynamicpruningexpression(sr_returned_date_sk#16 IN dynamicpruning#5)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(6) ColumnarToRow [codegen id : 2] -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] +(9) ColumnarToRow [codegen id : 4] +Input [4]: [sr_store_sk#13, sr_return_amt#14, sr_net_loss#15, sr_returned_date_sk#16] -(7) Filter [codegen id : 2] -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Condition : isnotnull(sr_store_sk#12) +(10) Filter [codegen id : 4] +Input [4]: [sr_store_sk#13, sr_return_amt#14, sr_net_loss#15, sr_returned_date_sk#16] +Condition : isnotnull(sr_store_sk#13) -(8) Project [codegen id : 2] -Output [6]: [sr_store_sk#12 AS store_sk#16, sr_returned_date_sk#15 AS date_sk#17, 0.00 AS sales_price#18, 0.00 AS profit#19, sr_return_amt#13 AS return_amt#20, sr_net_loss#14 AS net_loss#21] -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] +(11) Project [codegen id : 4] +Output [6]: [sr_store_sk#13 AS store_sk#17, sr_returned_date_sk#16 AS date_sk#18, 0.00 AS sales_price#19, 0.00 AS profit#20, sr_return_amt#14 AS return_amt#21, sr_net_loss#15 AS net_loss#22] +Input [4]: [sr_store_sk#13, sr_return_amt#14, sr_net_loss#15, sr_returned_date_sk#16] -(9) Union +(12) ReusedExchange [Reuses operator id: 88] +Output [1]: [d_date_sk#23] -(10) ReusedExchange [Reuses operator id: 79] -Output [1]: [d_date_sk#22] - -(11) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [date_sk#7] -Right keys [1]: [d_date_sk#22] +(13) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [date_sk#18] +Right keys [1]: [d_date_sk#23] Join type: Inner Join condition: None -(12) Project [codegen id : 5] -Output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] -Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] +(14) Project [codegen id : 4] +Output [5]: [store_sk#17, sales_price#19, profit#20, return_amt#21, net_loss#22] +Input [7]: [store_sk#17, date_sk#18, sales_price#19, profit#20, return_amt#21, net_loss#22, d_date_sk#23] -(13) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#23, s_store_id#24] +(15) Union + +(16) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#24, s_store_id#25] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(14) ColumnarToRow [codegen id : 4] -Input [2]: [s_store_sk#23, s_store_id#24] +(17) ColumnarToRow [codegen id : 5] +Input [2]: [s_store_sk#24, s_store_id#25] -(15) Filter [codegen id : 4] -Input [2]: [s_store_sk#23, s_store_id#24] -Condition : isnotnull(s_store_sk#23) +(18) Filter [codegen id : 5] +Input [2]: [s_store_sk#24, s_store_id#25] +Condition : isnotnull(s_store_sk#24) -(16) BroadcastExchange -Input [2]: [s_store_sk#23, s_store_id#24] +(19) BroadcastExchange +Input [2]: [s_store_sk#24, s_store_id#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(17) BroadcastHashJoin [codegen id : 5] +(20) BroadcastHashJoin [codegen id : 6] Left keys [1]: [store_sk#6] -Right keys [1]: [s_store_sk#23] +Right keys [1]: [s_store_sk#24] Join type: Inner Join condition: None -(18) Project [codegen id : 5] -Output [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#24] -Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#23, s_store_id#24] +(21) Project [codegen id : 6] +Output [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] +Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#24, s_store_id#25] -(19) HashAggregate [codegen id : 5] -Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#24] -Keys [1]: [s_store_id#24] +(22) HashAggregate [codegen id : 6] +Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] +Keys [1]: [s_store_id#25] Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum#25, sum#26, sum#27, sum#28] -Results [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] +Aggregate Attributes [4]: [sum#26, sum#27, sum#28, sum#29] +Results [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] -(20) Exchange -Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] -Arguments: hashpartitioning(s_store_id#24, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(23) Exchange +Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] +Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(21) HashAggregate [codegen id : 6] -Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] -Keys [1]: [s_store_id#24] +(24) HashAggregate [codegen id : 7] +Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] +Keys [1]: [s_store_id#25] Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#33, sum(UnscaledValue(return_amt#10))#34, sum(UnscaledValue(profit#9))#35, sum(UnscaledValue(net_loss#11))#36] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#8))#33,17,2) AS sales#37, MakeDecimal(sum(UnscaledValue(return_amt#10))#34,17,2) AS returns#38, (MakeDecimal(sum(UnscaledValue(profit#9))#35,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#36,17,2)) AS profit#39, store channel AS channel#40, concat(store, s_store_id#24) AS id#41] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#34, sum(UnscaledValue(return_amt#10))#35, sum(UnscaledValue(profit#9))#36, sum(UnscaledValue(net_loss#11))#37] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#8))#34,17,2) AS sales#38, MakeDecimal(sum(UnscaledValue(return_amt#10))#35,17,2) AS returns#39, (MakeDecimal(sum(UnscaledValue(profit#9))#36,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#37,17,2)) AS profit#40, store channel AS channel#41, concat(store, s_store_id#25) AS id#42] -(22) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +(25) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#45), dynamicpruningexpression(cs_sold_date_sk#45 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#46), dynamicpruningexpression(cs_sold_date_sk#46 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct -(23) ColumnarToRow [codegen id : 7] -Input [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +(26) ColumnarToRow [codegen id : 9] +Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] + +(27) Filter [codegen id : 9] +Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] +Condition : isnotnull(cs_catalog_page_sk#43) -(24) Filter [codegen id : 7] -Input [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Condition : isnotnull(cs_catalog_page_sk#42) +(28) Project [codegen id : 9] +Output [6]: [cs_catalog_page_sk#43 AS page_sk#47, cs_sold_date_sk#46 AS date_sk#48, cs_ext_sales_price#44 AS sales_price#49, cs_net_profit#45 AS profit#50, 0.00 AS return_amt#51, 0.00 AS net_loss#52] +Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] -(25) Project [codegen id : 7] -Output [6]: [cs_catalog_page_sk#42 AS page_sk#46, cs_sold_date_sk#45 AS date_sk#47, cs_ext_sales_price#43 AS sales_price#48, cs_net_profit#44 AS profit#49, 0.00 AS return_amt#50, 0.00 AS net_loss#51] -Input [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +(29) ReusedExchange [Reuses operator id: 88] +Output [1]: [d_date_sk#53] -(26) Scan parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#52, cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] +(30) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [date_sk#48] +Right keys [1]: [d_date_sk#53] +Join type: Inner +Join condition: None + +(31) Project [codegen id : 9] +Output [5]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52] +Input [7]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52, d_date_sk#53] + +(32) Scan parquet spark_catalog.default.catalog_returns +Output [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#55), dynamicpruningexpression(cr_returned_date_sk#55 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cr_returned_date_sk#57), dynamicpruningexpression(cr_returned_date_sk#57 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct -(27) ColumnarToRow [codegen id : 8] -Input [4]: [cr_catalog_page_sk#52, cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] +(33) ColumnarToRow [codegen id : 11] +Input [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] -(28) Filter [codegen id : 8] -Input [4]: [cr_catalog_page_sk#52, cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] -Condition : isnotnull(cr_catalog_page_sk#52) +(34) Filter [codegen id : 11] +Input [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] +Condition : isnotnull(cr_catalog_page_sk#54) -(29) Project [codegen id : 8] -Output [6]: [cr_catalog_page_sk#52 AS page_sk#56, cr_returned_date_sk#55 AS date_sk#57, 0.00 AS sales_price#58, 0.00 AS profit#59, cr_return_amount#53 AS return_amt#60, cr_net_loss#54 AS net_loss#61] -Input [4]: [cr_catalog_page_sk#52, cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] +(35) Project [codegen id : 11] +Output [6]: [cr_catalog_page_sk#54 AS page_sk#58, cr_returned_date_sk#57 AS date_sk#59, 0.00 AS sales_price#60, 0.00 AS profit#61, cr_return_amount#55 AS return_amt#62, cr_net_loss#56 AS net_loss#63] +Input [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] -(30) Union +(36) ReusedExchange [Reuses operator id: 88] +Output [1]: [d_date_sk#64] -(31) ReusedExchange [Reuses operator id: 79] -Output [1]: [d_date_sk#62] - -(32) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [date_sk#47] -Right keys [1]: [d_date_sk#62] +(37) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [date_sk#59] +Right keys [1]: [d_date_sk#64] Join type: Inner Join condition: None -(33) Project [codegen id : 11] -Output [5]: [page_sk#46, sales_price#48, profit#49, return_amt#50, net_loss#51] -Input [7]: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51, d_date_sk#62] +(38) Project [codegen id : 11] +Output [5]: [page_sk#58, sales_price#60, profit#61, return_amt#62, net_loss#63] +Input [7]: [page_sk#58, date_sk#59, sales_price#60, profit#61, return_amt#62, net_loss#63, d_date_sk#64] + +(39) Union -(34) Scan parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#63, cp_catalog_page_id#64] +(40) Scan parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(35) ColumnarToRow [codegen id : 10] -Input [2]: [cp_catalog_page_sk#63, cp_catalog_page_id#64] +(41) ColumnarToRow [codegen id : 12] +Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] -(36) Filter [codegen id : 10] -Input [2]: [cp_catalog_page_sk#63, cp_catalog_page_id#64] -Condition : isnotnull(cp_catalog_page_sk#63) +(42) Filter [codegen id : 12] +Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] +Condition : isnotnull(cp_catalog_page_sk#65) -(37) BroadcastExchange -Input [2]: [cp_catalog_page_sk#63, cp_catalog_page_id#64] +(43) BroadcastExchange +Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(38) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [page_sk#46] -Right keys [1]: [cp_catalog_page_sk#63] +(44) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [page_sk#47] +Right keys [1]: [cp_catalog_page_sk#65] Join type: Inner Join condition: None -(39) Project [codegen id : 11] -Output [5]: [sales_price#48, profit#49, return_amt#50, net_loss#51, cp_catalog_page_id#64] -Input [7]: [page_sk#46, sales_price#48, profit#49, return_amt#50, net_loss#51, cp_catalog_page_sk#63, cp_catalog_page_id#64] - -(40) HashAggregate [codegen id : 11] -Input [5]: [sales_price#48, profit#49, return_amt#50, net_loss#51, cp_catalog_page_id#64] -Keys [1]: [cp_catalog_page_id#64] -Functions [4]: [partial_sum(UnscaledValue(sales_price#48)), partial_sum(UnscaledValue(return_amt#50)), partial_sum(UnscaledValue(profit#49)), partial_sum(UnscaledValue(net_loss#51))] -Aggregate Attributes [4]: [sum#65, sum#66, sum#67, sum#68] -Results [5]: [cp_catalog_page_id#64, sum#69, sum#70, sum#71, sum#72] - -(41) Exchange -Input [5]: [cp_catalog_page_id#64, sum#69, sum#70, sum#71, sum#72] -Arguments: hashpartitioning(cp_catalog_page_id#64, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(42) HashAggregate [codegen id : 12] -Input [5]: [cp_catalog_page_id#64, sum#69, sum#70, sum#71, sum#72] -Keys [1]: [cp_catalog_page_id#64] -Functions [4]: [sum(UnscaledValue(sales_price#48)), sum(UnscaledValue(return_amt#50)), sum(UnscaledValue(profit#49)), sum(UnscaledValue(net_loss#51))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#48))#73, sum(UnscaledValue(return_amt#50))#74, sum(UnscaledValue(profit#49))#75, sum(UnscaledValue(net_loss#51))#76] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#48))#73,17,2) AS sales#77, MakeDecimal(sum(UnscaledValue(return_amt#50))#74,17,2) AS returns#78, (MakeDecimal(sum(UnscaledValue(profit#49))#75,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#51))#76,17,2)) AS profit#79, catalog channel AS channel#80, concat(catalog_page, cp_catalog_page_id#64) AS id#81] - -(43) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#82, ws_ext_sales_price#83, ws_net_profit#84, ws_sold_date_sk#85] +(45) Project [codegen id : 13] +Output [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#66] +Input [7]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_sk#65, cp_catalog_page_id#66] + +(46) HashAggregate [codegen id : 13] +Input [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#66] +Keys [1]: [cp_catalog_page_id#66] +Functions [4]: [partial_sum(UnscaledValue(sales_price#49)), partial_sum(UnscaledValue(return_amt#51)), partial_sum(UnscaledValue(profit#50)), partial_sum(UnscaledValue(net_loss#52))] +Aggregate Attributes [4]: [sum#67, sum#68, sum#69, sum#70] +Results [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] + +(47) Exchange +Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] +Arguments: hashpartitioning(cp_catalog_page_id#66, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(48) HashAggregate [codegen id : 14] +Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] +Keys [1]: [cp_catalog_page_id#66] +Functions [4]: [sum(UnscaledValue(sales_price#49)), sum(UnscaledValue(return_amt#51)), sum(UnscaledValue(profit#50)), sum(UnscaledValue(net_loss#52))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#49))#75, sum(UnscaledValue(return_amt#51))#76, sum(UnscaledValue(profit#50))#77, sum(UnscaledValue(net_loss#52))#78] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#49))#75,17,2) AS sales#79, MakeDecimal(sum(UnscaledValue(return_amt#51))#76,17,2) AS returns#80, (MakeDecimal(sum(UnscaledValue(profit#50))#77,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#52))#78,17,2)) AS profit#81, catalog channel AS channel#82, concat(catalog_page, cp_catalog_page_id#66) AS id#83] + +(49) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#85), dynamicpruningexpression(ws_sold_date_sk#85 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#87), dynamicpruningexpression(ws_sold_date_sk#87 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(44) ColumnarToRow [codegen id : 13] -Input [4]: [ws_web_site_sk#82, ws_ext_sales_price#83, ws_net_profit#84, ws_sold_date_sk#85] +(50) ColumnarToRow [codegen id : 16] +Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] + +(51) Filter [codegen id : 16] +Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] +Condition : isnotnull(ws_web_site_sk#84) -(45) Filter [codegen id : 13] -Input [4]: [ws_web_site_sk#82, ws_ext_sales_price#83, ws_net_profit#84, ws_sold_date_sk#85] -Condition : isnotnull(ws_web_site_sk#82) +(52) Project [codegen id : 16] +Output [6]: [ws_web_site_sk#84 AS wsr_web_site_sk#88, ws_sold_date_sk#87 AS date_sk#89, ws_ext_sales_price#85 AS sales_price#90, ws_net_profit#86 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] +Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] -(46) Project [codegen id : 13] -Output [6]: [ws_web_site_sk#82 AS wsr_web_site_sk#86, ws_sold_date_sk#85 AS date_sk#87, ws_ext_sales_price#83 AS sales_price#88, ws_net_profit#84 AS profit#89, 0.00 AS return_amt#90, 0.00 AS net_loss#91] -Input [4]: [ws_web_site_sk#82, ws_ext_sales_price#83, ws_net_profit#84, ws_sold_date_sk#85] +(53) ReusedExchange [Reuses operator id: 88] +Output [1]: [d_date_sk#94] -(47) Scan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96] +(54) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [date_sk#89] +Right keys [1]: [d_date_sk#94] +Join type: Inner +Join condition: None + +(55) Project [codegen id : 16] +Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] +Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#94] + +(56) Scan parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#95, wr_order_number#96, wr_return_amt#97, wr_net_loss#98, wr_returned_date_sk#99] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#96), dynamicpruningexpression(wr_returned_date_sk#96 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(wr_returned_date_sk#99), dynamicpruningexpression(wr_returned_date_sk#99 IN dynamicpruning#5)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 14] -Input [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96] +(57) ColumnarToRow [codegen id : 17] +Input [5]: [wr_item_sk#95, wr_order_number#96, wr_return_amt#97, wr_net_loss#98, wr_returned_date_sk#99] -(49) BroadcastExchange -Input [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96] +(58) BroadcastExchange +Input [5]: [wr_item_sk#95, wr_order_number#96, wr_return_amt#97, wr_net_loss#98, wr_returned_date_sk#99] Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[1, int, true] as bigint) & 4294967295))),false), [plan_id=5] -(50) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] +(59) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102, ws_sold_date_sk#103] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(51) ColumnarToRow -Input [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] +(60) ColumnarToRow +Input [4]: [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102, ws_sold_date_sk#103] -(52) Filter -Input [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] -Condition : ((isnotnull(ws_item_sk#97) AND isnotnull(ws_order_number#99)) AND isnotnull(ws_web_site_sk#98)) +(61) Filter +Input [4]: [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102, ws_sold_date_sk#103] +Condition : ((isnotnull(ws_item_sk#100) AND isnotnull(ws_order_number#102)) AND isnotnull(ws_web_site_sk#101)) -(53) Project -Output [3]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99] -Input [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] +(62) Project +Output [3]: [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102] +Input [4]: [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102, ws_sold_date_sk#103] -(54) BroadcastHashJoin [codegen id : 15] -Left keys [2]: [wr_item_sk#92, wr_order_number#93] -Right keys [2]: [ws_item_sk#97, ws_order_number#99] +(63) BroadcastHashJoin [codegen id : 19] +Left keys [2]: [wr_item_sk#95, wr_order_number#96] +Right keys [2]: [ws_item_sk#100, ws_order_number#102] Join type: Inner Join condition: None -(55) Project [codegen id : 15] -Output [6]: [ws_web_site_sk#98 AS wsr_web_site_sk#101, wr_returned_date_sk#96 AS date_sk#102, 0.00 AS sales_price#103, 0.00 AS profit#104, wr_return_amt#94 AS return_amt#105, wr_net_loss#95 AS net_loss#106] -Input [8]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96, ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99] - -(56) Union +(64) Project [codegen id : 19] +Output [6]: [ws_web_site_sk#101 AS wsr_web_site_sk#104, wr_returned_date_sk#99 AS date_sk#105, 0.00 AS sales_price#106, 0.00 AS profit#107, wr_return_amt#97 AS return_amt#108, wr_net_loss#98 AS net_loss#109] +Input [8]: [wr_item_sk#95, wr_order_number#96, wr_return_amt#97, wr_net_loss#98, wr_returned_date_sk#99, ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102] -(57) ReusedExchange [Reuses operator id: 79] -Output [1]: [d_date_sk#107] +(65) ReusedExchange [Reuses operator id: 88] +Output [1]: [d_date_sk#110] -(58) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [date_sk#87] -Right keys [1]: [d_date_sk#107] +(66) BroadcastHashJoin [codegen id : 19] +Left keys [1]: [date_sk#105] +Right keys [1]: [d_date_sk#110] Join type: Inner Join condition: None -(59) Project [codegen id : 18] -Output [5]: [wsr_web_site_sk#86, sales_price#88, profit#89, return_amt#90, net_loss#91] -Input [7]: [wsr_web_site_sk#86, date_sk#87, sales_price#88, profit#89, return_amt#90, net_loss#91, d_date_sk#107] +(67) Project [codegen id : 19] +Output [5]: [wsr_web_site_sk#104, sales_price#106, profit#107, return_amt#108, net_loss#109] +Input [7]: [wsr_web_site_sk#104, date_sk#105, sales_price#106, profit#107, return_amt#108, net_loss#109, d_date_sk#110] + +(68) Union -(60) Scan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#108, web_site_id#109] +(69) Scan parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#111, web_site_id#112] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(61) ColumnarToRow [codegen id : 17] -Input [2]: [web_site_sk#108, web_site_id#109] +(70) ColumnarToRow [codegen id : 20] +Input [2]: [web_site_sk#111, web_site_id#112] -(62) Filter [codegen id : 17] -Input [2]: [web_site_sk#108, web_site_id#109] -Condition : isnotnull(web_site_sk#108) +(71) Filter [codegen id : 20] +Input [2]: [web_site_sk#111, web_site_id#112] +Condition : isnotnull(web_site_sk#111) -(63) BroadcastExchange -Input [2]: [web_site_sk#108, web_site_id#109] +(72) BroadcastExchange +Input [2]: [web_site_sk#111, web_site_id#112] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(64) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [wsr_web_site_sk#86] -Right keys [1]: [web_site_sk#108] +(73) BroadcastHashJoin [codegen id : 21] +Left keys [1]: [wsr_web_site_sk#88] +Right keys [1]: [web_site_sk#111] Join type: Inner Join condition: None -(65) Project [codegen id : 18] -Output [5]: [sales_price#88, profit#89, return_amt#90, net_loss#91, web_site_id#109] -Input [7]: [wsr_web_site_sk#86, sales_price#88, profit#89, return_amt#90, net_loss#91, web_site_sk#108, web_site_id#109] - -(66) HashAggregate [codegen id : 18] -Input [5]: [sales_price#88, profit#89, return_amt#90, net_loss#91, web_site_id#109] -Keys [1]: [web_site_id#109] -Functions [4]: [partial_sum(UnscaledValue(sales_price#88)), partial_sum(UnscaledValue(return_amt#90)), partial_sum(UnscaledValue(profit#89)), partial_sum(UnscaledValue(net_loss#91))] -Aggregate Attributes [4]: [sum#110, sum#111, sum#112, sum#113] -Results [5]: [web_site_id#109, sum#114, sum#115, sum#116, sum#117] - -(67) Exchange -Input [5]: [web_site_id#109, sum#114, sum#115, sum#116, sum#117] -Arguments: hashpartitioning(web_site_id#109, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(68) HashAggregate [codegen id : 19] -Input [5]: [web_site_id#109, sum#114, sum#115, sum#116, sum#117] -Keys [1]: [web_site_id#109] -Functions [4]: [sum(UnscaledValue(sales_price#88)), sum(UnscaledValue(return_amt#90)), sum(UnscaledValue(profit#89)), sum(UnscaledValue(net_loss#91))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#88))#118, sum(UnscaledValue(return_amt#90))#119, sum(UnscaledValue(profit#89))#120, sum(UnscaledValue(net_loss#91))#121] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#88))#118,17,2) AS sales#122, MakeDecimal(sum(UnscaledValue(return_amt#90))#119,17,2) AS returns#123, (MakeDecimal(sum(UnscaledValue(profit#89))#120,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#91))#121,17,2)) AS profit#124, web channel AS channel#125, concat(web_site, web_site_id#109) AS id#126] - -(69) Union - -(70) Expand [codegen id : 20] -Input [5]: [sales#37, returns#38, profit#39, channel#40, id#41] -Arguments: [[sales#37, returns#38, profit#39, channel#40, id#41, 0], [sales#37, returns#38, profit#39, channel#40, null, 1], [sales#37, returns#38, profit#39, null, null, 3]], [sales#37, returns#38, profit#39, channel#127, id#128, spark_grouping_id#129] - -(71) HashAggregate [codegen id : 20] -Input [6]: [sales#37, returns#38, profit#39, channel#127, id#128, spark_grouping_id#129] -Keys [3]: [channel#127, id#128, spark_grouping_id#129] -Functions [3]: [partial_sum(sales#37), partial_sum(returns#38), partial_sum(profit#39)] -Aggregate Attributes [6]: [sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135] -Results [9]: [channel#127, id#128, spark_grouping_id#129, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] - -(72) Exchange -Input [9]: [channel#127, id#128, spark_grouping_id#129, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] -Arguments: hashpartitioning(channel#127, id#128, spark_grouping_id#129, 5), ENSURE_REQUIREMENTS, [plan_id=8] - -(73) HashAggregate [codegen id : 21] -Input [9]: [channel#127, id#128, spark_grouping_id#129, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] -Keys [3]: [channel#127, id#128, spark_grouping_id#129] -Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] -Aggregate Attributes [3]: [sum(sales#37)#142, sum(returns#38)#143, sum(profit#39)#144] -Results [5]: [channel#127, id#128, sum(sales#37)#142 AS sales#145, sum(returns#38)#143 AS returns#146, sum(profit#39)#144 AS profit#147] - -(74) TakeOrderedAndProject -Input [5]: [channel#127, id#128, sales#145, returns#146, profit#147] -Arguments: 100, [channel#127 ASC NULLS FIRST, id#128 ASC NULLS FIRST], [channel#127, id#128, sales#145, returns#146, profit#147] +(74) Project [codegen id : 21] +Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#112] +Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#111, web_site_id#112] + +(75) HashAggregate [codegen id : 21] +Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#112] +Keys [1]: [web_site_id#112] +Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] +Aggregate Attributes [4]: [sum#113, sum#114, sum#115, sum#116] +Results [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] + +(76) Exchange +Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] +Arguments: hashpartitioning(web_site_id#112, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(77) HashAggregate [codegen id : 22] +Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] +Keys [1]: [web_site_id#112] +Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#90))#121, sum(UnscaledValue(return_amt#92))#122, sum(UnscaledValue(profit#91))#123, sum(UnscaledValue(net_loss#93))#124] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#90))#121,17,2) AS sales#125, MakeDecimal(sum(UnscaledValue(return_amt#92))#122,17,2) AS returns#126, (MakeDecimal(sum(UnscaledValue(profit#91))#123,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#93))#124,17,2)) AS profit#127, web channel AS channel#128, concat(web_site, web_site_id#112) AS id#129] + +(78) Union + +(79) Expand [codegen id : 23] +Input [5]: [sales#38, returns#39, profit#40, channel#41, id#42] +Arguments: [[sales#38, returns#39, profit#40, channel#41, id#42, 0], [sales#38, returns#39, profit#40, channel#41, null, 1], [sales#38, returns#39, profit#40, null, null, 3]], [sales#38, returns#39, profit#40, channel#130, id#131, spark_grouping_id#132] + +(80) HashAggregate [codegen id : 23] +Input [6]: [sales#38, returns#39, profit#40, channel#130, id#131, spark_grouping_id#132] +Keys [3]: [channel#130, id#131, spark_grouping_id#132] +Functions [3]: [partial_sum(sales#38), partial_sum(returns#39), partial_sum(profit#40)] +Aggregate Attributes [6]: [sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] +Results [9]: [channel#130, id#131, spark_grouping_id#132, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] + +(81) Exchange +Input [9]: [channel#130, id#131, spark_grouping_id#132, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] +Arguments: hashpartitioning(channel#130, id#131, spark_grouping_id#132, 5), ENSURE_REQUIREMENTS, [plan_id=8] + +(82) HashAggregate [codegen id : 24] +Input [9]: [channel#130, id#131, spark_grouping_id#132, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] +Keys [3]: [channel#130, id#131, spark_grouping_id#132] +Functions [3]: [sum(sales#38), sum(returns#39), sum(profit#40)] +Aggregate Attributes [3]: [sum(sales#38)#145, sum(returns#39)#146, sum(profit#40)#147] +Results [5]: [channel#130, id#131, sum(sales#38)#145 AS sales#148, sum(returns#39)#146 AS returns#149, sum(profit#40)#147 AS profit#150] + +(83) TakeOrderedAndProject +Input [5]: [channel#130, id#131, sales#148, returns#149, profit#150] +Arguments: 100, [channel#130 ASC NULLS FIRST, id#131 ASC NULLS FIRST], [channel#130, id#131, sales#148, returns#149, profit#150] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (79) -+- * Project (78) - +- * Filter (77) - +- * ColumnarToRow (76) - +- Scan parquet spark_catalog.default.date_dim (75) +BroadcastExchange (88) ++- * Project (87) + +- * Filter (86) + +- * ColumnarToRow (85) + +- Scan parquet spark_catalog.default.date_dim (84) -(75) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#148] +(84) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#12, d_date#151] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] ReadSchema: struct -(76) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#22, d_date#148] +(85) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#12, d_date#151] -(77) Filter [codegen id : 1] -Input [2]: [d_date_sk#22, d_date#148] -Condition : (((isnotnull(d_date#148) AND (d_date#148 >= 2000-08-23)) AND (d_date#148 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) +(86) Filter [codegen id : 1] +Input [2]: [d_date_sk#12, d_date#151] +Condition : (((isnotnull(d_date#151) AND (d_date#151 >= 2000-08-23)) AND (d_date#151 <= 2000-09-06)) AND isnotnull(d_date_sk#12)) -(78) Project [codegen id : 1] -Output [1]: [d_date_sk#22] -Input [2]: [d_date_sk#22, d_date#148] +(87) Project [codegen id : 1] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_date#151] -(79) BroadcastExchange -Input [1]: [d_date_sk#22] +(88) BroadcastExchange +Input [1]: [d_date_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -Subquery:2 Hosting operator id = 5 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 8 Hosting Expression = sr_returned_date_sk#16 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 25 Hosting Expression = cs_sold_date_sk#46 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#55 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 32 Hosting Expression = cr_returned_date_sk#57 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#85 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 49 Hosting Expression = ws_sold_date_sk#87 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 47 Hosting Expression = wr_returned_date_sk#96 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 56 Hosting Expression = wr_returned_date_sk#99 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt index c6cd47e3f48ef..94909af44ff17 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt @@ -1,26 +1,26 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (21) + WholeStageCodegen (24) HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (20) + WholeStageCodegen (23) HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] Expand [sales,returns,profit,channel,id] InputAdapter Union - WholeStageCodegen (6) + WholeStageCodegen (7) HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] InputAdapter Exchange [s_store_id] #2 - WholeStageCodegen (5) + WholeStageCodegen (6) HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] Project [sales_price,profit,return_amt,net_loss,s_store_id] BroadcastHashJoin [store_sk,s_store_sk] - Project [store_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (1) + InputAdapter + Union + WholeStageCodegen (2) + Project [store_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] Project [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] Filter [ss_store_sk] ColumnarToRow @@ -34,82 +34,94 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - WholeStageCodegen (2) + InputAdapter + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (4) + Project [store_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] Project [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] Filter [sr_store_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) + WholeStageCodegen (5) Filter [s_store_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store [s_store_sk,s_store_id] - WholeStageCodegen (12) + WholeStageCodegen (14) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] InputAdapter Exchange [cp_catalog_page_id] #5 - WholeStageCodegen (11) + WholeStageCodegen (13) HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] BroadcastHashJoin [page_sk,cp_catalog_page_sk] - Project [page_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (7) + InputAdapter + Union + WholeStageCodegen (9) + Project [page_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] Project [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] Filter [cs_catalog_page_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (8) + InputAdapter + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (11) + Project [page_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] Project [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] Filter [cr_catalog_page_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #6 - WholeStageCodegen (10) + WholeStageCodegen (12) Filter [cp_catalog_page_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (19) + WholeStageCodegen (22) HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] InputAdapter Exchange [web_site_id] #7 - WholeStageCodegen (18) + WholeStageCodegen (21) HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] Project [sales_price,profit,return_amt,net_loss,web_site_id] BroadcastHashJoin [wsr_web_site_sk,web_site_sk] - Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (13) + InputAdapter + Union + WholeStageCodegen (16) + Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] Project [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] Filter [ws_web_site_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (15) + InputAdapter + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (19) + Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] Project [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] InputAdapter BroadcastExchange #8 - WholeStageCodegen (14) + WholeStageCodegen (17) ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] @@ -119,11 +131,11 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #9 - WholeStageCodegen (17) + WholeStageCodegen (20) Filter [web_site_sk] ColumnarToRow InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt index 592168a16abd3..49636d2e31ef2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt @@ -1,18 +1,18 @@ == Physical Plan == -TakeOrderedAndProject (56) -+- * HashAggregate (55) - +- Exchange (54) - +- * HashAggregate (53) - +- * HashAggregate (52) - +- Exchange (51) - +- * HashAggregate (50) - +- * Project (49) - +- * BroadcastHashJoin Inner BuildRight (48) - :- * Project (46) - : +- * BroadcastHashJoin Inner BuildLeft (45) - : :- BroadcastExchange (41) - : : +- * Project (40) - : : +- * BroadcastHashJoin Inner BuildLeft (39) +TakeOrderedAndProject (59) ++- * HashAggregate (58) + +- Exchange (57) + +- * HashAggregate (56) + +- * HashAggregate (55) + +- Exchange (54) + +- * HashAggregate (53) + +- * Project (52) + +- * BroadcastHashJoin Inner BuildRight (51) + :- * Project (49) + : +- * BroadcastHashJoin Inner BuildLeft (48) + : :- BroadcastExchange (44) + : : +- * Project (43) + : : +- * BroadcastHashJoin Inner BuildLeft (42) : : :- BroadcastExchange (10) : : : +- * Project (9) : : : +- * BroadcastHashJoin Inner BuildRight (8) @@ -23,38 +23,41 @@ TakeOrderedAndProject (56) : : : +- * Filter (6) : : : +- * ColumnarToRow (5) : : : +- Scan parquet spark_catalog.default.store (4) - : : +- * HashAggregate (38) - : : +- Exchange (37) - : : +- * HashAggregate (36) - : : +- * Project (35) - : : +- * BroadcastHashJoin Inner BuildLeft (34) - : : :- BroadcastExchange (30) - : : : +- * Project (29) - : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : :- * Project (26) - : : : : +- * BroadcastHashJoin Inner BuildRight (25) - : : : : :- Union (19) - : : : : : :- * Project (14) - : : : : : : +- * Filter (13) - : : : : : : +- * ColumnarToRow (12) - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (11) - : : : : : +- * Project (18) - : : : : : +- * Filter (17) - : : : : : +- * ColumnarToRow (16) - : : : : : +- Scan parquet spark_catalog.default.web_sales (15) - : : : : +- BroadcastExchange (24) - : : : : +- * Project (23) - : : : : +- * Filter (22) - : : : : +- * ColumnarToRow (21) - : : : : +- Scan parquet spark_catalog.default.item (20) - : : : +- ReusedExchange (27) - : : +- * Filter (33) - : : +- * ColumnarToRow (32) - : : +- Scan parquet spark_catalog.default.customer (31) - : +- * Filter (44) - : +- * ColumnarToRow (43) - : +- Scan parquet spark_catalog.default.store_sales (42) - +- ReusedExchange (47) + : : +- * HashAggregate (41) + : : +- Exchange (40) + : : +- * HashAggregate (39) + : : +- * Project (38) + : : +- * BroadcastHashJoin Inner BuildLeft (37) + : : :- BroadcastExchange (33) + : : : +- * Project (32) + : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : :- Union (29) + : : : : :- * Project (21) + : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : :- * Project (14) + : : : : : : +- * Filter (13) + : : : : : : +- * ColumnarToRow (12) + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (11) + : : : : : +- BroadcastExchange (19) + : : : : : +- * Project (18) + : : : : : +- * Filter (17) + : : : : : +- * ColumnarToRow (16) + : : : : : +- Scan parquet spark_catalog.default.item (15) + : : : : +- * Project (28) + : : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : : :- * Project (25) + : : : : : +- * Filter (24) + : : : : : +- * ColumnarToRow (23) + : : : : : +- Scan parquet spark_catalog.default.web_sales (22) + : : : : +- ReusedExchange (26) + : : : +- ReusedExchange (30) + : : +- * Filter (36) + : : +- * ColumnarToRow (35) + : : +- Scan parquet spark_catalog.default.customer (34) + : +- * Filter (47) + : +- * ColumnarToRow (46) + : +- Scan parquet spark_catalog.default.store_sales (45) + +- ReusedExchange (50) (1) Scan parquet spark_catalog.default.customer_address @@ -111,379 +114,392 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 3] +(12) ColumnarToRow [codegen id : 4] Input [3]: [cs_bill_customer_sk#6, cs_item_sk#7, cs_sold_date_sk#8] -(13) Filter [codegen id : 3] +(13) Filter [codegen id : 4] Input [3]: [cs_bill_customer_sk#6, cs_item_sk#7, cs_sold_date_sk#8] Condition : (isnotnull(cs_item_sk#7) AND isnotnull(cs_bill_customer_sk#6)) -(14) Project [codegen id : 3] +(14) Project [codegen id : 4] Output [3]: [cs_sold_date_sk#8 AS sold_date_sk#10, cs_bill_customer_sk#6 AS customer_sk#11, cs_item_sk#7 AS item_sk#12] Input [3]: [cs_bill_customer_sk#6, cs_item_sk#7, cs_sold_date_sk#8] -(15) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#13, ws_bill_customer_sk#14, ws_sold_date_sk#15] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#15), dynamicpruningexpression(ws_sold_date_sk#15 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(16) ColumnarToRow [codegen id : 4] -Input [3]: [ws_item_sk#13, ws_bill_customer_sk#14, ws_sold_date_sk#15] - -(17) Filter [codegen id : 4] -Input [3]: [ws_item_sk#13, ws_bill_customer_sk#14, ws_sold_date_sk#15] -Condition : (isnotnull(ws_item_sk#13) AND isnotnull(ws_bill_customer_sk#14)) - -(18) Project [codegen id : 4] -Output [3]: [ws_sold_date_sk#15 AS sold_date_sk#16, ws_bill_customer_sk#14 AS customer_sk#17, ws_item_sk#13 AS item_sk#18] -Input [3]: [ws_item_sk#13, ws_bill_customer_sk#14, ws_sold_date_sk#15] - -(19) Union - -(20) Scan parquet spark_catalog.default.item -Output [3]: [i_item_sk#19, i_class#20, i_category#21] +(15) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#13, i_class#14, i_category#15] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category), IsNotNull(i_class), EqualTo(i_category,Women ), EqualTo(i_class,maternity ), IsNotNull(i_item_sk)] ReadSchema: struct -(21) ColumnarToRow [codegen id : 5] -Input [3]: [i_item_sk#19, i_class#20, i_category#21] +(16) ColumnarToRow [codegen id : 3] +Input [3]: [i_item_sk#13, i_class#14, i_category#15] -(22) Filter [codegen id : 5] -Input [3]: [i_item_sk#19, i_class#20, i_category#21] -Condition : ((((isnotnull(i_category#21) AND isnotnull(i_class#20)) AND (i_category#21 = Women )) AND (i_class#20 = maternity )) AND isnotnull(i_item_sk#19)) +(17) Filter [codegen id : 3] +Input [3]: [i_item_sk#13, i_class#14, i_category#15] +Condition : ((((isnotnull(i_category#15) AND isnotnull(i_class#14)) AND (i_category#15 = Women )) AND (i_class#14 = maternity )) AND isnotnull(i_item_sk#13)) -(23) Project [codegen id : 5] -Output [1]: [i_item_sk#19] -Input [3]: [i_item_sk#19, i_class#20, i_category#21] +(18) Project [codegen id : 3] +Output [1]: [i_item_sk#13] +Input [3]: [i_item_sk#13, i_class#14, i_category#15] -(24) BroadcastExchange -Input [1]: [i_item_sk#19] +(19) BroadcastExchange +Input [1]: [i_item_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(25) BroadcastHashJoin [codegen id : 7] +(20) BroadcastHashJoin [codegen id : 4] Left keys [1]: [item_sk#12] -Right keys [1]: [i_item_sk#19] +Right keys [1]: [i_item_sk#13] Join type: Inner Join condition: None -(26) Project [codegen id : 7] +(21) Project [codegen id : 4] Output [2]: [sold_date_sk#10, customer_sk#11] -Input [4]: [sold_date_sk#10, customer_sk#11, item_sk#12, i_item_sk#19] +Input [4]: [sold_date_sk#10, customer_sk#11, item_sk#12, i_item_sk#13] + +(22) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_item_sk#16, ws_bill_customer_sk#17, ws_sold_date_sk#18] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#18), dynamicpruningexpression(ws_sold_date_sk#18 IN dynamicpruning#9)] +PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] +ReadSchema: struct + +(23) ColumnarToRow [codegen id : 6] +Input [3]: [ws_item_sk#16, ws_bill_customer_sk#17, ws_sold_date_sk#18] -(27) ReusedExchange [Reuses operator id: 61] -Output [1]: [d_date_sk#22] +(24) Filter [codegen id : 6] +Input [3]: [ws_item_sk#16, ws_bill_customer_sk#17, ws_sold_date_sk#18] +Condition : (isnotnull(ws_item_sk#16) AND isnotnull(ws_bill_customer_sk#17)) -(28) BroadcastHashJoin [codegen id : 7] +(25) Project [codegen id : 6] +Output [3]: [ws_sold_date_sk#18 AS sold_date_sk#19, ws_bill_customer_sk#17 AS customer_sk#20, ws_item_sk#16 AS item_sk#21] +Input [3]: [ws_item_sk#16, ws_bill_customer_sk#17, ws_sold_date_sk#18] + +(26) ReusedExchange [Reuses operator id: 19] +Output [1]: [i_item_sk#22] + +(27) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [item_sk#21] +Right keys [1]: [i_item_sk#22] +Join type: Inner +Join condition: None + +(28) Project [codegen id : 6] +Output [2]: [sold_date_sk#19, customer_sk#20] +Input [4]: [sold_date_sk#19, customer_sk#20, item_sk#21, i_item_sk#22] + +(29) Union + +(30) ReusedExchange [Reuses operator id: 64] +Output [1]: [d_date_sk#23] + +(31) BroadcastHashJoin [codegen id : 8] Left keys [1]: [sold_date_sk#10] -Right keys [1]: [d_date_sk#22] +Right keys [1]: [d_date_sk#23] Join type: Inner Join condition: None -(29) Project [codegen id : 7] +(32) Project [codegen id : 8] Output [1]: [customer_sk#11] -Input [3]: [sold_date_sk#10, customer_sk#11, d_date_sk#22] +Input [3]: [sold_date_sk#10, customer_sk#11, d_date_sk#23] -(30) BroadcastExchange +(33) BroadcastExchange Input [1]: [customer_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(31) Scan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#23, c_current_addr_sk#24] +(34) Scan parquet spark_catalog.default.customer +Output [2]: [c_customer_sk#24, c_current_addr_sk#25] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(32) ColumnarToRow -Input [2]: [c_customer_sk#23, c_current_addr_sk#24] +(35) ColumnarToRow +Input [2]: [c_customer_sk#24, c_current_addr_sk#25] -(33) Filter -Input [2]: [c_customer_sk#23, c_current_addr_sk#24] -Condition : (isnotnull(c_customer_sk#23) AND isnotnull(c_current_addr_sk#24)) +(36) Filter +Input [2]: [c_customer_sk#24, c_current_addr_sk#25] +Condition : (isnotnull(c_customer_sk#24) AND isnotnull(c_current_addr_sk#25)) -(34) BroadcastHashJoin [codegen id : 8] +(37) BroadcastHashJoin [codegen id : 9] Left keys [1]: [customer_sk#11] -Right keys [1]: [c_customer_sk#23] +Right keys [1]: [c_customer_sk#24] Join type: Inner Join condition: None -(35) Project [codegen id : 8] -Output [2]: [c_customer_sk#23, c_current_addr_sk#24] -Input [3]: [customer_sk#11, c_customer_sk#23, c_current_addr_sk#24] +(38) Project [codegen id : 9] +Output [2]: [c_customer_sk#24, c_current_addr_sk#25] +Input [3]: [customer_sk#11, c_customer_sk#24, c_current_addr_sk#25] -(36) HashAggregate [codegen id : 8] -Input [2]: [c_customer_sk#23, c_current_addr_sk#24] -Keys [2]: [c_customer_sk#23, c_current_addr_sk#24] +(39) HashAggregate [codegen id : 9] +Input [2]: [c_customer_sk#24, c_current_addr_sk#25] +Keys [2]: [c_customer_sk#24, c_current_addr_sk#25] Functions: [] Aggregate Attributes: [] -Results [2]: [c_customer_sk#23, c_current_addr_sk#24] +Results [2]: [c_customer_sk#24, c_current_addr_sk#25] -(37) Exchange -Input [2]: [c_customer_sk#23, c_current_addr_sk#24] -Arguments: hashpartitioning(c_customer_sk#23, c_current_addr_sk#24, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(40) Exchange +Input [2]: [c_customer_sk#24, c_current_addr_sk#25] +Arguments: hashpartitioning(c_customer_sk#24, c_current_addr_sk#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(38) HashAggregate -Input [2]: [c_customer_sk#23, c_current_addr_sk#24] -Keys [2]: [c_customer_sk#23, c_current_addr_sk#24] +(41) HashAggregate +Input [2]: [c_customer_sk#24, c_current_addr_sk#25] +Keys [2]: [c_customer_sk#24, c_current_addr_sk#25] Functions: [] Aggregate Attributes: [] -Results [2]: [c_customer_sk#23, c_current_addr_sk#24] +Results [2]: [c_customer_sk#24, c_current_addr_sk#25] -(39) BroadcastHashJoin [codegen id : 9] +(42) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ca_address_sk#1] -Right keys [1]: [c_current_addr_sk#24] +Right keys [1]: [c_current_addr_sk#25] Join type: Inner Join condition: None -(40) Project [codegen id : 9] -Output [1]: [c_customer_sk#23] -Input [3]: [ca_address_sk#1, c_customer_sk#23, c_current_addr_sk#24] +(43) Project [codegen id : 10] +Output [1]: [c_customer_sk#24] +Input [3]: [ca_address_sk#1, c_customer_sk#24, c_current_addr_sk#25] -(41) BroadcastExchange -Input [1]: [c_customer_sk#23] +(44) BroadcastExchange +Input [1]: [c_customer_sk#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(42) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#25, ss_ext_sales_price#26, ss_sold_date_sk#27] +(45) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_customer_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#27), dynamicpruningexpression(ss_sold_date_sk#27 IN dynamicpruning#28)] +PartitionFilters: [isnotnull(ss_sold_date_sk#28), dynamicpruningexpression(ss_sold_date_sk#28 IN dynamicpruning#29)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(43) ColumnarToRow -Input [3]: [ss_customer_sk#25, ss_ext_sales_price#26, ss_sold_date_sk#27] +(46) ColumnarToRow +Input [3]: [ss_customer_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] -(44) Filter -Input [3]: [ss_customer_sk#25, ss_ext_sales_price#26, ss_sold_date_sk#27] -Condition : isnotnull(ss_customer_sk#25) +(47) Filter +Input [3]: [ss_customer_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] +Condition : isnotnull(ss_customer_sk#26) -(45) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_customer_sk#23] -Right keys [1]: [ss_customer_sk#25] +(48) BroadcastHashJoin [codegen id : 12] +Left keys [1]: [c_customer_sk#24] +Right keys [1]: [ss_customer_sk#26] Join type: Inner Join condition: None -(46) Project [codegen id : 11] -Output [3]: [c_customer_sk#23, ss_ext_sales_price#26, ss_sold_date_sk#27] -Input [4]: [c_customer_sk#23, ss_customer_sk#25, ss_ext_sales_price#26, ss_sold_date_sk#27] +(49) Project [codegen id : 12] +Output [3]: [c_customer_sk#24, ss_ext_sales_price#27, ss_sold_date_sk#28] +Input [4]: [c_customer_sk#24, ss_customer_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] -(47) ReusedExchange [Reuses operator id: 66] -Output [1]: [d_date_sk#29] +(50) ReusedExchange [Reuses operator id: 69] +Output [1]: [d_date_sk#30] -(48) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#27] -Right keys [1]: [d_date_sk#29] +(51) BroadcastHashJoin [codegen id : 12] +Left keys [1]: [ss_sold_date_sk#28] +Right keys [1]: [d_date_sk#30] Join type: Inner Join condition: None -(49) Project [codegen id : 11] -Output [2]: [c_customer_sk#23, ss_ext_sales_price#26] -Input [4]: [c_customer_sk#23, ss_ext_sales_price#26, ss_sold_date_sk#27, d_date_sk#29] - -(50) HashAggregate [codegen id : 11] -Input [2]: [c_customer_sk#23, ss_ext_sales_price#26] -Keys [1]: [c_customer_sk#23] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#30] -Results [2]: [c_customer_sk#23, sum#31] - -(51) Exchange -Input [2]: [c_customer_sk#23, sum#31] -Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(52) HashAggregate [codegen id : 12] -Input [2]: [c_customer_sk#23, sum#31] -Keys [1]: [c_customer_sk#23] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#26))#32] -Results [1]: [cast((MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#26))#32,17,2) / 50) as int) AS segment#33] +(52) Project [codegen id : 12] +Output [2]: [c_customer_sk#24, ss_ext_sales_price#27] +Input [4]: [c_customer_sk#24, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#30] (53) HashAggregate [codegen id : 12] -Input [1]: [segment#33] -Keys [1]: [segment#33] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#34] -Results [2]: [segment#33, count#35] +Input [2]: [c_customer_sk#24, ss_ext_sales_price#27] +Keys [1]: [c_customer_sk#24] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#27))] +Aggregate Attributes [1]: [sum#31] +Results [2]: [c_customer_sk#24, sum#32] (54) Exchange -Input [2]: [segment#33, count#35] -Arguments: hashpartitioning(segment#33, 5), ENSURE_REQUIREMENTS, [plan_id=8] +Input [2]: [c_customer_sk#24, sum#32] +Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, [plan_id=7] (55) HashAggregate [codegen id : 13] -Input [2]: [segment#33, count#35] -Keys [1]: [segment#33] +Input [2]: [c_customer_sk#24, sum#32] +Keys [1]: [c_customer_sk#24] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#27))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#27))#33] +Results [1]: [cast((MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#27))#33,17,2) / 50) as int) AS segment#34] + +(56) HashAggregate [codegen id : 13] +Input [1]: [segment#34] +Keys [1]: [segment#34] +Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#35] +Results [2]: [segment#34, count#36] + +(57) Exchange +Input [2]: [segment#34, count#36] +Arguments: hashpartitioning(segment#34, 5), ENSURE_REQUIREMENTS, [plan_id=8] + +(58) HashAggregate [codegen id : 14] +Input [2]: [segment#34, count#36] +Keys [1]: [segment#34] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#36] -Results [3]: [segment#33, count(1)#36 AS num_customers#37, (segment#33 * 50) AS segment_base#38] +Aggregate Attributes [1]: [count(1)#37] +Results [3]: [segment#34, count(1)#37 AS num_customers#38, (segment#34 * 50) AS segment_base#39] -(56) TakeOrderedAndProject -Input [3]: [segment#33, num_customers#37, segment_base#38] -Arguments: 100, [segment#33 ASC NULLS FIRST, num_customers#37 ASC NULLS FIRST], [segment#33, num_customers#37, segment_base#38] +(59) TakeOrderedAndProject +Input [3]: [segment#34, num_customers#38, segment_base#39] +Arguments: 100, [segment#34 ASC NULLS FIRST, num_customers#38 ASC NULLS FIRST], [segment#34, num_customers#38, segment_base#39] ===== Subqueries ===== Subquery:1 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (61) -+- * Project (60) - +- * Filter (59) - +- * ColumnarToRow (58) - +- Scan parquet spark_catalog.default.date_dim (57) +BroadcastExchange (64) ++- * Project (63) + +- * Filter (62) + +- * ColumnarToRow (61) + +- Scan parquet spark_catalog.default.date_dim (60) -(57) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#22, d_year#39, d_moy#40] +(60) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#23, d_year#40, d_moy#41] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct -(58) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#22, d_year#39, d_moy#40] +(61) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#23, d_year#40, d_moy#41] -(59) Filter [codegen id : 1] -Input [3]: [d_date_sk#22, d_year#39, d_moy#40] -Condition : ((((isnotnull(d_moy#40) AND isnotnull(d_year#39)) AND (d_moy#40 = 12)) AND (d_year#39 = 1998)) AND isnotnull(d_date_sk#22)) +(62) Filter [codegen id : 1] +Input [3]: [d_date_sk#23, d_year#40, d_moy#41] +Condition : ((((isnotnull(d_moy#41) AND isnotnull(d_year#40)) AND (d_moy#41 = 12)) AND (d_year#40 = 1998)) AND isnotnull(d_date_sk#23)) -(60) Project [codegen id : 1] -Output [1]: [d_date_sk#22] -Input [3]: [d_date_sk#22, d_year#39, d_moy#40] +(63) Project [codegen id : 1] +Output [1]: [d_date_sk#23] +Input [3]: [d_date_sk#23, d_year#40, d_moy#41] -(61) BroadcastExchange -Input [1]: [d_date_sk#22] +(64) BroadcastExchange +Input [1]: [d_date_sk#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#15 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#18 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 42 Hosting Expression = ss_sold_date_sk#27 IN dynamicpruning#28 -BroadcastExchange (66) -+- * Project (65) - +- * Filter (64) - +- * ColumnarToRow (63) - +- Scan parquet spark_catalog.default.date_dim (62) +Subquery:3 Hosting operator id = 45 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#29 +BroadcastExchange (69) ++- * Project (68) + +- * Filter (67) + +- * ColumnarToRow (66) + +- Scan parquet spark_catalog.default.date_dim (65) -(62) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#29, d_month_seq#41] +(65) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#30, d_month_seq#42] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,ScalarSubquery#42), LessThanOrEqual(d_month_seq,ScalarSubquery#43), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,ScalarSubquery#43), LessThanOrEqual(d_month_seq,ScalarSubquery#44), IsNotNull(d_date_sk)] ReadSchema: struct -(63) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#29, d_month_seq#41] +(66) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#30, d_month_seq#42] -(64) Filter [codegen id : 1] -Input [2]: [d_date_sk#29, d_month_seq#41] -Condition : (((isnotnull(d_month_seq#41) AND (d_month_seq#41 >= ReusedSubquery Subquery scalar-subquery#42, [id=#10])) AND (d_month_seq#41 <= ReusedSubquery Subquery scalar-subquery#43, [id=#11])) AND isnotnull(d_date_sk#29)) +(67) Filter [codegen id : 1] +Input [2]: [d_date_sk#30, d_month_seq#42] +Condition : (((isnotnull(d_month_seq#42) AND (d_month_seq#42 >= ReusedSubquery Subquery scalar-subquery#43, [id=#10])) AND (d_month_seq#42 <= ReusedSubquery Subquery scalar-subquery#44, [id=#11])) AND isnotnull(d_date_sk#30)) -(65) Project [codegen id : 1] -Output [1]: [d_date_sk#29] -Input [2]: [d_date_sk#29, d_month_seq#41] +(68) Project [codegen id : 1] +Output [1]: [d_date_sk#30] +Input [2]: [d_date_sk#30, d_month_seq#42] -(66) BroadcastExchange -Input [1]: [d_date_sk#29] +(69) BroadcastExchange +Input [1]: [d_date_sk#30] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -Subquery:4 Hosting operator id = 64 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#10] +Subquery:4 Hosting operator id = 67 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#10] -Subquery:5 Hosting operator id = 64 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#11] +Subquery:5 Hosting operator id = 67 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#11] -Subquery:6 Hosting operator id = 62 Hosting Expression = Subquery scalar-subquery#42, [id=#10] -* HashAggregate (73) -+- Exchange (72) - +- * HashAggregate (71) - +- * Project (70) - +- * Filter (69) - +- * ColumnarToRow (68) - +- Scan parquet spark_catalog.default.date_dim (67) +Subquery:6 Hosting operator id = 65 Hosting Expression = Subquery scalar-subquery#43, [id=#10] +* HashAggregate (76) ++- Exchange (75) + +- * HashAggregate (74) + +- * Project (73) + +- * Filter (72) + +- * ColumnarToRow (71) + +- Scan parquet spark_catalog.default.date_dim (70) -(67) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#44, d_year#45, d_moy#46] +(70) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_month_seq#45, d_year#46, d_moy#47] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] ReadSchema: struct -(68) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#44, d_year#45, d_moy#46] +(71) ColumnarToRow [codegen id : 1] +Input [3]: [d_month_seq#45, d_year#46, d_moy#47] -(69) Filter [codegen id : 1] -Input [3]: [d_month_seq#44, d_year#45, d_moy#46] -Condition : (((isnotnull(d_year#45) AND isnotnull(d_moy#46)) AND (d_year#45 = 1998)) AND (d_moy#46 = 12)) +(72) Filter [codegen id : 1] +Input [3]: [d_month_seq#45, d_year#46, d_moy#47] +Condition : (((isnotnull(d_year#46) AND isnotnull(d_moy#47)) AND (d_year#46 = 1998)) AND (d_moy#47 = 12)) -(70) Project [codegen id : 1] -Output [1]: [(d_month_seq#44 + 1) AS (d_month_seq + 1)#47] -Input [3]: [d_month_seq#44, d_year#45, d_moy#46] +(73) Project [codegen id : 1] +Output [1]: [(d_month_seq#45 + 1) AS (d_month_seq + 1)#48] +Input [3]: [d_month_seq#45, d_year#46, d_moy#47] -(71) HashAggregate [codegen id : 1] -Input [1]: [(d_month_seq + 1)#47] -Keys [1]: [(d_month_seq + 1)#47] +(74) HashAggregate [codegen id : 1] +Input [1]: [(d_month_seq + 1)#48] +Keys [1]: [(d_month_seq + 1)#48] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 1)#47] +Results [1]: [(d_month_seq + 1)#48] -(72) Exchange -Input [1]: [(d_month_seq + 1)#47] -Arguments: hashpartitioning((d_month_seq + 1)#47, 5), ENSURE_REQUIREMENTS, [plan_id=13] +(75) Exchange +Input [1]: [(d_month_seq + 1)#48] +Arguments: hashpartitioning((d_month_seq + 1)#48, 5), ENSURE_REQUIREMENTS, [plan_id=13] -(73) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 1)#47] -Keys [1]: [(d_month_seq + 1)#47] +(76) HashAggregate [codegen id : 2] +Input [1]: [(d_month_seq + 1)#48] +Keys [1]: [(d_month_seq + 1)#48] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 1)#47] +Results [1]: [(d_month_seq + 1)#48] -Subquery:7 Hosting operator id = 62 Hosting Expression = Subquery scalar-subquery#43, [id=#11] -* HashAggregate (80) -+- Exchange (79) - +- * HashAggregate (78) - +- * Project (77) - +- * Filter (76) - +- * ColumnarToRow (75) - +- Scan parquet spark_catalog.default.date_dim (74) +Subquery:7 Hosting operator id = 65 Hosting Expression = Subquery scalar-subquery#44, [id=#11] +* HashAggregate (83) ++- Exchange (82) + +- * HashAggregate (81) + +- * Project (80) + +- * Filter (79) + +- * ColumnarToRow (78) + +- Scan parquet spark_catalog.default.date_dim (77) -(74) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#48, d_year#49, d_moy#50] +(77) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_month_seq#49, d_year#50, d_moy#51] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] ReadSchema: struct -(75) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#48, d_year#49, d_moy#50] +(78) ColumnarToRow [codegen id : 1] +Input [3]: [d_month_seq#49, d_year#50, d_moy#51] -(76) Filter [codegen id : 1] -Input [3]: [d_month_seq#48, d_year#49, d_moy#50] -Condition : (((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 1998)) AND (d_moy#50 = 12)) +(79) Filter [codegen id : 1] +Input [3]: [d_month_seq#49, d_year#50, d_moy#51] +Condition : (((isnotnull(d_year#50) AND isnotnull(d_moy#51)) AND (d_year#50 = 1998)) AND (d_moy#51 = 12)) -(77) Project [codegen id : 1] -Output [1]: [(d_month_seq#48 + 3) AS (d_month_seq + 3)#51] -Input [3]: [d_month_seq#48, d_year#49, d_moy#50] +(80) Project [codegen id : 1] +Output [1]: [(d_month_seq#49 + 3) AS (d_month_seq + 3)#52] +Input [3]: [d_month_seq#49, d_year#50, d_moy#51] -(78) HashAggregate [codegen id : 1] -Input [1]: [(d_month_seq + 3)#51] -Keys [1]: [(d_month_seq + 3)#51] +(81) HashAggregate [codegen id : 1] +Input [1]: [(d_month_seq + 3)#52] +Keys [1]: [(d_month_seq + 3)#52] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 3)#51] +Results [1]: [(d_month_seq + 3)#52] -(79) Exchange -Input [1]: [(d_month_seq + 3)#51] -Arguments: hashpartitioning((d_month_seq + 3)#51, 5), ENSURE_REQUIREMENTS, [plan_id=14] +(82) Exchange +Input [1]: [(d_month_seq + 3)#52] +Arguments: hashpartitioning((d_month_seq + 3)#52, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(80) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 3)#51] -Keys [1]: [(d_month_seq + 3)#51] +(83) HashAggregate [codegen id : 2] +Input [1]: [(d_month_seq + 3)#52] +Keys [1]: [(d_month_seq + 3)#52] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 3)#51] +Results [1]: [(d_month_seq + 3)#52] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/simplified.txt index 6e3b04eec936b..fa63498cb6f0e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/simplified.txt @@ -1,14 +1,14 @@ TakeOrderedAndProject [segment,num_customers,segment_base] - WholeStageCodegen (13) + WholeStageCodegen (14) HashAggregate [segment,count] [count(1),num_customers,segment_base,count] InputAdapter Exchange [segment] #1 - WholeStageCodegen (12) + WholeStageCodegen (13) HashAggregate [segment] [count,count] HashAggregate [c_customer_sk,sum] [sum(UnscaledValue(ss_ext_sales_price)),segment,sum] InputAdapter Exchange [c_customer_sk] #2 - WholeStageCodegen (11) + WholeStageCodegen (12) HashAggregate [c_customer_sk,ss_ext_sales_price] [sum,sum] Project [c_customer_sk,ss_ext_sales_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -16,7 +16,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] BroadcastHashJoin [c_customer_sk,ss_customer_sk] InputAdapter BroadcastExchange #3 - WholeStageCodegen (9) + WholeStageCodegen (10) Project [c_customer_sk] BroadcastHashJoin [ca_address_sk,c_current_addr_sk] InputAdapter @@ -38,20 +38,20 @@ TakeOrderedAndProject [segment,num_customers,segment_base] HashAggregate [c_customer_sk,c_current_addr_sk] InputAdapter Exchange [c_customer_sk,c_current_addr_sk] #6 - WholeStageCodegen (8) + WholeStageCodegen (9) HashAggregate [c_customer_sk,c_current_addr_sk] Project [c_customer_sk,c_current_addr_sk] BroadcastHashJoin [customer_sk,c_customer_sk] InputAdapter BroadcastExchange #7 - WholeStageCodegen (7) + WholeStageCodegen (8) Project [customer_sk] BroadcastHashJoin [sold_date_sk,d_date_sk] - Project [sold_date_sk,customer_sk] - BroadcastHashJoin [item_sk,i_item_sk] - InputAdapter - Union - WholeStageCodegen (3) + InputAdapter + Union + WholeStageCodegen (4) + Project [sold_date_sk,customer_sk] + BroadcastHashJoin [item_sk,i_item_sk] Project [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] Filter [cs_item_sk,cs_bill_customer_sk] ColumnarToRow @@ -65,21 +65,25 @@ TakeOrderedAndProject [segment,num_customers,segment_base] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (4) + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (3) + Project [i_item_sk] + Filter [i_category,i_class,i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + WholeStageCodegen (6) + Project [sold_date_sk,customer_sk] + BroadcastHashJoin [item_sk,i_item_sk] Project [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] Filter [ws_item_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (5) - Project [i_item_sk] - Filter [i_category,i_class,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + InputAdapter + ReusedExchange [i_item_sk] #9 InputAdapter ReusedExchange [d_date_sk] #8 Filter [c_customer_sk,c_current_addr_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt index c2c0f217f2cfe..2633dc7aab8fc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt @@ -1,60 +1,63 @@ == Physical Plan == -TakeOrderedAndProject (56) -+- * HashAggregate (55) - +- Exchange (54) - +- * HashAggregate (53) - +- * HashAggregate (52) - +- Exchange (51) - +- * HashAggregate (50) - +- * Project (49) - +- * BroadcastHashJoin Inner BuildRight (48) - :- * Project (46) - : +- * BroadcastHashJoin Inner BuildRight (45) - : :- * Project (40) - : : +- * BroadcastHashJoin Inner BuildRight (39) - : : :- * Project (34) - : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : :- * HashAggregate (28) - : : : : +- Exchange (27) - : : : : +- * HashAggregate (26) - : : : : +- * Project (25) - : : : : +- * BroadcastHashJoin Inner BuildRight (24) - : : : : :- * Project (19) - : : : : : +- * BroadcastHashJoin Inner BuildRight (18) - : : : : : :- * Project (16) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : : : : :- Union (9) - : : : : : : : :- * Project (4) - : : : : : : : : +- * Filter (3) - : : : : : : : : +- * ColumnarToRow (2) - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : : : : : +- * Project (8) - : : : : : : : +- * Filter (7) - : : : : : : : +- * ColumnarToRow (6) - : : : : : : : +- Scan parquet spark_catalog.default.web_sales (5) - : : : : : : +- BroadcastExchange (14) - : : : : : : +- * Project (13) - : : : : : : +- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet spark_catalog.default.item (10) - : : : : : +- ReusedExchange (17) - : : : : +- BroadcastExchange (23) - : : : : +- * Filter (22) - : : : : +- * ColumnarToRow (21) - : : : : +- Scan parquet spark_catalog.default.customer (20) - : : : +- BroadcastExchange (32) - : : : +- * Filter (31) - : : : +- * ColumnarToRow (30) - : : : +- Scan parquet spark_catalog.default.store_sales (29) - : : +- BroadcastExchange (38) - : : +- * Filter (37) - : : +- * ColumnarToRow (36) - : : +- Scan parquet spark_catalog.default.customer_address (35) - : +- BroadcastExchange (44) - : +- * Filter (43) - : +- * ColumnarToRow (42) - : +- Scan parquet spark_catalog.default.store (41) - +- ReusedExchange (47) +TakeOrderedAndProject (59) ++- * HashAggregate (58) + +- Exchange (57) + +- * HashAggregate (56) + +- * HashAggregate (55) + +- Exchange (54) + +- * HashAggregate (53) + +- * Project (52) + +- * BroadcastHashJoin Inner BuildRight (51) + :- * Project (49) + : +- * BroadcastHashJoin Inner BuildRight (48) + : :- * Project (43) + : : +- * BroadcastHashJoin Inner BuildRight (42) + : : :- * Project (37) + : : : +- * BroadcastHashJoin Inner BuildRight (36) + : : : :- * HashAggregate (31) + : : : : +- Exchange (30) + : : : : +- * HashAggregate (29) + : : : : +- * Project (28) + : : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : : :- * Project (22) + : : : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : : : :- Union (19) + : : : : : : :- * Project (11) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (10) + : : : : : : : :- * Project (4) + : : : : : : : : +- * Filter (3) + : : : : : : : : +- * ColumnarToRow (2) + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) + : : : : : : : +- BroadcastExchange (9) + : : : : : : : +- * Project (8) + : : : : : : : +- * Filter (7) + : : : : : : : +- * ColumnarToRow (6) + : : : : : : : +- Scan parquet spark_catalog.default.item (5) + : : : : : : +- * Project (18) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : : :- * Project (15) + : : : : : : : +- * Filter (14) + : : : : : : : +- * ColumnarToRow (13) + : : : : : : : +- Scan parquet spark_catalog.default.web_sales (12) + : : : : : : +- ReusedExchange (16) + : : : : : +- ReusedExchange (20) + : : : : +- BroadcastExchange (26) + : : : : +- * Filter (25) + : : : : +- * ColumnarToRow (24) + : : : : +- Scan parquet spark_catalog.default.customer (23) + : : : +- BroadcastExchange (35) + : : : +- * Filter (34) + : : : +- * ColumnarToRow (33) + : : : +- Scan parquet spark_catalog.default.store_sales (32) + : : +- BroadcastExchange (41) + : : +- * Filter (40) + : : +- * ColumnarToRow (39) + : : +- Scan parquet spark_catalog.default.customer_address (38) + : +- BroadcastExchange (47) + : +- * Filter (46) + : +- * ColumnarToRow (45) + : +- Scan parquet spark_catalog.default.store (44) + +- ReusedExchange (50) (1) Scan parquet spark_catalog.default.catalog_sales @@ -65,425 +68,438 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 1] +(2) ColumnarToRow [codegen id : 2] Input [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] -(3) Filter [codegen id : 1] +(3) Filter [codegen id : 2] Input [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_bill_customer_sk#1)) -(4) Project [codegen id : 1] +(4) Project [codegen id : 2] Output [3]: [cs_sold_date_sk#3 AS sold_date_sk#5, cs_bill_customer_sk#1 AS customer_sk#6, cs_item_sk#2 AS item_sk#7] Input [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] -(5) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#10), dynamicpruningexpression(ws_sold_date_sk#10 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 2] -Input [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] - -(7) Filter [codegen id : 2] -Input [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] -Condition : (isnotnull(ws_item_sk#8) AND isnotnull(ws_bill_customer_sk#9)) - -(8) Project [codegen id : 2] -Output [3]: [ws_sold_date_sk#10 AS sold_date_sk#11, ws_bill_customer_sk#9 AS customer_sk#12, ws_item_sk#8 AS item_sk#13] -Input [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] - -(9) Union - -(10) Scan parquet spark_catalog.default.item -Output [3]: [i_item_sk#14, i_class#15, i_category#16] +(5) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category), IsNotNull(i_class), EqualTo(i_category,Women ), EqualTo(i_class,maternity ), IsNotNull(i_item_sk)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 3] -Input [3]: [i_item_sk#14, i_class#15, i_category#16] +(6) ColumnarToRow [codegen id : 1] +Input [3]: [i_item_sk#8, i_class#9, i_category#10] -(12) Filter [codegen id : 3] -Input [3]: [i_item_sk#14, i_class#15, i_category#16] -Condition : ((((isnotnull(i_category#16) AND isnotnull(i_class#15)) AND (i_category#16 = Women )) AND (i_class#15 = maternity )) AND isnotnull(i_item_sk#14)) +(7) Filter [codegen id : 1] +Input [3]: [i_item_sk#8, i_class#9, i_category#10] +Condition : ((((isnotnull(i_category#10) AND isnotnull(i_class#9)) AND (i_category#10 = Women )) AND (i_class#9 = maternity )) AND isnotnull(i_item_sk#8)) -(13) Project [codegen id : 3] -Output [1]: [i_item_sk#14] -Input [3]: [i_item_sk#14, i_class#15, i_category#16] +(8) Project [codegen id : 1] +Output [1]: [i_item_sk#8] +Input [3]: [i_item_sk#8, i_class#9, i_category#10] -(14) BroadcastExchange -Input [1]: [i_item_sk#14] +(9) BroadcastExchange +Input [1]: [i_item_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(15) BroadcastHashJoin [codegen id : 6] +(10) BroadcastHashJoin [codegen id : 2] Left keys [1]: [item_sk#7] -Right keys [1]: [i_item_sk#14] +Right keys [1]: [i_item_sk#8] Join type: Inner Join condition: None -(16) Project [codegen id : 6] +(11) Project [codegen id : 2] Output [2]: [sold_date_sk#5, customer_sk#6] -Input [4]: [sold_date_sk#5, customer_sk#6, item_sk#7, i_item_sk#14] +Input [4]: [sold_date_sk#5, customer_sk#6, item_sk#7, i_item_sk#8] + +(12) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_item_sk#11, ws_bill_customer_sk#12, ws_sold_date_sk#13] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#4)] +PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] +ReadSchema: struct + +(13) ColumnarToRow [codegen id : 4] +Input [3]: [ws_item_sk#11, ws_bill_customer_sk#12, ws_sold_date_sk#13] -(17) ReusedExchange [Reuses operator id: 61] -Output [1]: [d_date_sk#17] +(14) Filter [codegen id : 4] +Input [3]: [ws_item_sk#11, ws_bill_customer_sk#12, ws_sold_date_sk#13] +Condition : (isnotnull(ws_item_sk#11) AND isnotnull(ws_bill_customer_sk#12)) -(18) BroadcastHashJoin [codegen id : 6] +(15) Project [codegen id : 4] +Output [3]: [ws_sold_date_sk#13 AS sold_date_sk#14, ws_bill_customer_sk#12 AS customer_sk#15, ws_item_sk#11 AS item_sk#16] +Input [3]: [ws_item_sk#11, ws_bill_customer_sk#12, ws_sold_date_sk#13] + +(16) ReusedExchange [Reuses operator id: 9] +Output [1]: [i_item_sk#17] + +(17) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [item_sk#16] +Right keys [1]: [i_item_sk#17] +Join type: Inner +Join condition: None + +(18) Project [codegen id : 4] +Output [2]: [sold_date_sk#14, customer_sk#15] +Input [4]: [sold_date_sk#14, customer_sk#15, item_sk#16, i_item_sk#17] + +(19) Union + +(20) ReusedExchange [Reuses operator id: 64] +Output [1]: [d_date_sk#18] + +(21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [sold_date_sk#5] -Right keys [1]: [d_date_sk#17] +Right keys [1]: [d_date_sk#18] Join type: Inner Join condition: None -(19) Project [codegen id : 6] +(22) Project [codegen id : 7] Output [1]: [customer_sk#6] -Input [3]: [sold_date_sk#5, customer_sk#6, d_date_sk#17] +Input [3]: [sold_date_sk#5, customer_sk#6, d_date_sk#18] -(20) Scan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#18, c_current_addr_sk#19] +(23) Scan parquet spark_catalog.default.customer +Output [2]: [c_customer_sk#19, c_current_addr_sk#20] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(21) ColumnarToRow [codegen id : 5] -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] +(24) ColumnarToRow [codegen id : 6] +Input [2]: [c_customer_sk#19, c_current_addr_sk#20] -(22) Filter [codegen id : 5] -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Condition : (isnotnull(c_customer_sk#18) AND isnotnull(c_current_addr_sk#19)) +(25) Filter [codegen id : 6] +Input [2]: [c_customer_sk#19, c_current_addr_sk#20] +Condition : (isnotnull(c_customer_sk#19) AND isnotnull(c_current_addr_sk#20)) -(23) BroadcastExchange -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] +(26) BroadcastExchange +Input [2]: [c_customer_sk#19, c_current_addr_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(24) BroadcastHashJoin [codegen id : 6] +(27) BroadcastHashJoin [codegen id : 7] Left keys [1]: [customer_sk#6] -Right keys [1]: [c_customer_sk#18] +Right keys [1]: [c_customer_sk#19] Join type: Inner Join condition: None -(25) Project [codegen id : 6] -Output [2]: [c_customer_sk#18, c_current_addr_sk#19] -Input [3]: [customer_sk#6, c_customer_sk#18, c_current_addr_sk#19] +(28) Project [codegen id : 7] +Output [2]: [c_customer_sk#19, c_current_addr_sk#20] +Input [3]: [customer_sk#6, c_customer_sk#19, c_current_addr_sk#20] -(26) HashAggregate [codegen id : 6] -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Keys [2]: [c_customer_sk#18, c_current_addr_sk#19] +(29) HashAggregate [codegen id : 7] +Input [2]: [c_customer_sk#19, c_current_addr_sk#20] +Keys [2]: [c_customer_sk#19, c_current_addr_sk#20] Functions: [] Aggregate Attributes: [] -Results [2]: [c_customer_sk#18, c_current_addr_sk#19] +Results [2]: [c_customer_sk#19, c_current_addr_sk#20] -(27) Exchange -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Arguments: hashpartitioning(c_customer_sk#18, c_current_addr_sk#19, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(30) Exchange +Input [2]: [c_customer_sk#19, c_current_addr_sk#20] +Arguments: hashpartitioning(c_customer_sk#19, c_current_addr_sk#20, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(28) HashAggregate [codegen id : 11] -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Keys [2]: [c_customer_sk#18, c_current_addr_sk#19] +(31) HashAggregate [codegen id : 12] +Input [2]: [c_customer_sk#19, c_current_addr_sk#20] +Keys [2]: [c_customer_sk#19, c_current_addr_sk#20] Functions: [] Aggregate Attributes: [] -Results [2]: [c_customer_sk#18, c_current_addr_sk#19] +Results [2]: [c_customer_sk#19, c_current_addr_sk#20] -(29) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] +(32) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_customer_sk#21, ss_ext_sales_price#22, ss_sold_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_sold_date_sk#22 IN dynamicpruning#23)] +PartitionFilters: [isnotnull(ss_sold_date_sk#23), dynamicpruningexpression(ss_sold_date_sk#23 IN dynamicpruning#24)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(30) ColumnarToRow [codegen id : 7] -Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] +(33) ColumnarToRow [codegen id : 8] +Input [3]: [ss_customer_sk#21, ss_ext_sales_price#22, ss_sold_date_sk#23] -(31) Filter [codegen id : 7] -Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] -Condition : isnotnull(ss_customer_sk#20) +(34) Filter [codegen id : 8] +Input [3]: [ss_customer_sk#21, ss_ext_sales_price#22, ss_sold_date_sk#23] +Condition : isnotnull(ss_customer_sk#21) -(32) BroadcastExchange -Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] +(35) BroadcastExchange +Input [3]: [ss_customer_sk#21, ss_ext_sales_price#22, ss_sold_date_sk#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(33) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_customer_sk#18] -Right keys [1]: [ss_customer_sk#20] +(36) BroadcastHashJoin [codegen id : 12] +Left keys [1]: [c_customer_sk#19] +Right keys [1]: [ss_customer_sk#21] Join type: Inner Join condition: None -(34) Project [codegen id : 11] -Output [4]: [c_customer_sk#18, c_current_addr_sk#19, ss_ext_sales_price#21, ss_sold_date_sk#22] -Input [5]: [c_customer_sk#18, c_current_addr_sk#19, ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] +(37) Project [codegen id : 12] +Output [4]: [c_customer_sk#19, c_current_addr_sk#20, ss_ext_sales_price#22, ss_sold_date_sk#23] +Input [5]: [c_customer_sk#19, c_current_addr_sk#20, ss_customer_sk#21, ss_ext_sales_price#22, ss_sold_date_sk#23] -(35) Scan parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#24, ca_county#25, ca_state#26] +(38) Scan parquet spark_catalog.default.customer_address +Output [3]: [ca_address_sk#25, ca_county#26, ca_state#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county), IsNotNull(ca_state)] ReadSchema: struct -(36) ColumnarToRow [codegen id : 8] -Input [3]: [ca_address_sk#24, ca_county#25, ca_state#26] +(39) ColumnarToRow [codegen id : 9] +Input [3]: [ca_address_sk#25, ca_county#26, ca_state#27] -(37) Filter [codegen id : 8] -Input [3]: [ca_address_sk#24, ca_county#25, ca_state#26] -Condition : ((isnotnull(ca_address_sk#24) AND isnotnull(ca_county#25)) AND isnotnull(ca_state#26)) +(40) Filter [codegen id : 9] +Input [3]: [ca_address_sk#25, ca_county#26, ca_state#27] +Condition : ((isnotnull(ca_address_sk#25) AND isnotnull(ca_county#26)) AND isnotnull(ca_state#27)) -(38) BroadcastExchange -Input [3]: [ca_address_sk#24, ca_county#25, ca_state#26] +(41) BroadcastExchange +Input [3]: [ca_address_sk#25, ca_county#26, ca_state#27] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(39) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#19] -Right keys [1]: [ca_address_sk#24] +(42) BroadcastHashJoin [codegen id : 12] +Left keys [1]: [c_current_addr_sk#20] +Right keys [1]: [ca_address_sk#25] Join type: Inner Join condition: None -(40) Project [codegen id : 11] -Output [5]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_county#25, ca_state#26] -Input [7]: [c_customer_sk#18, c_current_addr_sk#19, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_address_sk#24, ca_county#25, ca_state#26] +(43) Project [codegen id : 12] +Output [5]: [c_customer_sk#19, ss_ext_sales_price#22, ss_sold_date_sk#23, ca_county#26, ca_state#27] +Input [7]: [c_customer_sk#19, c_current_addr_sk#20, ss_ext_sales_price#22, ss_sold_date_sk#23, ca_address_sk#25, ca_county#26, ca_state#27] -(41) Scan parquet spark_catalog.default.store -Output [2]: [s_county#27, s_state#28] +(44) Scan parquet spark_catalog.default.store +Output [2]: [s_county#28, s_state#29] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_county), IsNotNull(s_state)] ReadSchema: struct -(42) ColumnarToRow [codegen id : 9] -Input [2]: [s_county#27, s_state#28] +(45) ColumnarToRow [codegen id : 10] +Input [2]: [s_county#28, s_state#29] -(43) Filter [codegen id : 9] -Input [2]: [s_county#27, s_state#28] -Condition : (isnotnull(s_county#27) AND isnotnull(s_state#28)) +(46) Filter [codegen id : 10] +Input [2]: [s_county#28, s_state#29] +Condition : (isnotnull(s_county#28) AND isnotnull(s_state#29)) -(44) BroadcastExchange -Input [2]: [s_county#27, s_state#28] +(47) BroadcastExchange +Input [2]: [s_county#28, s_state#29] Arguments: HashedRelationBroadcastMode(List(input[0, string, false], input[1, string, false]),false), [plan_id=6] -(45) BroadcastHashJoin [codegen id : 11] -Left keys [2]: [ca_county#25, ca_state#26] -Right keys [2]: [s_county#27, s_state#28] +(48) BroadcastHashJoin [codegen id : 12] +Left keys [2]: [ca_county#26, ca_state#27] +Right keys [2]: [s_county#28, s_state#29] Join type: Inner Join condition: None -(46) Project [codegen id : 11] -Output [3]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22] -Input [7]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_county#25, ca_state#26, s_county#27, s_state#28] +(49) Project [codegen id : 12] +Output [3]: [c_customer_sk#19, ss_ext_sales_price#22, ss_sold_date_sk#23] +Input [7]: [c_customer_sk#19, ss_ext_sales_price#22, ss_sold_date_sk#23, ca_county#26, ca_state#27, s_county#28, s_state#29] -(47) ReusedExchange [Reuses operator id: 66] -Output [1]: [d_date_sk#29] +(50) ReusedExchange [Reuses operator id: 69] +Output [1]: [d_date_sk#30] -(48) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#22] -Right keys [1]: [d_date_sk#29] +(51) BroadcastHashJoin [codegen id : 12] +Left keys [1]: [ss_sold_date_sk#23] +Right keys [1]: [d_date_sk#30] Join type: Inner Join condition: None -(49) Project [codegen id : 11] -Output [2]: [c_customer_sk#18, ss_ext_sales_price#21] -Input [4]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, d_date_sk#29] - -(50) HashAggregate [codegen id : 11] -Input [2]: [c_customer_sk#18, ss_ext_sales_price#21] -Keys [1]: [c_customer_sk#18] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#21))] -Aggregate Attributes [1]: [sum#30] -Results [2]: [c_customer_sk#18, sum#31] - -(51) Exchange -Input [2]: [c_customer_sk#18, sum#31] -Arguments: hashpartitioning(c_customer_sk#18, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(52) HashAggregate [codegen id : 12] -Input [2]: [c_customer_sk#18, sum#31] -Keys [1]: [c_customer_sk#18] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#21))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#21))#32] -Results [1]: [cast((MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#21))#32,17,2) / 50) as int) AS segment#33] +(52) Project [codegen id : 12] +Output [2]: [c_customer_sk#19, ss_ext_sales_price#22] +Input [4]: [c_customer_sk#19, ss_ext_sales_price#22, ss_sold_date_sk#23, d_date_sk#30] (53) HashAggregate [codegen id : 12] -Input [1]: [segment#33] -Keys [1]: [segment#33] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#34] -Results [2]: [segment#33, count#35] +Input [2]: [c_customer_sk#19, ss_ext_sales_price#22] +Keys [1]: [c_customer_sk#19] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#22))] +Aggregate Attributes [1]: [sum#31] +Results [2]: [c_customer_sk#19, sum#32] (54) Exchange -Input [2]: [segment#33, count#35] -Arguments: hashpartitioning(segment#33, 5), ENSURE_REQUIREMENTS, [plan_id=8] +Input [2]: [c_customer_sk#19, sum#32] +Arguments: hashpartitioning(c_customer_sk#19, 5), ENSURE_REQUIREMENTS, [plan_id=7] (55) HashAggregate [codegen id : 13] -Input [2]: [segment#33, count#35] -Keys [1]: [segment#33] +Input [2]: [c_customer_sk#19, sum#32] +Keys [1]: [c_customer_sk#19] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#22))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#22))#33] +Results [1]: [cast((MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#22))#33,17,2) / 50) as int) AS segment#34] + +(56) HashAggregate [codegen id : 13] +Input [1]: [segment#34] +Keys [1]: [segment#34] +Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#35] +Results [2]: [segment#34, count#36] + +(57) Exchange +Input [2]: [segment#34, count#36] +Arguments: hashpartitioning(segment#34, 5), ENSURE_REQUIREMENTS, [plan_id=8] + +(58) HashAggregate [codegen id : 14] +Input [2]: [segment#34, count#36] +Keys [1]: [segment#34] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#36] -Results [3]: [segment#33, count(1)#36 AS num_customers#37, (segment#33 * 50) AS segment_base#38] +Aggregate Attributes [1]: [count(1)#37] +Results [3]: [segment#34, count(1)#37 AS num_customers#38, (segment#34 * 50) AS segment_base#39] -(56) TakeOrderedAndProject -Input [3]: [segment#33, num_customers#37, segment_base#38] -Arguments: 100, [segment#33 ASC NULLS FIRST, num_customers#37 ASC NULLS FIRST], [segment#33, num_customers#37, segment_base#38] +(59) TakeOrderedAndProject +Input [3]: [segment#34, num_customers#38, segment_base#39] +Arguments: 100, [segment#34 ASC NULLS FIRST, num_customers#38 ASC NULLS FIRST], [segment#34, num_customers#38, segment_base#39] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (61) -+- * Project (60) - +- * Filter (59) - +- * ColumnarToRow (58) - +- Scan parquet spark_catalog.default.date_dim (57) +BroadcastExchange (64) ++- * Project (63) + +- * Filter (62) + +- * ColumnarToRow (61) + +- Scan parquet spark_catalog.default.date_dim (60) -(57) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#39, d_moy#40] +(60) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#18, d_year#40, d_moy#41] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct -(58) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#17, d_year#39, d_moy#40] +(61) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#18, d_year#40, d_moy#41] -(59) Filter [codegen id : 1] -Input [3]: [d_date_sk#17, d_year#39, d_moy#40] -Condition : ((((isnotnull(d_moy#40) AND isnotnull(d_year#39)) AND (d_moy#40 = 12)) AND (d_year#39 = 1998)) AND isnotnull(d_date_sk#17)) +(62) Filter [codegen id : 1] +Input [3]: [d_date_sk#18, d_year#40, d_moy#41] +Condition : ((((isnotnull(d_moy#41) AND isnotnull(d_year#40)) AND (d_moy#41 = 12)) AND (d_year#40 = 1998)) AND isnotnull(d_date_sk#18)) -(60) Project [codegen id : 1] -Output [1]: [d_date_sk#17] -Input [3]: [d_date_sk#17, d_year#39, d_moy#40] +(63) Project [codegen id : 1] +Output [1]: [d_date_sk#18] +Input [3]: [d_date_sk#18, d_year#40, d_moy#41] -(61) BroadcastExchange -Input [1]: [d_date_sk#17] +(64) BroadcastExchange +Input [1]: [d_date_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -Subquery:2 Hosting operator id = 5 Hosting Expression = ws_sold_date_sk#10 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#4 -Subquery:3 Hosting operator id = 29 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 -BroadcastExchange (66) -+- * Project (65) - +- * Filter (64) - +- * ColumnarToRow (63) - +- Scan parquet spark_catalog.default.date_dim (62) +Subquery:3 Hosting operator id = 32 Hosting Expression = ss_sold_date_sk#23 IN dynamicpruning#24 +BroadcastExchange (69) ++- * Project (68) + +- * Filter (67) + +- * ColumnarToRow (66) + +- Scan parquet spark_catalog.default.date_dim (65) -(62) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#29, d_month_seq#41] +(65) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#30, d_month_seq#42] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,ScalarSubquery#42), LessThanOrEqual(d_month_seq,ScalarSubquery#43), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,ScalarSubquery#43), LessThanOrEqual(d_month_seq,ScalarSubquery#44), IsNotNull(d_date_sk)] ReadSchema: struct -(63) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#29, d_month_seq#41] +(66) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#30, d_month_seq#42] -(64) Filter [codegen id : 1] -Input [2]: [d_date_sk#29, d_month_seq#41] -Condition : (((isnotnull(d_month_seq#41) AND (d_month_seq#41 >= ReusedSubquery Subquery scalar-subquery#42, [id=#10])) AND (d_month_seq#41 <= ReusedSubquery Subquery scalar-subquery#43, [id=#11])) AND isnotnull(d_date_sk#29)) +(67) Filter [codegen id : 1] +Input [2]: [d_date_sk#30, d_month_seq#42] +Condition : (((isnotnull(d_month_seq#42) AND (d_month_seq#42 >= ReusedSubquery Subquery scalar-subquery#43, [id=#10])) AND (d_month_seq#42 <= ReusedSubquery Subquery scalar-subquery#44, [id=#11])) AND isnotnull(d_date_sk#30)) -(65) Project [codegen id : 1] -Output [1]: [d_date_sk#29] -Input [2]: [d_date_sk#29, d_month_seq#41] +(68) Project [codegen id : 1] +Output [1]: [d_date_sk#30] +Input [2]: [d_date_sk#30, d_month_seq#42] -(66) BroadcastExchange -Input [1]: [d_date_sk#29] +(69) BroadcastExchange +Input [1]: [d_date_sk#30] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -Subquery:4 Hosting operator id = 64 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#10] +Subquery:4 Hosting operator id = 67 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#10] -Subquery:5 Hosting operator id = 64 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#11] +Subquery:5 Hosting operator id = 67 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#11] -Subquery:6 Hosting operator id = 62 Hosting Expression = Subquery scalar-subquery#42, [id=#10] -* HashAggregate (73) -+- Exchange (72) - +- * HashAggregate (71) - +- * Project (70) - +- * Filter (69) - +- * ColumnarToRow (68) - +- Scan parquet spark_catalog.default.date_dim (67) +Subquery:6 Hosting operator id = 65 Hosting Expression = Subquery scalar-subquery#43, [id=#10] +* HashAggregate (76) ++- Exchange (75) + +- * HashAggregate (74) + +- * Project (73) + +- * Filter (72) + +- * ColumnarToRow (71) + +- Scan parquet spark_catalog.default.date_dim (70) -(67) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#44, d_year#45, d_moy#46] +(70) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_month_seq#45, d_year#46, d_moy#47] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] ReadSchema: struct -(68) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#44, d_year#45, d_moy#46] +(71) ColumnarToRow [codegen id : 1] +Input [3]: [d_month_seq#45, d_year#46, d_moy#47] -(69) Filter [codegen id : 1] -Input [3]: [d_month_seq#44, d_year#45, d_moy#46] -Condition : (((isnotnull(d_year#45) AND isnotnull(d_moy#46)) AND (d_year#45 = 1998)) AND (d_moy#46 = 12)) +(72) Filter [codegen id : 1] +Input [3]: [d_month_seq#45, d_year#46, d_moy#47] +Condition : (((isnotnull(d_year#46) AND isnotnull(d_moy#47)) AND (d_year#46 = 1998)) AND (d_moy#47 = 12)) -(70) Project [codegen id : 1] -Output [1]: [(d_month_seq#44 + 1) AS (d_month_seq + 1)#47] -Input [3]: [d_month_seq#44, d_year#45, d_moy#46] +(73) Project [codegen id : 1] +Output [1]: [(d_month_seq#45 + 1) AS (d_month_seq + 1)#48] +Input [3]: [d_month_seq#45, d_year#46, d_moy#47] -(71) HashAggregate [codegen id : 1] -Input [1]: [(d_month_seq + 1)#47] -Keys [1]: [(d_month_seq + 1)#47] +(74) HashAggregate [codegen id : 1] +Input [1]: [(d_month_seq + 1)#48] +Keys [1]: [(d_month_seq + 1)#48] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 1)#47] +Results [1]: [(d_month_seq + 1)#48] -(72) Exchange -Input [1]: [(d_month_seq + 1)#47] -Arguments: hashpartitioning((d_month_seq + 1)#47, 5), ENSURE_REQUIREMENTS, [plan_id=13] +(75) Exchange +Input [1]: [(d_month_seq + 1)#48] +Arguments: hashpartitioning((d_month_seq + 1)#48, 5), ENSURE_REQUIREMENTS, [plan_id=13] -(73) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 1)#47] -Keys [1]: [(d_month_seq + 1)#47] +(76) HashAggregate [codegen id : 2] +Input [1]: [(d_month_seq + 1)#48] +Keys [1]: [(d_month_seq + 1)#48] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 1)#47] +Results [1]: [(d_month_seq + 1)#48] -Subquery:7 Hosting operator id = 62 Hosting Expression = Subquery scalar-subquery#43, [id=#11] -* HashAggregate (80) -+- Exchange (79) - +- * HashAggregate (78) - +- * Project (77) - +- * Filter (76) - +- * ColumnarToRow (75) - +- Scan parquet spark_catalog.default.date_dim (74) +Subquery:7 Hosting operator id = 65 Hosting Expression = Subquery scalar-subquery#44, [id=#11] +* HashAggregate (83) ++- Exchange (82) + +- * HashAggregate (81) + +- * Project (80) + +- * Filter (79) + +- * ColumnarToRow (78) + +- Scan parquet spark_catalog.default.date_dim (77) -(74) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#48, d_year#49, d_moy#50] +(77) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_month_seq#49, d_year#50, d_moy#51] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] ReadSchema: struct -(75) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#48, d_year#49, d_moy#50] +(78) ColumnarToRow [codegen id : 1] +Input [3]: [d_month_seq#49, d_year#50, d_moy#51] -(76) Filter [codegen id : 1] -Input [3]: [d_month_seq#48, d_year#49, d_moy#50] -Condition : (((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 1998)) AND (d_moy#50 = 12)) +(79) Filter [codegen id : 1] +Input [3]: [d_month_seq#49, d_year#50, d_moy#51] +Condition : (((isnotnull(d_year#50) AND isnotnull(d_moy#51)) AND (d_year#50 = 1998)) AND (d_moy#51 = 12)) -(77) Project [codegen id : 1] -Output [1]: [(d_month_seq#48 + 3) AS (d_month_seq + 3)#51] -Input [3]: [d_month_seq#48, d_year#49, d_moy#50] +(80) Project [codegen id : 1] +Output [1]: [(d_month_seq#49 + 3) AS (d_month_seq + 3)#52] +Input [3]: [d_month_seq#49, d_year#50, d_moy#51] -(78) HashAggregate [codegen id : 1] -Input [1]: [(d_month_seq + 3)#51] -Keys [1]: [(d_month_seq + 3)#51] +(81) HashAggregate [codegen id : 1] +Input [1]: [(d_month_seq + 3)#52] +Keys [1]: [(d_month_seq + 3)#52] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 3)#51] +Results [1]: [(d_month_seq + 3)#52] -(79) Exchange -Input [1]: [(d_month_seq + 3)#51] -Arguments: hashpartitioning((d_month_seq + 3)#51, 5), ENSURE_REQUIREMENTS, [plan_id=14] +(82) Exchange +Input [1]: [(d_month_seq + 3)#52] +Arguments: hashpartitioning((d_month_seq + 3)#52, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(80) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 3)#51] -Keys [1]: [(d_month_seq + 3)#51] +(83) HashAggregate [codegen id : 2] +Input [1]: [(d_month_seq + 3)#52] +Keys [1]: [(d_month_seq + 3)#52] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 3)#51] +Results [1]: [(d_month_seq + 3)#52] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt index 9547f9ba40463..5bba555942e77 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt @@ -1,14 +1,14 @@ TakeOrderedAndProject [segment,num_customers,segment_base] - WholeStageCodegen (13) + WholeStageCodegen (14) HashAggregate [segment,count] [count(1),num_customers,segment_base,count] InputAdapter Exchange [segment] #1 - WholeStageCodegen (12) + WholeStageCodegen (13) HashAggregate [segment] [count,count] HashAggregate [c_customer_sk,sum] [sum(UnscaledValue(ss_ext_sales_price)),segment,sum] InputAdapter Exchange [c_customer_sk] #2 - WholeStageCodegen (11) + WholeStageCodegen (12) HashAggregate [c_customer_sk,ss_ext_sales_price] [sum,sum] Project [c_customer_sk,ss_ext_sales_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -21,17 +21,17 @@ TakeOrderedAndProject [segment,num_customers,segment_base] HashAggregate [c_customer_sk,c_current_addr_sk] InputAdapter Exchange [c_customer_sk,c_current_addr_sk] #3 - WholeStageCodegen (6) + WholeStageCodegen (7) HashAggregate [c_customer_sk,c_current_addr_sk] Project [c_customer_sk,c_current_addr_sk] BroadcastHashJoin [customer_sk,c_customer_sk] Project [customer_sk] BroadcastHashJoin [sold_date_sk,d_date_sk] - Project [sold_date_sk,customer_sk] - BroadcastHashJoin [item_sk,i_item_sk] - InputAdapter - Union - WholeStageCodegen (1) + InputAdapter + Union + WholeStageCodegen (2) + Project [sold_date_sk,customer_sk] + BroadcastHashJoin [item_sk,i_item_sk] Project [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] Filter [cs_item_sk,cs_bill_customer_sk] ColumnarToRow @@ -45,33 +45,37 @@ TakeOrderedAndProject [segment,num_customers,segment_base] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (2) + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + Project [i_item_sk] + Filter [i_category,i_class,i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + WholeStageCodegen (4) + Project [sold_date_sk,customer_sk] + BroadcastHashJoin [item_sk,i_item_sk] Project [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] Filter [ws_item_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Project [i_item_sk] - Filter [i_category,i_class,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + InputAdapter + ReusedExchange [i_item_sk] #5 InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #6 - WholeStageCodegen (5) + WholeStageCodegen (6) Filter [c_customer_sk,c_current_addr_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter BroadcastExchange #7 - WholeStageCodegen (7) + WholeStageCodegen (8) Filter [ss_customer_sk] ColumnarToRow InputAdapter @@ -112,14 +116,14 @@ TakeOrderedAndProject [segment,num_customers,segment_base] Scan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] InputAdapter BroadcastExchange #11 - WholeStageCodegen (8) + WholeStageCodegen (9) Filter [ca_address_sk,ca_county,ca_state] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state] InputAdapter BroadcastExchange #12 - WholeStageCodegen (9) + WholeStageCodegen (10) Filter [s_county,s_state] ColumnarToRow InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt index 958a20cb6e9c1..790d0a423500e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt @@ -1,94 +1,103 @@ == Physical Plan == -TakeOrderedAndProject (90) -+- * HashAggregate (89) - +- Exchange (88) - +- * HashAggregate (87) - +- Union (86) - :- * HashAggregate (75) - : +- Exchange (74) - : +- * HashAggregate (73) - : +- Union (72) - : :- * HashAggregate (21) - : : +- Exchange (20) - : : +- * HashAggregate (19) - : : +- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- Union (9) - : : : : :- * Project (4) - : : : : : +- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : +- * Project (8) - : : : : +- * Filter (7) - : : : : +- * ColumnarToRow (6) - : : : : +- Scan parquet spark_catalog.default.store_returns (5) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (16) - : : +- * Filter (15) - : : +- * ColumnarToRow (14) - : : +- Scan parquet spark_catalog.default.store (13) - : :- * HashAggregate (42) - : : +- Exchange (41) - : : +- * HashAggregate (40) - : : +- * Project (39) - : : +- * BroadcastHashJoin Inner BuildRight (38) - : : :- * Project (33) - : : : +- * BroadcastHashJoin Inner BuildRight (32) - : : : :- Union (30) - : : : : :- * Project (25) - : : : : : +- * Filter (24) - : : : : : +- * ColumnarToRow (23) - : : : : : +- Scan parquet spark_catalog.default.catalog_sales (22) - : : : : +- * Project (29) - : : : : +- * Filter (28) - : : : : +- * ColumnarToRow (27) - : : : : +- Scan parquet spark_catalog.default.catalog_returns (26) - : : : +- ReusedExchange (31) - : : +- BroadcastExchange (37) - : : +- * Filter (36) - : : +- * ColumnarToRow (35) - : : +- Scan parquet spark_catalog.default.catalog_page (34) - : +- * HashAggregate (71) - : +- Exchange (70) - : +- * HashAggregate (69) - : +- * Project (68) - : +- * BroadcastHashJoin Inner BuildRight (67) - : :- * Project (62) - : : +- * BroadcastHashJoin Inner BuildRight (61) - : : :- Union (59) - : : : :- * Project (46) - : : : : +- * Filter (45) - : : : : +- * ColumnarToRow (44) - : : : : +- Scan parquet spark_catalog.default.web_sales (43) - : : : +- * Project (58) - : : : +- * SortMergeJoin Inner (57) - : : : :- * Sort (50) - : : : : +- Exchange (49) - : : : : +- * ColumnarToRow (48) - : : : : +- Scan parquet spark_catalog.default.web_returns (47) - : : : +- * Sort (56) - : : : +- Exchange (55) - : : : +- * Project (54) - : : : +- * Filter (53) - : : : +- * ColumnarToRow (52) - : : : +- Scan parquet spark_catalog.default.web_sales (51) - : : +- ReusedExchange (60) - : +- BroadcastExchange (66) - : +- * Filter (65) - : +- * ColumnarToRow (64) - : +- Scan parquet spark_catalog.default.web_site (63) - :- * HashAggregate (80) - : +- Exchange (79) - : +- * HashAggregate (78) - : +- * HashAggregate (77) - : +- ReusedExchange (76) - +- * HashAggregate (85) - +- Exchange (84) - +- * HashAggregate (83) - +- * HashAggregate (82) - +- ReusedExchange (81) +TakeOrderedAndProject (99) ++- * HashAggregate (98) + +- Exchange (97) + +- * HashAggregate (96) + +- Union (95) + :- * HashAggregate (84) + : +- Exchange (83) + : +- * HashAggregate (82) + : +- Union (81) + : :- * HashAggregate (24) + : : +- Exchange (23) + : : +- * HashAggregate (22) + : : +- * Project (21) + : : +- * BroadcastHashJoin Inner BuildRight (20) + : : :- Union (15) + : : : :- * Project (7) + : : : : +- * BroadcastHashJoin Inner BuildRight (6) + : : : : :- * Project (4) + : : : : : +- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet spark_catalog.default.store_sales (1) + : : : : +- ReusedExchange (5) + : : : +- * Project (14) + : : : +- * BroadcastHashJoin Inner BuildRight (13) + : : : :- * Project (11) + : : : : +- * Filter (10) + : : : : +- * ColumnarToRow (9) + : : : : +- Scan parquet spark_catalog.default.store_returns (8) + : : : +- ReusedExchange (12) + : : +- BroadcastExchange (19) + : : +- * Filter (18) + : : +- * ColumnarToRow (17) + : : +- Scan parquet spark_catalog.default.store (16) + : :- * HashAggregate (48) + : : +- Exchange (47) + : : +- * HashAggregate (46) + : : +- * Project (45) + : : +- * BroadcastHashJoin Inner BuildRight (44) + : : :- Union (39) + : : : :- * Project (31) + : : : : +- * BroadcastHashJoin Inner BuildRight (30) + : : : : :- * Project (28) + : : : : : +- * Filter (27) + : : : : : +- * ColumnarToRow (26) + : : : : : +- Scan parquet spark_catalog.default.catalog_sales (25) + : : : : +- ReusedExchange (29) + : : : +- * Project (38) + : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : :- * Project (35) + : : : : +- * Filter (34) + : : : : +- * ColumnarToRow (33) + : : : : +- Scan parquet spark_catalog.default.catalog_returns (32) + : : : +- ReusedExchange (36) + : : +- BroadcastExchange (43) + : : +- * Filter (42) + : : +- * ColumnarToRow (41) + : : +- Scan parquet spark_catalog.default.catalog_page (40) + : +- * HashAggregate (80) + : +- Exchange (79) + : +- * HashAggregate (78) + : +- * Project (77) + : +- * BroadcastHashJoin Inner BuildRight (76) + : :- Union (71) + : : :- * Project (55) + : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : :- * Project (52) + : : : : +- * Filter (51) + : : : : +- * ColumnarToRow (50) + : : : : +- Scan parquet spark_catalog.default.web_sales (49) + : : : +- ReusedExchange (53) + : : +- * Project (70) + : : +- * BroadcastHashJoin Inner BuildRight (69) + : : :- * Project (67) + : : : +- * SortMergeJoin Inner (66) + : : : :- * Sort (59) + : : : : +- Exchange (58) + : : : : +- * ColumnarToRow (57) + : : : : +- Scan parquet spark_catalog.default.web_returns (56) + : : : +- * Sort (65) + : : : +- Exchange (64) + : : : +- * Project (63) + : : : +- * Filter (62) + : : : +- * ColumnarToRow (61) + : : : +- Scan parquet spark_catalog.default.web_sales (60) + : : +- ReusedExchange (68) + : +- BroadcastExchange (75) + : +- * Filter (74) + : +- * ColumnarToRow (73) + : +- Scan parquet spark_catalog.default.web_site (72) + :- * HashAggregate (89) + : +- Exchange (88) + : +- * HashAggregate (87) + : +- * HashAggregate (86) + : +- ReusedExchange (85) + +- * HashAggregate (94) + +- Exchange (93) + +- * HashAggregate (92) + +- * HashAggregate (91) + +- ReusedExchange (90) (1) Scan parquet spark_catalog.default.store_sales @@ -99,470 +108,509 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 1] +(2) ColumnarToRow [codegen id : 2] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 1] +(3) Filter [codegen id : 2] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] Condition : isnotnull(ss_store_sk#1) -(4) Project [codegen id : 1] +(4) Project [codegen id : 2] Output [6]: [ss_store_sk#1 AS store_sk#6, ss_sold_date_sk#4 AS date_sk#7, ss_ext_sales_price#2 AS sales_price#8, ss_net_profit#3 AS profit#9, 0.00 AS return_amt#10, 0.00 AS net_loss#11] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -(5) Scan parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] +(5) ReusedExchange [Reuses operator id: 104] +Output [1]: [d_date_sk#12] + +(6) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [date_sk#7] +Right keys [1]: [d_date_sk#12] +Join type: Inner +Join condition: None + +(7) Project [codegen id : 2] +Output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] +Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#12] + +(8) Scan parquet spark_catalog.default.store_returns +Output [4]: [sr_store_sk#13, sr_return_amt#14, sr_net_loss#15, sr_returned_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#15), dynamicpruningexpression(sr_returned_date_sk#15 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(sr_returned_date_sk#16), dynamicpruningexpression(sr_returned_date_sk#16 IN dynamicpruning#5)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(6) ColumnarToRow [codegen id : 2] -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] - -(7) Filter [codegen id : 2] -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Condition : isnotnull(sr_store_sk#12) +(9) ColumnarToRow [codegen id : 4] +Input [4]: [sr_store_sk#13, sr_return_amt#14, sr_net_loss#15, sr_returned_date_sk#16] -(8) Project [codegen id : 2] -Output [6]: [sr_store_sk#12 AS store_sk#16, sr_returned_date_sk#15 AS date_sk#17, 0.00 AS sales_price#18, 0.00 AS profit#19, sr_return_amt#13 AS return_amt#20, sr_net_loss#14 AS net_loss#21] -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] +(10) Filter [codegen id : 4] +Input [4]: [sr_store_sk#13, sr_return_amt#14, sr_net_loss#15, sr_returned_date_sk#16] +Condition : isnotnull(sr_store_sk#13) -(9) Union +(11) Project [codegen id : 4] +Output [6]: [sr_store_sk#13 AS store_sk#17, sr_returned_date_sk#16 AS date_sk#18, 0.00 AS sales_price#19, 0.00 AS profit#20, sr_return_amt#14 AS return_amt#21, sr_net_loss#15 AS net_loss#22] +Input [4]: [sr_store_sk#13, sr_return_amt#14, sr_net_loss#15, sr_returned_date_sk#16] -(10) ReusedExchange [Reuses operator id: 95] -Output [1]: [d_date_sk#22] +(12) ReusedExchange [Reuses operator id: 104] +Output [1]: [d_date_sk#23] -(11) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [date_sk#7] -Right keys [1]: [d_date_sk#22] +(13) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [date_sk#18] +Right keys [1]: [d_date_sk#23] Join type: Inner Join condition: None -(12) Project [codegen id : 5] -Output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] -Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] +(14) Project [codegen id : 4] +Output [5]: [store_sk#17, sales_price#19, profit#20, return_amt#21, net_loss#22] +Input [7]: [store_sk#17, date_sk#18, sales_price#19, profit#20, return_amt#21, net_loss#22, d_date_sk#23] -(13) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#23, s_store_id#24] +(15) Union + +(16) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#24, s_store_id#25] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(14) ColumnarToRow [codegen id : 4] -Input [2]: [s_store_sk#23, s_store_id#24] +(17) ColumnarToRow [codegen id : 5] +Input [2]: [s_store_sk#24, s_store_id#25] -(15) Filter [codegen id : 4] -Input [2]: [s_store_sk#23, s_store_id#24] -Condition : isnotnull(s_store_sk#23) +(18) Filter [codegen id : 5] +Input [2]: [s_store_sk#24, s_store_id#25] +Condition : isnotnull(s_store_sk#24) -(16) BroadcastExchange -Input [2]: [s_store_sk#23, s_store_id#24] +(19) BroadcastExchange +Input [2]: [s_store_sk#24, s_store_id#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(17) BroadcastHashJoin [codegen id : 5] +(20) BroadcastHashJoin [codegen id : 6] Left keys [1]: [store_sk#6] -Right keys [1]: [s_store_sk#23] +Right keys [1]: [s_store_sk#24] Join type: Inner Join condition: None -(18) Project [codegen id : 5] -Output [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#24] -Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#23, s_store_id#24] +(21) Project [codegen id : 6] +Output [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] +Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#24, s_store_id#25] -(19) HashAggregate [codegen id : 5] -Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#24] -Keys [1]: [s_store_id#24] +(22) HashAggregate [codegen id : 6] +Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] +Keys [1]: [s_store_id#25] Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum#25, sum#26, sum#27, sum#28] -Results [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] +Aggregate Attributes [4]: [sum#26, sum#27, sum#28, sum#29] +Results [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] -(20) Exchange -Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] -Arguments: hashpartitioning(s_store_id#24, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(23) Exchange +Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] +Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(21) HashAggregate [codegen id : 6] -Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] -Keys [1]: [s_store_id#24] +(24) HashAggregate [codegen id : 7] +Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] +Keys [1]: [s_store_id#25] Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#33, sum(UnscaledValue(return_amt#10))#34, sum(UnscaledValue(profit#9))#35, sum(UnscaledValue(net_loss#11))#36] -Results [5]: [store channel AS channel#37, concat(store, s_store_id#24) AS id#38, MakeDecimal(sum(UnscaledValue(sales_price#8))#33,17,2) AS sales#39, MakeDecimal(sum(UnscaledValue(return_amt#10))#34,17,2) AS returns#40, (MakeDecimal(sum(UnscaledValue(profit#9))#35,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#36,17,2)) AS profit#41] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#34, sum(UnscaledValue(return_amt#10))#35, sum(UnscaledValue(profit#9))#36, sum(UnscaledValue(net_loss#11))#37] +Results [5]: [store channel AS channel#38, concat(store, s_store_id#25) AS id#39, MakeDecimal(sum(UnscaledValue(sales_price#8))#34,17,2) AS sales#40, MakeDecimal(sum(UnscaledValue(return_amt#10))#35,17,2) AS returns#41, (MakeDecimal(sum(UnscaledValue(profit#9))#36,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#37,17,2)) AS profit#42] -(22) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +(25) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#45), dynamicpruningexpression(cs_sold_date_sk#45 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#46), dynamicpruningexpression(cs_sold_date_sk#46 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct -(23) ColumnarToRow [codegen id : 7] -Input [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +(26) ColumnarToRow [codegen id : 9] +Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] + +(27) Filter [codegen id : 9] +Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] +Condition : isnotnull(cs_catalog_page_sk#43) -(24) Filter [codegen id : 7] -Input [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Condition : isnotnull(cs_catalog_page_sk#42) +(28) Project [codegen id : 9] +Output [6]: [cs_catalog_page_sk#43 AS page_sk#47, cs_sold_date_sk#46 AS date_sk#48, cs_ext_sales_price#44 AS sales_price#49, cs_net_profit#45 AS profit#50, 0.00 AS return_amt#51, 0.00 AS net_loss#52] +Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] + +(29) ReusedExchange [Reuses operator id: 104] +Output [1]: [d_date_sk#53] + +(30) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [date_sk#48] +Right keys [1]: [d_date_sk#53] +Join type: Inner +Join condition: None -(25) Project [codegen id : 7] -Output [6]: [cs_catalog_page_sk#42 AS page_sk#46, cs_sold_date_sk#45 AS date_sk#47, cs_ext_sales_price#43 AS sales_price#48, cs_net_profit#44 AS profit#49, 0.00 AS return_amt#50, 0.00 AS net_loss#51] -Input [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +(31) Project [codegen id : 9] +Output [5]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52] +Input [7]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52, d_date_sk#53] -(26) Scan parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#52, cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] +(32) Scan parquet spark_catalog.default.catalog_returns +Output [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#55), dynamicpruningexpression(cr_returned_date_sk#55 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cr_returned_date_sk#57), dynamicpruningexpression(cr_returned_date_sk#57 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct -(27) ColumnarToRow [codegen id : 8] -Input [4]: [cr_catalog_page_sk#52, cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] +(33) ColumnarToRow [codegen id : 11] +Input [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] -(28) Filter [codegen id : 8] -Input [4]: [cr_catalog_page_sk#52, cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] -Condition : isnotnull(cr_catalog_page_sk#52) +(34) Filter [codegen id : 11] +Input [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] +Condition : isnotnull(cr_catalog_page_sk#54) -(29) Project [codegen id : 8] -Output [6]: [cr_catalog_page_sk#52 AS page_sk#56, cr_returned_date_sk#55 AS date_sk#57, 0.00 AS sales_price#58, 0.00 AS profit#59, cr_return_amount#53 AS return_amt#60, cr_net_loss#54 AS net_loss#61] -Input [4]: [cr_catalog_page_sk#52, cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] +(35) Project [codegen id : 11] +Output [6]: [cr_catalog_page_sk#54 AS page_sk#58, cr_returned_date_sk#57 AS date_sk#59, 0.00 AS sales_price#60, 0.00 AS profit#61, cr_return_amount#55 AS return_amt#62, cr_net_loss#56 AS net_loss#63] +Input [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] -(30) Union +(36) ReusedExchange [Reuses operator id: 104] +Output [1]: [d_date_sk#64] -(31) ReusedExchange [Reuses operator id: 95] -Output [1]: [d_date_sk#62] - -(32) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [date_sk#47] -Right keys [1]: [d_date_sk#62] +(37) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [date_sk#59] +Right keys [1]: [d_date_sk#64] Join type: Inner Join condition: None -(33) Project [codegen id : 11] -Output [5]: [page_sk#46, sales_price#48, profit#49, return_amt#50, net_loss#51] -Input [7]: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51, d_date_sk#62] +(38) Project [codegen id : 11] +Output [5]: [page_sk#58, sales_price#60, profit#61, return_amt#62, net_loss#63] +Input [7]: [page_sk#58, date_sk#59, sales_price#60, profit#61, return_amt#62, net_loss#63, d_date_sk#64] + +(39) Union -(34) Scan parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#63, cp_catalog_page_id#64] +(40) Scan parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(35) ColumnarToRow [codegen id : 10] -Input [2]: [cp_catalog_page_sk#63, cp_catalog_page_id#64] +(41) ColumnarToRow [codegen id : 12] +Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] -(36) Filter [codegen id : 10] -Input [2]: [cp_catalog_page_sk#63, cp_catalog_page_id#64] -Condition : isnotnull(cp_catalog_page_sk#63) +(42) Filter [codegen id : 12] +Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] +Condition : isnotnull(cp_catalog_page_sk#65) -(37) BroadcastExchange -Input [2]: [cp_catalog_page_sk#63, cp_catalog_page_id#64] +(43) BroadcastExchange +Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(38) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [page_sk#46] -Right keys [1]: [cp_catalog_page_sk#63] +(44) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [page_sk#47] +Right keys [1]: [cp_catalog_page_sk#65] Join type: Inner Join condition: None -(39) Project [codegen id : 11] -Output [5]: [sales_price#48, profit#49, return_amt#50, net_loss#51, cp_catalog_page_id#64] -Input [7]: [page_sk#46, sales_price#48, profit#49, return_amt#50, net_loss#51, cp_catalog_page_sk#63, cp_catalog_page_id#64] - -(40) HashAggregate [codegen id : 11] -Input [5]: [sales_price#48, profit#49, return_amt#50, net_loss#51, cp_catalog_page_id#64] -Keys [1]: [cp_catalog_page_id#64] -Functions [4]: [partial_sum(UnscaledValue(sales_price#48)), partial_sum(UnscaledValue(return_amt#50)), partial_sum(UnscaledValue(profit#49)), partial_sum(UnscaledValue(net_loss#51))] -Aggregate Attributes [4]: [sum#65, sum#66, sum#67, sum#68] -Results [5]: [cp_catalog_page_id#64, sum#69, sum#70, sum#71, sum#72] - -(41) Exchange -Input [5]: [cp_catalog_page_id#64, sum#69, sum#70, sum#71, sum#72] -Arguments: hashpartitioning(cp_catalog_page_id#64, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(42) HashAggregate [codegen id : 12] -Input [5]: [cp_catalog_page_id#64, sum#69, sum#70, sum#71, sum#72] -Keys [1]: [cp_catalog_page_id#64] -Functions [4]: [sum(UnscaledValue(sales_price#48)), sum(UnscaledValue(return_amt#50)), sum(UnscaledValue(profit#49)), sum(UnscaledValue(net_loss#51))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#48))#73, sum(UnscaledValue(return_amt#50))#74, sum(UnscaledValue(profit#49))#75, sum(UnscaledValue(net_loss#51))#76] -Results [5]: [catalog channel AS channel#77, concat(catalog_page, cp_catalog_page_id#64) AS id#78, MakeDecimal(sum(UnscaledValue(sales_price#48))#73,17,2) AS sales#79, MakeDecimal(sum(UnscaledValue(return_amt#50))#74,17,2) AS returns#80, (MakeDecimal(sum(UnscaledValue(profit#49))#75,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#51))#76,17,2)) AS profit#81] - -(43) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#82, ws_ext_sales_price#83, ws_net_profit#84, ws_sold_date_sk#85] +(45) Project [codegen id : 13] +Output [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#66] +Input [7]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_sk#65, cp_catalog_page_id#66] + +(46) HashAggregate [codegen id : 13] +Input [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#66] +Keys [1]: [cp_catalog_page_id#66] +Functions [4]: [partial_sum(UnscaledValue(sales_price#49)), partial_sum(UnscaledValue(return_amt#51)), partial_sum(UnscaledValue(profit#50)), partial_sum(UnscaledValue(net_loss#52))] +Aggregate Attributes [4]: [sum#67, sum#68, sum#69, sum#70] +Results [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] + +(47) Exchange +Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] +Arguments: hashpartitioning(cp_catalog_page_id#66, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(48) HashAggregate [codegen id : 14] +Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] +Keys [1]: [cp_catalog_page_id#66] +Functions [4]: [sum(UnscaledValue(sales_price#49)), sum(UnscaledValue(return_amt#51)), sum(UnscaledValue(profit#50)), sum(UnscaledValue(net_loss#52))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#49))#75, sum(UnscaledValue(return_amt#51))#76, sum(UnscaledValue(profit#50))#77, sum(UnscaledValue(net_loss#52))#78] +Results [5]: [catalog channel AS channel#79, concat(catalog_page, cp_catalog_page_id#66) AS id#80, MakeDecimal(sum(UnscaledValue(sales_price#49))#75,17,2) AS sales#81, MakeDecimal(sum(UnscaledValue(return_amt#51))#76,17,2) AS returns#82, (MakeDecimal(sum(UnscaledValue(profit#50))#77,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#52))#78,17,2)) AS profit#83] + +(49) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#85), dynamicpruningexpression(ws_sold_date_sk#85 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#87), dynamicpruningexpression(ws_sold_date_sk#87 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(44) ColumnarToRow [codegen id : 13] -Input [4]: [ws_web_site_sk#82, ws_ext_sales_price#83, ws_net_profit#84, ws_sold_date_sk#85] +(50) ColumnarToRow [codegen id : 16] +Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] + +(51) Filter [codegen id : 16] +Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] +Condition : isnotnull(ws_web_site_sk#84) + +(52) Project [codegen id : 16] +Output [6]: [ws_web_site_sk#84 AS wsr_web_site_sk#88, ws_sold_date_sk#87 AS date_sk#89, ws_ext_sales_price#85 AS sales_price#90, ws_net_profit#86 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] +Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] -(45) Filter [codegen id : 13] -Input [4]: [ws_web_site_sk#82, ws_ext_sales_price#83, ws_net_profit#84, ws_sold_date_sk#85] -Condition : isnotnull(ws_web_site_sk#82) +(53) ReusedExchange [Reuses operator id: 104] +Output [1]: [d_date_sk#94] -(46) Project [codegen id : 13] -Output [6]: [ws_web_site_sk#82 AS wsr_web_site_sk#86, ws_sold_date_sk#85 AS date_sk#87, ws_ext_sales_price#83 AS sales_price#88, ws_net_profit#84 AS profit#89, 0.00 AS return_amt#90, 0.00 AS net_loss#91] -Input [4]: [ws_web_site_sk#82, ws_ext_sales_price#83, ws_net_profit#84, ws_sold_date_sk#85] +(54) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [date_sk#89] +Right keys [1]: [d_date_sk#94] +Join type: Inner +Join condition: None -(47) Scan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96] +(55) Project [codegen id : 16] +Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] +Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#94] + +(56) Scan parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#95, wr_order_number#96, wr_return_amt#97, wr_net_loss#98, wr_returned_date_sk#99] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#96), dynamicpruningexpression(wr_returned_date_sk#96 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(wr_returned_date_sk#99), dynamicpruningexpression(wr_returned_date_sk#99 IN dynamicpruning#5)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 14] -Input [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96] +(57) ColumnarToRow [codegen id : 17] +Input [5]: [wr_item_sk#95, wr_order_number#96, wr_return_amt#97, wr_net_loss#98, wr_returned_date_sk#99] -(49) Exchange -Input [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96] -Arguments: hashpartitioning(wr_item_sk#92, wr_order_number#93, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(58) Exchange +Input [5]: [wr_item_sk#95, wr_order_number#96, wr_return_amt#97, wr_net_loss#98, wr_returned_date_sk#99] +Arguments: hashpartitioning(wr_item_sk#95, wr_order_number#96, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(50) Sort [codegen id : 15] -Input [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96] -Arguments: [wr_item_sk#92 ASC NULLS FIRST, wr_order_number#93 ASC NULLS FIRST], false, 0 +(59) Sort [codegen id : 18] +Input [5]: [wr_item_sk#95, wr_order_number#96, wr_return_amt#97, wr_net_loss#98, wr_returned_date_sk#99] +Arguments: [wr_item_sk#95 ASC NULLS FIRST, wr_order_number#96 ASC NULLS FIRST], false, 0 -(51) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] +(60) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102, ws_sold_date_sk#103] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(52) ColumnarToRow [codegen id : 16] -Input [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] +(61) ColumnarToRow [codegen id : 19] +Input [4]: [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102, ws_sold_date_sk#103] -(53) Filter [codegen id : 16] -Input [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] -Condition : ((isnotnull(ws_item_sk#97) AND isnotnull(ws_order_number#99)) AND isnotnull(ws_web_site_sk#98)) +(62) Filter [codegen id : 19] +Input [4]: [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102, ws_sold_date_sk#103] +Condition : ((isnotnull(ws_item_sk#100) AND isnotnull(ws_order_number#102)) AND isnotnull(ws_web_site_sk#101)) -(54) Project [codegen id : 16] -Output [3]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99] -Input [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] +(63) Project [codegen id : 19] +Output [3]: [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102] +Input [4]: [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102, ws_sold_date_sk#103] -(55) Exchange -Input [3]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99] -Arguments: hashpartitioning(ws_item_sk#97, ws_order_number#99, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(64) Exchange +Input [3]: [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102] +Arguments: hashpartitioning(ws_item_sk#100, ws_order_number#102, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(56) Sort [codegen id : 17] -Input [3]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99] -Arguments: [ws_item_sk#97 ASC NULLS FIRST, ws_order_number#99 ASC NULLS FIRST], false, 0 +(65) Sort [codegen id : 20] +Input [3]: [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102] +Arguments: [ws_item_sk#100 ASC NULLS FIRST, ws_order_number#102 ASC NULLS FIRST], false, 0 -(57) SortMergeJoin [codegen id : 18] -Left keys [2]: [wr_item_sk#92, wr_order_number#93] -Right keys [2]: [ws_item_sk#97, ws_order_number#99] +(66) SortMergeJoin [codegen id : 22] +Left keys [2]: [wr_item_sk#95, wr_order_number#96] +Right keys [2]: [ws_item_sk#100, ws_order_number#102] Join type: Inner Join condition: None -(58) Project [codegen id : 18] -Output [6]: [ws_web_site_sk#98 AS wsr_web_site_sk#101, wr_returned_date_sk#96 AS date_sk#102, 0.00 AS sales_price#103, 0.00 AS profit#104, wr_return_amt#94 AS return_amt#105, wr_net_loss#95 AS net_loss#106] -Input [8]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96, ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99] - -(59) Union +(67) Project [codegen id : 22] +Output [6]: [ws_web_site_sk#101 AS wsr_web_site_sk#104, wr_returned_date_sk#99 AS date_sk#105, 0.00 AS sales_price#106, 0.00 AS profit#107, wr_return_amt#97 AS return_amt#108, wr_net_loss#98 AS net_loss#109] +Input [8]: [wr_item_sk#95, wr_order_number#96, wr_return_amt#97, wr_net_loss#98, wr_returned_date_sk#99, ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102] -(60) ReusedExchange [Reuses operator id: 95] -Output [1]: [d_date_sk#107] +(68) ReusedExchange [Reuses operator id: 104] +Output [1]: [d_date_sk#110] -(61) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [date_sk#87] -Right keys [1]: [d_date_sk#107] +(69) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [date_sk#105] +Right keys [1]: [d_date_sk#110] Join type: Inner Join condition: None -(62) Project [codegen id : 21] -Output [5]: [wsr_web_site_sk#86, sales_price#88, profit#89, return_amt#90, net_loss#91] -Input [7]: [wsr_web_site_sk#86, date_sk#87, sales_price#88, profit#89, return_amt#90, net_loss#91, d_date_sk#107] +(70) Project [codegen id : 22] +Output [5]: [wsr_web_site_sk#104, sales_price#106, profit#107, return_amt#108, net_loss#109] +Input [7]: [wsr_web_site_sk#104, date_sk#105, sales_price#106, profit#107, return_amt#108, net_loss#109, d_date_sk#110] -(63) Scan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#108, web_site_id#109] +(71) Union + +(72) Scan parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#111, web_site_id#112] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(64) ColumnarToRow [codegen id : 20] -Input [2]: [web_site_sk#108, web_site_id#109] +(73) ColumnarToRow [codegen id : 23] +Input [2]: [web_site_sk#111, web_site_id#112] -(65) Filter [codegen id : 20] -Input [2]: [web_site_sk#108, web_site_id#109] -Condition : isnotnull(web_site_sk#108) +(74) Filter [codegen id : 23] +Input [2]: [web_site_sk#111, web_site_id#112] +Condition : isnotnull(web_site_sk#111) -(66) BroadcastExchange -Input [2]: [web_site_sk#108, web_site_id#109] +(75) BroadcastExchange +Input [2]: [web_site_sk#111, web_site_id#112] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(67) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [wsr_web_site_sk#86] -Right keys [1]: [web_site_sk#108] +(76) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [wsr_web_site_sk#88] +Right keys [1]: [web_site_sk#111] Join type: Inner Join condition: None -(68) Project [codegen id : 21] -Output [5]: [sales_price#88, profit#89, return_amt#90, net_loss#91, web_site_id#109] -Input [7]: [wsr_web_site_sk#86, sales_price#88, profit#89, return_amt#90, net_loss#91, web_site_sk#108, web_site_id#109] - -(69) HashAggregate [codegen id : 21] -Input [5]: [sales_price#88, profit#89, return_amt#90, net_loss#91, web_site_id#109] -Keys [1]: [web_site_id#109] -Functions [4]: [partial_sum(UnscaledValue(sales_price#88)), partial_sum(UnscaledValue(return_amt#90)), partial_sum(UnscaledValue(profit#89)), partial_sum(UnscaledValue(net_loss#91))] -Aggregate Attributes [4]: [sum#110, sum#111, sum#112, sum#113] -Results [5]: [web_site_id#109, sum#114, sum#115, sum#116, sum#117] - -(70) Exchange -Input [5]: [web_site_id#109, sum#114, sum#115, sum#116, sum#117] -Arguments: hashpartitioning(web_site_id#109, 5), ENSURE_REQUIREMENTS, [plan_id=8] - -(71) HashAggregate [codegen id : 22] -Input [5]: [web_site_id#109, sum#114, sum#115, sum#116, sum#117] -Keys [1]: [web_site_id#109] -Functions [4]: [sum(UnscaledValue(sales_price#88)), sum(UnscaledValue(return_amt#90)), sum(UnscaledValue(profit#89)), sum(UnscaledValue(net_loss#91))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#88))#118, sum(UnscaledValue(return_amt#90))#119, sum(UnscaledValue(profit#89))#120, sum(UnscaledValue(net_loss#91))#121] -Results [5]: [web channel AS channel#122, concat(web_site, web_site_id#109) AS id#123, MakeDecimal(sum(UnscaledValue(sales_price#88))#118,17,2) AS sales#124, MakeDecimal(sum(UnscaledValue(return_amt#90))#119,17,2) AS returns#125, (MakeDecimal(sum(UnscaledValue(profit#89))#120,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#91))#121,17,2)) AS profit#126] - -(72) Union - -(73) HashAggregate [codegen id : 23] -Input [5]: [channel#37, id#38, sales#39, returns#40, profit#41] -Keys [2]: [channel#37, id#38] -Functions [3]: [partial_sum(sales#39), partial_sum(returns#40), partial_sum(profit#41)] -Aggregate Attributes [6]: [sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Results [8]: [channel#37, id#38, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] - -(74) Exchange -Input [8]: [channel#37, id#38, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] -Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, [plan_id=9] - -(75) HashAggregate [codegen id : 24] -Input [8]: [channel#37, id#38, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] -Keys [2]: [channel#37, id#38] -Functions [3]: [sum(sales#39), sum(returns#40), sum(profit#41)] -Aggregate Attributes [3]: [sum(sales#39)#139, sum(returns#40)#140, sum(profit#41)#141] -Results [5]: [channel#37, id#38, cast(sum(sales#39)#139 as decimal(37,2)) AS sales#142, cast(sum(returns#40)#140 as decimal(37,2)) AS returns#143, cast(sum(profit#41)#141 as decimal(38,2)) AS profit#144] - -(76) ReusedExchange [Reuses operator id: 74] -Output [8]: [channel#145, id#146, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] - -(77) HashAggregate [codegen id : 48] -Input [8]: [channel#145, id#146, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] -Keys [2]: [channel#145, id#146] -Functions [3]: [sum(sales#153), sum(returns#154), sum(profit#155)] -Aggregate Attributes [3]: [sum(sales#153)#139, sum(returns#154)#140, sum(profit#155)#141] -Results [4]: [channel#145, sum(sales#153)#139 AS sales#156, sum(returns#154)#140 AS returns#157, sum(profit#155)#141 AS profit#158] - -(78) HashAggregate [codegen id : 48] -Input [4]: [channel#145, sales#156, returns#157, profit#158] -Keys [1]: [channel#145] -Functions [3]: [partial_sum(sales#156), partial_sum(returns#157), partial_sum(profit#158)] -Aggregate Attributes [6]: [sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] -Results [7]: [channel#145, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] +(77) Project [codegen id : 24] +Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#112] +Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#111, web_site_id#112] -(79) Exchange -Input [7]: [channel#145, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] -Arguments: hashpartitioning(channel#145, 5), ENSURE_REQUIREMENTS, [plan_id=10] +(78) HashAggregate [codegen id : 24] +Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#112] +Keys [1]: [web_site_id#112] +Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] +Aggregate Attributes [4]: [sum#113, sum#114, sum#115, sum#116] +Results [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] -(80) HashAggregate [codegen id : 49] -Input [7]: [channel#145, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] -Keys [1]: [channel#145] +(79) Exchange +Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] +Arguments: hashpartitioning(web_site_id#112, 5), ENSURE_REQUIREMENTS, [plan_id=8] + +(80) HashAggregate [codegen id : 25] +Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] +Keys [1]: [web_site_id#112] +Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#90))#121, sum(UnscaledValue(return_amt#92))#122, sum(UnscaledValue(profit#91))#123, sum(UnscaledValue(net_loss#93))#124] +Results [5]: [web channel AS channel#125, concat(web_site, web_site_id#112) AS id#126, MakeDecimal(sum(UnscaledValue(sales_price#90))#121,17,2) AS sales#127, MakeDecimal(sum(UnscaledValue(return_amt#92))#122,17,2) AS returns#128, (MakeDecimal(sum(UnscaledValue(profit#91))#123,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#93))#124,17,2)) AS profit#129] + +(81) Union + +(82) HashAggregate [codegen id : 26] +Input [5]: [channel#38, id#39, sales#40, returns#41, profit#42] +Keys [2]: [channel#38, id#39] +Functions [3]: [partial_sum(sales#40), partial_sum(returns#41), partial_sum(profit#42)] +Aggregate Attributes [6]: [sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135] +Results [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] + +(83) Exchange +Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] +Arguments: hashpartitioning(channel#38, id#39, 5), ENSURE_REQUIREMENTS, [plan_id=9] + +(84) HashAggregate [codegen id : 27] +Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] +Keys [2]: [channel#38, id#39] +Functions [3]: [sum(sales#40), sum(returns#41), sum(profit#42)] +Aggregate Attributes [3]: [sum(sales#40)#142, sum(returns#41)#143, sum(profit#42)#144] +Results [5]: [channel#38, id#39, cast(sum(sales#40)#142 as decimal(37,2)) AS sales#145, cast(sum(returns#41)#143 as decimal(37,2)) AS returns#146, cast(sum(profit#42)#144 as decimal(38,2)) AS profit#147] + +(85) ReusedExchange [Reuses operator id: 83] +Output [8]: [channel#148, id#149, sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155] + +(86) HashAggregate [codegen id : 54] +Input [8]: [channel#148, id#149, sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155] +Keys [2]: [channel#148, id#149] Functions [3]: [sum(sales#156), sum(returns#157), sum(profit#158)] -Aggregate Attributes [3]: [sum(sales#156)#171, sum(returns#157)#172, sum(profit#158)#173] -Results [5]: [channel#145, null AS id#174, sum(sales#156)#171 AS sum(sales)#175, sum(returns#157)#172 AS sum(returns)#176, sum(profit#158)#173 AS sum(profit)#177] +Aggregate Attributes [3]: [sum(sales#156)#142, sum(returns#157)#143, sum(profit#158)#144] +Results [4]: [channel#148, sum(sales#156)#142 AS sales#159, sum(returns#157)#143 AS returns#160, sum(profit#158)#144 AS profit#161] -(81) ReusedExchange [Reuses operator id: 74] -Output [8]: [channel#178, id#179, sum#180, isEmpty#181, sum#182, isEmpty#183, sum#184, isEmpty#185] +(87) HashAggregate [codegen id : 54] +Input [4]: [channel#148, sales#159, returns#160, profit#161] +Keys [1]: [channel#148] +Functions [3]: [partial_sum(sales#159), partial_sum(returns#160), partial_sum(profit#161)] +Aggregate Attributes [6]: [sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] +Results [7]: [channel#148, sum#168, isEmpty#169, sum#170, isEmpty#171, sum#172, isEmpty#173] -(82) HashAggregate [codegen id : 73] -Input [8]: [channel#178, id#179, sum#180, isEmpty#181, sum#182, isEmpty#183, sum#184, isEmpty#185] -Keys [2]: [channel#178, id#179] -Functions [3]: [sum(sales#186), sum(returns#187), sum(profit#188)] -Aggregate Attributes [3]: [sum(sales#186)#139, sum(returns#187)#140, sum(profit#188)#141] -Results [3]: [sum(sales#186)#139 AS sales#189, sum(returns#187)#140 AS returns#190, sum(profit#188)#141 AS profit#191] +(88) Exchange +Input [7]: [channel#148, sum#168, isEmpty#169, sum#170, isEmpty#171, sum#172, isEmpty#173] +Arguments: hashpartitioning(channel#148, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(89) HashAggregate [codegen id : 55] +Input [7]: [channel#148, sum#168, isEmpty#169, sum#170, isEmpty#171, sum#172, isEmpty#173] +Keys [1]: [channel#148] +Functions [3]: [sum(sales#159), sum(returns#160), sum(profit#161)] +Aggregate Attributes [3]: [sum(sales#159)#174, sum(returns#160)#175, sum(profit#161)#176] +Results [5]: [channel#148, null AS id#177, sum(sales#159)#174 AS sum(sales)#178, sum(returns#160)#175 AS sum(returns)#179, sum(profit#161)#176 AS sum(profit)#180] + +(90) ReusedExchange [Reuses operator id: 83] +Output [8]: [channel#181, id#182, sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188] + +(91) HashAggregate [codegen id : 82] +Input [8]: [channel#181, id#182, sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188] +Keys [2]: [channel#181, id#182] +Functions [3]: [sum(sales#189), sum(returns#190), sum(profit#191)] +Aggregate Attributes [3]: [sum(sales#189)#142, sum(returns#190)#143, sum(profit#191)#144] +Results [3]: [sum(sales#189)#142 AS sales#192, sum(returns#190)#143 AS returns#193, sum(profit#191)#144 AS profit#194] -(83) HashAggregate [codegen id : 73] -Input [3]: [sales#189, returns#190, profit#191] +(92) HashAggregate [codegen id : 82] +Input [3]: [sales#192, returns#193, profit#194] Keys: [] -Functions [3]: [partial_sum(sales#189), partial_sum(returns#190), partial_sum(profit#191)] -Aggregate Attributes [6]: [sum#192, isEmpty#193, sum#194, isEmpty#195, sum#196, isEmpty#197] -Results [6]: [sum#198, isEmpty#199, sum#200, isEmpty#201, sum#202, isEmpty#203] +Functions [3]: [partial_sum(sales#192), partial_sum(returns#193), partial_sum(profit#194)] +Aggregate Attributes [6]: [sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200] +Results [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] -(84) Exchange -Input [6]: [sum#198, isEmpty#199, sum#200, isEmpty#201, sum#202, isEmpty#203] +(93) Exchange +Input [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=11] -(85) HashAggregate [codegen id : 74] -Input [6]: [sum#198, isEmpty#199, sum#200, isEmpty#201, sum#202, isEmpty#203] +(94) HashAggregate [codegen id : 83] +Input [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] Keys: [] -Functions [3]: [sum(sales#189), sum(returns#190), sum(profit#191)] -Aggregate Attributes [3]: [sum(sales#189)#204, sum(returns#190)#205, sum(profit#191)#206] -Results [5]: [null AS channel#207, null AS id#208, sum(sales#189)#204 AS sum(sales)#209, sum(returns#190)#205 AS sum(returns)#210, sum(profit#191)#206 AS sum(profit)#211] +Functions [3]: [sum(sales#192), sum(returns#193), sum(profit#194)] +Aggregate Attributes [3]: [sum(sales#192)#207, sum(returns#193)#208, sum(profit#194)#209] +Results [5]: [null AS channel#210, null AS id#211, sum(sales#192)#207 AS sum(sales)#212, sum(returns#193)#208 AS sum(returns)#213, sum(profit#194)#209 AS sum(profit)#214] -(86) Union +(95) Union -(87) HashAggregate [codegen id : 75] -Input [5]: [channel#37, id#38, sales#142, returns#143, profit#144] -Keys [5]: [channel#37, id#38, sales#142, returns#143, profit#144] +(96) HashAggregate [codegen id : 84] +Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] +Keys [5]: [channel#38, id#39, sales#145, returns#146, profit#147] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#37, id#38, sales#142, returns#143, profit#144] +Results [5]: [channel#38, id#39, sales#145, returns#146, profit#147] -(88) Exchange -Input [5]: [channel#37, id#38, sales#142, returns#143, profit#144] -Arguments: hashpartitioning(channel#37, id#38, sales#142, returns#143, profit#144, 5), ENSURE_REQUIREMENTS, [plan_id=12] +(97) Exchange +Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] +Arguments: hashpartitioning(channel#38, id#39, sales#145, returns#146, profit#147, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(89) HashAggregate [codegen id : 76] -Input [5]: [channel#37, id#38, sales#142, returns#143, profit#144] -Keys [5]: [channel#37, id#38, sales#142, returns#143, profit#144] +(98) HashAggregate [codegen id : 85] +Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] +Keys [5]: [channel#38, id#39, sales#145, returns#146, profit#147] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#37, id#38, sales#142, returns#143, profit#144] +Results [5]: [channel#38, id#39, sales#145, returns#146, profit#147] -(90) TakeOrderedAndProject -Input [5]: [channel#37, id#38, sales#142, returns#143, profit#144] -Arguments: 100, [channel#37 ASC NULLS FIRST, id#38 ASC NULLS FIRST], [channel#37, id#38, sales#142, returns#143, profit#144] +(99) TakeOrderedAndProject +Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] +Arguments: 100, [channel#38 ASC NULLS FIRST, id#39 ASC NULLS FIRST], [channel#38, id#39, sales#145, returns#146, profit#147] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (95) -+- * Project (94) - +- * Filter (93) - +- * ColumnarToRow (92) - +- Scan parquet spark_catalog.default.date_dim (91) +BroadcastExchange (104) ++- * Project (103) + +- * Filter (102) + +- * ColumnarToRow (101) + +- Scan parquet spark_catalog.default.date_dim (100) -(91) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#212] +(100) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#12, d_date#215] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] ReadSchema: struct -(92) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#22, d_date#212] +(101) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#12, d_date#215] -(93) Filter [codegen id : 1] -Input [2]: [d_date_sk#22, d_date#212] -Condition : (((isnotnull(d_date#212) AND (d_date#212 >= 1998-08-04)) AND (d_date#212 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) +(102) Filter [codegen id : 1] +Input [2]: [d_date_sk#12, d_date#215] +Condition : (((isnotnull(d_date#215) AND (d_date#215 >= 1998-08-04)) AND (d_date#215 <= 1998-08-18)) AND isnotnull(d_date_sk#12)) -(94) Project [codegen id : 1] -Output [1]: [d_date_sk#22] -Input [2]: [d_date_sk#22, d_date#212] +(103) Project [codegen id : 1] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_date#215] -(95) BroadcastExchange -Input [1]: [d_date_sk#22] +(104) BroadcastExchange +Input [1]: [d_date_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:2 Hosting operator id = 5 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 8 Hosting Expression = sr_returned_date_sk#16 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 25 Hosting Expression = cs_sold_date_sk#46 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#55 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 32 Hosting Expression = cr_returned_date_sk#57 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#85 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 49 Hosting Expression = ws_sold_date_sk#87 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 47 Hosting Expression = wr_returned_date_sk#96 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 56 Hosting Expression = wr_returned_date_sk#99 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/simplified.txt index d4833851f01ee..29b7b093ef4b6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/simplified.txt @@ -1,33 +1,33 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (76) + WholeStageCodegen (85) HashAggregate [channel,id,sales,returns,profit] InputAdapter Exchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (75) + WholeStageCodegen (84) HashAggregate [channel,id,sales,returns,profit] InputAdapter Union - WholeStageCodegen (24) + WholeStageCodegen (27) HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel,id] #2 - WholeStageCodegen (23) + WholeStageCodegen (26) HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (6) + WholeStageCodegen (7) HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] InputAdapter Exchange [s_store_id] #3 - WholeStageCodegen (5) + WholeStageCodegen (6) HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] Project [sales_price,profit,return_amt,net_loss,s_store_id] BroadcastHashJoin [store_sk,s_store_sk] - Project [store_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (1) + InputAdapter + Union + WholeStageCodegen (2) + Project [store_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] Project [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] Filter [ss_store_sk] ColumnarToRow @@ -41,123 +41,135 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - WholeStageCodegen (2) + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (4) + Project [store_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] Project [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] Filter [sr_store_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 + InputAdapter + ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #5 - WholeStageCodegen (4) + WholeStageCodegen (5) Filter [s_store_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store [s_store_sk,s_store_id] - WholeStageCodegen (12) + WholeStageCodegen (14) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] InputAdapter Exchange [cp_catalog_page_id] #6 - WholeStageCodegen (11) + WholeStageCodegen (13) HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] BroadcastHashJoin [page_sk,cp_catalog_page_sk] - Project [page_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (7) + InputAdapter + Union + WholeStageCodegen (9) + Project [page_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] Project [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] Filter [cs_catalog_page_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (8) + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (11) + Project [page_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] Project [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] Filter [cr_catalog_page_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 + InputAdapter + ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #7 - WholeStageCodegen (10) + WholeStageCodegen (12) Filter [cp_catalog_page_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (22) + WholeStageCodegen (25) HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] InputAdapter Exchange [web_site_id] #8 - WholeStageCodegen (21) + WholeStageCodegen (24) HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] Project [sales_price,profit,return_amt,net_loss,web_site_id] BroadcastHashJoin [wsr_web_site_sk,web_site_sk] - Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (13) + InputAdapter + Union + WholeStageCodegen (16) + Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] Project [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] Filter [ws_web_site_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (18) + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (22) + Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] Project [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] SortMergeJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] InputAdapter - WholeStageCodegen (15) + WholeStageCodegen (18) Sort [wr_item_sk,wr_order_number] InputAdapter Exchange [wr_item_sk,wr_order_number] #9 - WholeStageCodegen (14) + WholeStageCodegen (17) ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter - WholeStageCodegen (17) + WholeStageCodegen (20) Sort [ws_item_sk,ws_order_number] InputAdapter Exchange [ws_item_sk,ws_order_number] #10 - WholeStageCodegen (16) + WholeStageCodegen (19) Project [ws_item_sk,ws_web_site_sk,ws_order_number] Filter [ws_item_sk,ws_order_number,ws_web_site_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #4 + InputAdapter + ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #11 - WholeStageCodegen (20) + WholeStageCodegen (23) Filter [web_site_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - WholeStageCodegen (49) + WholeStageCodegen (55) HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel] #12 - WholeStageCodegen (48) + WholeStageCodegen (54) HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (74) + WholeStageCodegen (83) HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange #13 - WholeStageCodegen (73) + WholeStageCodegen (82) HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt index 34c6ecf3cf2fa..d361cdd32281e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt @@ -1,91 +1,100 @@ == Physical Plan == -TakeOrderedAndProject (87) -+- * HashAggregate (86) - +- Exchange (85) - +- * HashAggregate (84) - +- Union (83) - :- * HashAggregate (72) - : +- Exchange (71) - : +- * HashAggregate (70) - : +- Union (69) - : :- * HashAggregate (21) - : : +- Exchange (20) - : : +- * HashAggregate (19) - : : +- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- Union (9) - : : : : :- * Project (4) - : : : : : +- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : +- * Project (8) - : : : : +- * Filter (7) - : : : : +- * ColumnarToRow (6) - : : : : +- Scan parquet spark_catalog.default.store_returns (5) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (16) - : : +- * Filter (15) - : : +- * ColumnarToRow (14) - : : +- Scan parquet spark_catalog.default.store (13) - : :- * HashAggregate (42) - : : +- Exchange (41) - : : +- * HashAggregate (40) - : : +- * Project (39) - : : +- * BroadcastHashJoin Inner BuildRight (38) - : : :- * Project (33) - : : : +- * BroadcastHashJoin Inner BuildRight (32) - : : : :- Union (30) - : : : : :- * Project (25) - : : : : : +- * Filter (24) - : : : : : +- * ColumnarToRow (23) - : : : : : +- Scan parquet spark_catalog.default.catalog_sales (22) - : : : : +- * Project (29) - : : : : +- * Filter (28) - : : : : +- * ColumnarToRow (27) - : : : : +- Scan parquet spark_catalog.default.catalog_returns (26) - : : : +- ReusedExchange (31) - : : +- BroadcastExchange (37) - : : +- * Filter (36) - : : +- * ColumnarToRow (35) - : : +- Scan parquet spark_catalog.default.catalog_page (34) - : +- * HashAggregate (68) - : +- Exchange (67) - : +- * HashAggregate (66) - : +- * Project (65) - : +- * BroadcastHashJoin Inner BuildRight (64) - : :- * Project (59) - : : +- * BroadcastHashJoin Inner BuildRight (58) - : : :- Union (56) - : : : :- * Project (46) - : : : : +- * Filter (45) - : : : : +- * ColumnarToRow (44) - : : : : +- Scan parquet spark_catalog.default.web_sales (43) - : : : +- * Project (55) - : : : +- * BroadcastHashJoin Inner BuildLeft (54) - : : : :- BroadcastExchange (49) - : : : : +- * ColumnarToRow (48) - : : : : +- Scan parquet spark_catalog.default.web_returns (47) - : : : +- * Project (53) - : : : +- * Filter (52) - : : : +- * ColumnarToRow (51) - : : : +- Scan parquet spark_catalog.default.web_sales (50) - : : +- ReusedExchange (57) - : +- BroadcastExchange (63) - : +- * Filter (62) - : +- * ColumnarToRow (61) - : +- Scan parquet spark_catalog.default.web_site (60) - :- * HashAggregate (77) - : +- Exchange (76) - : +- * HashAggregate (75) - : +- * HashAggregate (74) - : +- ReusedExchange (73) - +- * HashAggregate (82) - +- Exchange (81) - +- * HashAggregate (80) - +- * HashAggregate (79) - +- ReusedExchange (78) +TakeOrderedAndProject (96) ++- * HashAggregate (95) + +- Exchange (94) + +- * HashAggregate (93) + +- Union (92) + :- * HashAggregate (81) + : +- Exchange (80) + : +- * HashAggregate (79) + : +- Union (78) + : :- * HashAggregate (24) + : : +- Exchange (23) + : : +- * HashAggregate (22) + : : +- * Project (21) + : : +- * BroadcastHashJoin Inner BuildRight (20) + : : :- Union (15) + : : : :- * Project (7) + : : : : +- * BroadcastHashJoin Inner BuildRight (6) + : : : : :- * Project (4) + : : : : : +- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet spark_catalog.default.store_sales (1) + : : : : +- ReusedExchange (5) + : : : +- * Project (14) + : : : +- * BroadcastHashJoin Inner BuildRight (13) + : : : :- * Project (11) + : : : : +- * Filter (10) + : : : : +- * ColumnarToRow (9) + : : : : +- Scan parquet spark_catalog.default.store_returns (8) + : : : +- ReusedExchange (12) + : : +- BroadcastExchange (19) + : : +- * Filter (18) + : : +- * ColumnarToRow (17) + : : +- Scan parquet spark_catalog.default.store (16) + : :- * HashAggregate (48) + : : +- Exchange (47) + : : +- * HashAggregate (46) + : : +- * Project (45) + : : +- * BroadcastHashJoin Inner BuildRight (44) + : : :- Union (39) + : : : :- * Project (31) + : : : : +- * BroadcastHashJoin Inner BuildRight (30) + : : : : :- * Project (28) + : : : : : +- * Filter (27) + : : : : : +- * ColumnarToRow (26) + : : : : : +- Scan parquet spark_catalog.default.catalog_sales (25) + : : : : +- ReusedExchange (29) + : : : +- * Project (38) + : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : :- * Project (35) + : : : : +- * Filter (34) + : : : : +- * ColumnarToRow (33) + : : : : +- Scan parquet spark_catalog.default.catalog_returns (32) + : : : +- ReusedExchange (36) + : : +- BroadcastExchange (43) + : : +- * Filter (42) + : : +- * ColumnarToRow (41) + : : +- Scan parquet spark_catalog.default.catalog_page (40) + : +- * HashAggregate (77) + : +- Exchange (76) + : +- * HashAggregate (75) + : +- * Project (74) + : +- * BroadcastHashJoin Inner BuildRight (73) + : :- Union (68) + : : :- * Project (55) + : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : :- * Project (52) + : : : : +- * Filter (51) + : : : : +- * ColumnarToRow (50) + : : : : +- Scan parquet spark_catalog.default.web_sales (49) + : : : +- ReusedExchange (53) + : : +- * Project (67) + : : +- * BroadcastHashJoin Inner BuildRight (66) + : : :- * Project (64) + : : : +- * BroadcastHashJoin Inner BuildLeft (63) + : : : :- BroadcastExchange (58) + : : : : +- * ColumnarToRow (57) + : : : : +- Scan parquet spark_catalog.default.web_returns (56) + : : : +- * Project (62) + : : : +- * Filter (61) + : : : +- * ColumnarToRow (60) + : : : +- Scan parquet spark_catalog.default.web_sales (59) + : : +- ReusedExchange (65) + : +- BroadcastExchange (72) + : +- * Filter (71) + : +- * ColumnarToRow (70) + : +- Scan parquet spark_catalog.default.web_site (69) + :- * HashAggregate (86) + : +- Exchange (85) + : +- * HashAggregate (84) + : +- * HashAggregate (83) + : +- ReusedExchange (82) + +- * HashAggregate (91) + +- Exchange (90) + +- * HashAggregate (89) + +- * HashAggregate (88) + +- ReusedExchange (87) (1) Scan parquet spark_catalog.default.store_sales @@ -96,458 +105,497 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 1] +(2) ColumnarToRow [codegen id : 2] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 1] +(3) Filter [codegen id : 2] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] Condition : isnotnull(ss_store_sk#1) -(4) Project [codegen id : 1] +(4) Project [codegen id : 2] Output [6]: [ss_store_sk#1 AS store_sk#6, ss_sold_date_sk#4 AS date_sk#7, ss_ext_sales_price#2 AS sales_price#8, ss_net_profit#3 AS profit#9, 0.00 AS return_amt#10, 0.00 AS net_loss#11] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -(5) Scan parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] +(5) ReusedExchange [Reuses operator id: 101] +Output [1]: [d_date_sk#12] + +(6) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [date_sk#7] +Right keys [1]: [d_date_sk#12] +Join type: Inner +Join condition: None + +(7) Project [codegen id : 2] +Output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] +Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#12] + +(8) Scan parquet spark_catalog.default.store_returns +Output [4]: [sr_store_sk#13, sr_return_amt#14, sr_net_loss#15, sr_returned_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#15), dynamicpruningexpression(sr_returned_date_sk#15 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(sr_returned_date_sk#16), dynamicpruningexpression(sr_returned_date_sk#16 IN dynamicpruning#5)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(6) ColumnarToRow [codegen id : 2] -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] - -(7) Filter [codegen id : 2] -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Condition : isnotnull(sr_store_sk#12) +(9) ColumnarToRow [codegen id : 4] +Input [4]: [sr_store_sk#13, sr_return_amt#14, sr_net_loss#15, sr_returned_date_sk#16] -(8) Project [codegen id : 2] -Output [6]: [sr_store_sk#12 AS store_sk#16, sr_returned_date_sk#15 AS date_sk#17, 0.00 AS sales_price#18, 0.00 AS profit#19, sr_return_amt#13 AS return_amt#20, sr_net_loss#14 AS net_loss#21] -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] +(10) Filter [codegen id : 4] +Input [4]: [sr_store_sk#13, sr_return_amt#14, sr_net_loss#15, sr_returned_date_sk#16] +Condition : isnotnull(sr_store_sk#13) -(9) Union +(11) Project [codegen id : 4] +Output [6]: [sr_store_sk#13 AS store_sk#17, sr_returned_date_sk#16 AS date_sk#18, 0.00 AS sales_price#19, 0.00 AS profit#20, sr_return_amt#14 AS return_amt#21, sr_net_loss#15 AS net_loss#22] +Input [4]: [sr_store_sk#13, sr_return_amt#14, sr_net_loss#15, sr_returned_date_sk#16] -(10) ReusedExchange [Reuses operator id: 92] -Output [1]: [d_date_sk#22] +(12) ReusedExchange [Reuses operator id: 101] +Output [1]: [d_date_sk#23] -(11) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [date_sk#7] -Right keys [1]: [d_date_sk#22] +(13) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [date_sk#18] +Right keys [1]: [d_date_sk#23] Join type: Inner Join condition: None -(12) Project [codegen id : 5] -Output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] -Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] +(14) Project [codegen id : 4] +Output [5]: [store_sk#17, sales_price#19, profit#20, return_amt#21, net_loss#22] +Input [7]: [store_sk#17, date_sk#18, sales_price#19, profit#20, return_amt#21, net_loss#22, d_date_sk#23] -(13) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#23, s_store_id#24] +(15) Union + +(16) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#24, s_store_id#25] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(14) ColumnarToRow [codegen id : 4] -Input [2]: [s_store_sk#23, s_store_id#24] +(17) ColumnarToRow [codegen id : 5] +Input [2]: [s_store_sk#24, s_store_id#25] -(15) Filter [codegen id : 4] -Input [2]: [s_store_sk#23, s_store_id#24] -Condition : isnotnull(s_store_sk#23) +(18) Filter [codegen id : 5] +Input [2]: [s_store_sk#24, s_store_id#25] +Condition : isnotnull(s_store_sk#24) -(16) BroadcastExchange -Input [2]: [s_store_sk#23, s_store_id#24] +(19) BroadcastExchange +Input [2]: [s_store_sk#24, s_store_id#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(17) BroadcastHashJoin [codegen id : 5] +(20) BroadcastHashJoin [codegen id : 6] Left keys [1]: [store_sk#6] -Right keys [1]: [s_store_sk#23] +Right keys [1]: [s_store_sk#24] Join type: Inner Join condition: None -(18) Project [codegen id : 5] -Output [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#24] -Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#23, s_store_id#24] +(21) Project [codegen id : 6] +Output [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] +Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#24, s_store_id#25] -(19) HashAggregate [codegen id : 5] -Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#24] -Keys [1]: [s_store_id#24] +(22) HashAggregate [codegen id : 6] +Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] +Keys [1]: [s_store_id#25] Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum#25, sum#26, sum#27, sum#28] -Results [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] +Aggregate Attributes [4]: [sum#26, sum#27, sum#28, sum#29] +Results [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] -(20) Exchange -Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] -Arguments: hashpartitioning(s_store_id#24, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(23) Exchange +Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] +Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(21) HashAggregate [codegen id : 6] -Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] -Keys [1]: [s_store_id#24] +(24) HashAggregate [codegen id : 7] +Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] +Keys [1]: [s_store_id#25] Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#33, sum(UnscaledValue(return_amt#10))#34, sum(UnscaledValue(profit#9))#35, sum(UnscaledValue(net_loss#11))#36] -Results [5]: [store channel AS channel#37, concat(store, s_store_id#24) AS id#38, MakeDecimal(sum(UnscaledValue(sales_price#8))#33,17,2) AS sales#39, MakeDecimal(sum(UnscaledValue(return_amt#10))#34,17,2) AS returns#40, (MakeDecimal(sum(UnscaledValue(profit#9))#35,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#36,17,2)) AS profit#41] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#34, sum(UnscaledValue(return_amt#10))#35, sum(UnscaledValue(profit#9))#36, sum(UnscaledValue(net_loss#11))#37] +Results [5]: [store channel AS channel#38, concat(store, s_store_id#25) AS id#39, MakeDecimal(sum(UnscaledValue(sales_price#8))#34,17,2) AS sales#40, MakeDecimal(sum(UnscaledValue(return_amt#10))#35,17,2) AS returns#41, (MakeDecimal(sum(UnscaledValue(profit#9))#36,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#37,17,2)) AS profit#42] -(22) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +(25) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#45), dynamicpruningexpression(cs_sold_date_sk#45 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#46), dynamicpruningexpression(cs_sold_date_sk#46 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct -(23) ColumnarToRow [codegen id : 7] -Input [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +(26) ColumnarToRow [codegen id : 9] +Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] + +(27) Filter [codegen id : 9] +Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] +Condition : isnotnull(cs_catalog_page_sk#43) -(24) Filter [codegen id : 7] -Input [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Condition : isnotnull(cs_catalog_page_sk#42) +(28) Project [codegen id : 9] +Output [6]: [cs_catalog_page_sk#43 AS page_sk#47, cs_sold_date_sk#46 AS date_sk#48, cs_ext_sales_price#44 AS sales_price#49, cs_net_profit#45 AS profit#50, 0.00 AS return_amt#51, 0.00 AS net_loss#52] +Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] + +(29) ReusedExchange [Reuses operator id: 101] +Output [1]: [d_date_sk#53] + +(30) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [date_sk#48] +Right keys [1]: [d_date_sk#53] +Join type: Inner +Join condition: None -(25) Project [codegen id : 7] -Output [6]: [cs_catalog_page_sk#42 AS page_sk#46, cs_sold_date_sk#45 AS date_sk#47, cs_ext_sales_price#43 AS sales_price#48, cs_net_profit#44 AS profit#49, 0.00 AS return_amt#50, 0.00 AS net_loss#51] -Input [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +(31) Project [codegen id : 9] +Output [5]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52] +Input [7]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52, d_date_sk#53] -(26) Scan parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#52, cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] +(32) Scan parquet spark_catalog.default.catalog_returns +Output [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#55), dynamicpruningexpression(cr_returned_date_sk#55 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cr_returned_date_sk#57), dynamicpruningexpression(cr_returned_date_sk#57 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct -(27) ColumnarToRow [codegen id : 8] -Input [4]: [cr_catalog_page_sk#52, cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] +(33) ColumnarToRow [codegen id : 11] +Input [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] -(28) Filter [codegen id : 8] -Input [4]: [cr_catalog_page_sk#52, cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] -Condition : isnotnull(cr_catalog_page_sk#52) +(34) Filter [codegen id : 11] +Input [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] +Condition : isnotnull(cr_catalog_page_sk#54) -(29) Project [codegen id : 8] -Output [6]: [cr_catalog_page_sk#52 AS page_sk#56, cr_returned_date_sk#55 AS date_sk#57, 0.00 AS sales_price#58, 0.00 AS profit#59, cr_return_amount#53 AS return_amt#60, cr_net_loss#54 AS net_loss#61] -Input [4]: [cr_catalog_page_sk#52, cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] +(35) Project [codegen id : 11] +Output [6]: [cr_catalog_page_sk#54 AS page_sk#58, cr_returned_date_sk#57 AS date_sk#59, 0.00 AS sales_price#60, 0.00 AS profit#61, cr_return_amount#55 AS return_amt#62, cr_net_loss#56 AS net_loss#63] +Input [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] -(30) Union +(36) ReusedExchange [Reuses operator id: 101] +Output [1]: [d_date_sk#64] -(31) ReusedExchange [Reuses operator id: 92] -Output [1]: [d_date_sk#62] - -(32) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [date_sk#47] -Right keys [1]: [d_date_sk#62] +(37) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [date_sk#59] +Right keys [1]: [d_date_sk#64] Join type: Inner Join condition: None -(33) Project [codegen id : 11] -Output [5]: [page_sk#46, sales_price#48, profit#49, return_amt#50, net_loss#51] -Input [7]: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51, d_date_sk#62] +(38) Project [codegen id : 11] +Output [5]: [page_sk#58, sales_price#60, profit#61, return_amt#62, net_loss#63] +Input [7]: [page_sk#58, date_sk#59, sales_price#60, profit#61, return_amt#62, net_loss#63, d_date_sk#64] + +(39) Union -(34) Scan parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#63, cp_catalog_page_id#64] +(40) Scan parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(35) ColumnarToRow [codegen id : 10] -Input [2]: [cp_catalog_page_sk#63, cp_catalog_page_id#64] +(41) ColumnarToRow [codegen id : 12] +Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] -(36) Filter [codegen id : 10] -Input [2]: [cp_catalog_page_sk#63, cp_catalog_page_id#64] -Condition : isnotnull(cp_catalog_page_sk#63) +(42) Filter [codegen id : 12] +Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] +Condition : isnotnull(cp_catalog_page_sk#65) -(37) BroadcastExchange -Input [2]: [cp_catalog_page_sk#63, cp_catalog_page_id#64] +(43) BroadcastExchange +Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(38) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [page_sk#46] -Right keys [1]: [cp_catalog_page_sk#63] +(44) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [page_sk#47] +Right keys [1]: [cp_catalog_page_sk#65] Join type: Inner Join condition: None -(39) Project [codegen id : 11] -Output [5]: [sales_price#48, profit#49, return_amt#50, net_loss#51, cp_catalog_page_id#64] -Input [7]: [page_sk#46, sales_price#48, profit#49, return_amt#50, net_loss#51, cp_catalog_page_sk#63, cp_catalog_page_id#64] - -(40) HashAggregate [codegen id : 11] -Input [5]: [sales_price#48, profit#49, return_amt#50, net_loss#51, cp_catalog_page_id#64] -Keys [1]: [cp_catalog_page_id#64] -Functions [4]: [partial_sum(UnscaledValue(sales_price#48)), partial_sum(UnscaledValue(return_amt#50)), partial_sum(UnscaledValue(profit#49)), partial_sum(UnscaledValue(net_loss#51))] -Aggregate Attributes [4]: [sum#65, sum#66, sum#67, sum#68] -Results [5]: [cp_catalog_page_id#64, sum#69, sum#70, sum#71, sum#72] - -(41) Exchange -Input [5]: [cp_catalog_page_id#64, sum#69, sum#70, sum#71, sum#72] -Arguments: hashpartitioning(cp_catalog_page_id#64, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(42) HashAggregate [codegen id : 12] -Input [5]: [cp_catalog_page_id#64, sum#69, sum#70, sum#71, sum#72] -Keys [1]: [cp_catalog_page_id#64] -Functions [4]: [sum(UnscaledValue(sales_price#48)), sum(UnscaledValue(return_amt#50)), sum(UnscaledValue(profit#49)), sum(UnscaledValue(net_loss#51))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#48))#73, sum(UnscaledValue(return_amt#50))#74, sum(UnscaledValue(profit#49))#75, sum(UnscaledValue(net_loss#51))#76] -Results [5]: [catalog channel AS channel#77, concat(catalog_page, cp_catalog_page_id#64) AS id#78, MakeDecimal(sum(UnscaledValue(sales_price#48))#73,17,2) AS sales#79, MakeDecimal(sum(UnscaledValue(return_amt#50))#74,17,2) AS returns#80, (MakeDecimal(sum(UnscaledValue(profit#49))#75,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#51))#76,17,2)) AS profit#81] - -(43) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#82, ws_ext_sales_price#83, ws_net_profit#84, ws_sold_date_sk#85] +(45) Project [codegen id : 13] +Output [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#66] +Input [7]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_sk#65, cp_catalog_page_id#66] + +(46) HashAggregate [codegen id : 13] +Input [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#66] +Keys [1]: [cp_catalog_page_id#66] +Functions [4]: [partial_sum(UnscaledValue(sales_price#49)), partial_sum(UnscaledValue(return_amt#51)), partial_sum(UnscaledValue(profit#50)), partial_sum(UnscaledValue(net_loss#52))] +Aggregate Attributes [4]: [sum#67, sum#68, sum#69, sum#70] +Results [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] + +(47) Exchange +Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] +Arguments: hashpartitioning(cp_catalog_page_id#66, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(48) HashAggregate [codegen id : 14] +Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] +Keys [1]: [cp_catalog_page_id#66] +Functions [4]: [sum(UnscaledValue(sales_price#49)), sum(UnscaledValue(return_amt#51)), sum(UnscaledValue(profit#50)), sum(UnscaledValue(net_loss#52))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#49))#75, sum(UnscaledValue(return_amt#51))#76, sum(UnscaledValue(profit#50))#77, sum(UnscaledValue(net_loss#52))#78] +Results [5]: [catalog channel AS channel#79, concat(catalog_page, cp_catalog_page_id#66) AS id#80, MakeDecimal(sum(UnscaledValue(sales_price#49))#75,17,2) AS sales#81, MakeDecimal(sum(UnscaledValue(return_amt#51))#76,17,2) AS returns#82, (MakeDecimal(sum(UnscaledValue(profit#50))#77,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#52))#78,17,2)) AS profit#83] + +(49) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#85), dynamicpruningexpression(ws_sold_date_sk#85 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#87), dynamicpruningexpression(ws_sold_date_sk#87 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(44) ColumnarToRow [codegen id : 13] -Input [4]: [ws_web_site_sk#82, ws_ext_sales_price#83, ws_net_profit#84, ws_sold_date_sk#85] +(50) ColumnarToRow [codegen id : 16] +Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] + +(51) Filter [codegen id : 16] +Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] +Condition : isnotnull(ws_web_site_sk#84) + +(52) Project [codegen id : 16] +Output [6]: [ws_web_site_sk#84 AS wsr_web_site_sk#88, ws_sold_date_sk#87 AS date_sk#89, ws_ext_sales_price#85 AS sales_price#90, ws_net_profit#86 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] +Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] -(45) Filter [codegen id : 13] -Input [4]: [ws_web_site_sk#82, ws_ext_sales_price#83, ws_net_profit#84, ws_sold_date_sk#85] -Condition : isnotnull(ws_web_site_sk#82) +(53) ReusedExchange [Reuses operator id: 101] +Output [1]: [d_date_sk#94] -(46) Project [codegen id : 13] -Output [6]: [ws_web_site_sk#82 AS wsr_web_site_sk#86, ws_sold_date_sk#85 AS date_sk#87, ws_ext_sales_price#83 AS sales_price#88, ws_net_profit#84 AS profit#89, 0.00 AS return_amt#90, 0.00 AS net_loss#91] -Input [4]: [ws_web_site_sk#82, ws_ext_sales_price#83, ws_net_profit#84, ws_sold_date_sk#85] +(54) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [date_sk#89] +Right keys [1]: [d_date_sk#94] +Join type: Inner +Join condition: None -(47) Scan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96] +(55) Project [codegen id : 16] +Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] +Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#94] + +(56) Scan parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#95, wr_order_number#96, wr_return_amt#97, wr_net_loss#98, wr_returned_date_sk#99] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#96), dynamicpruningexpression(wr_returned_date_sk#96 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(wr_returned_date_sk#99), dynamicpruningexpression(wr_returned_date_sk#99 IN dynamicpruning#5)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 14] -Input [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96] +(57) ColumnarToRow [codegen id : 17] +Input [5]: [wr_item_sk#95, wr_order_number#96, wr_return_amt#97, wr_net_loss#98, wr_returned_date_sk#99] -(49) BroadcastExchange -Input [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96] +(58) BroadcastExchange +Input [5]: [wr_item_sk#95, wr_order_number#96, wr_return_amt#97, wr_net_loss#98, wr_returned_date_sk#99] Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[1, int, true] as bigint) & 4294967295))),false), [plan_id=5] -(50) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] +(59) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102, ws_sold_date_sk#103] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(51) ColumnarToRow -Input [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] +(60) ColumnarToRow +Input [4]: [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102, ws_sold_date_sk#103] -(52) Filter -Input [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] -Condition : ((isnotnull(ws_item_sk#97) AND isnotnull(ws_order_number#99)) AND isnotnull(ws_web_site_sk#98)) +(61) Filter +Input [4]: [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102, ws_sold_date_sk#103] +Condition : ((isnotnull(ws_item_sk#100) AND isnotnull(ws_order_number#102)) AND isnotnull(ws_web_site_sk#101)) -(53) Project -Output [3]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99] -Input [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] +(62) Project +Output [3]: [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102] +Input [4]: [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102, ws_sold_date_sk#103] -(54) BroadcastHashJoin [codegen id : 15] -Left keys [2]: [wr_item_sk#92, wr_order_number#93] -Right keys [2]: [ws_item_sk#97, ws_order_number#99] +(63) BroadcastHashJoin [codegen id : 19] +Left keys [2]: [wr_item_sk#95, wr_order_number#96] +Right keys [2]: [ws_item_sk#100, ws_order_number#102] Join type: Inner Join condition: None -(55) Project [codegen id : 15] -Output [6]: [ws_web_site_sk#98 AS wsr_web_site_sk#101, wr_returned_date_sk#96 AS date_sk#102, 0.00 AS sales_price#103, 0.00 AS profit#104, wr_return_amt#94 AS return_amt#105, wr_net_loss#95 AS net_loss#106] -Input [8]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96, ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99] - -(56) Union +(64) Project [codegen id : 19] +Output [6]: [ws_web_site_sk#101 AS wsr_web_site_sk#104, wr_returned_date_sk#99 AS date_sk#105, 0.00 AS sales_price#106, 0.00 AS profit#107, wr_return_amt#97 AS return_amt#108, wr_net_loss#98 AS net_loss#109] +Input [8]: [wr_item_sk#95, wr_order_number#96, wr_return_amt#97, wr_net_loss#98, wr_returned_date_sk#99, ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102] -(57) ReusedExchange [Reuses operator id: 92] -Output [1]: [d_date_sk#107] +(65) ReusedExchange [Reuses operator id: 101] +Output [1]: [d_date_sk#110] -(58) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [date_sk#87] -Right keys [1]: [d_date_sk#107] +(66) BroadcastHashJoin [codegen id : 19] +Left keys [1]: [date_sk#105] +Right keys [1]: [d_date_sk#110] Join type: Inner Join condition: None -(59) Project [codegen id : 18] -Output [5]: [wsr_web_site_sk#86, sales_price#88, profit#89, return_amt#90, net_loss#91] -Input [7]: [wsr_web_site_sk#86, date_sk#87, sales_price#88, profit#89, return_amt#90, net_loss#91, d_date_sk#107] +(67) Project [codegen id : 19] +Output [5]: [wsr_web_site_sk#104, sales_price#106, profit#107, return_amt#108, net_loss#109] +Input [7]: [wsr_web_site_sk#104, date_sk#105, sales_price#106, profit#107, return_amt#108, net_loss#109, d_date_sk#110] -(60) Scan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#108, web_site_id#109] +(68) Union + +(69) Scan parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#111, web_site_id#112] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(61) ColumnarToRow [codegen id : 17] -Input [2]: [web_site_sk#108, web_site_id#109] +(70) ColumnarToRow [codegen id : 20] +Input [2]: [web_site_sk#111, web_site_id#112] -(62) Filter [codegen id : 17] -Input [2]: [web_site_sk#108, web_site_id#109] -Condition : isnotnull(web_site_sk#108) +(71) Filter [codegen id : 20] +Input [2]: [web_site_sk#111, web_site_id#112] +Condition : isnotnull(web_site_sk#111) -(63) BroadcastExchange -Input [2]: [web_site_sk#108, web_site_id#109] +(72) BroadcastExchange +Input [2]: [web_site_sk#111, web_site_id#112] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(64) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [wsr_web_site_sk#86] -Right keys [1]: [web_site_sk#108] +(73) BroadcastHashJoin [codegen id : 21] +Left keys [1]: [wsr_web_site_sk#88] +Right keys [1]: [web_site_sk#111] Join type: Inner Join condition: None -(65) Project [codegen id : 18] -Output [5]: [sales_price#88, profit#89, return_amt#90, net_loss#91, web_site_id#109] -Input [7]: [wsr_web_site_sk#86, sales_price#88, profit#89, return_amt#90, net_loss#91, web_site_sk#108, web_site_id#109] - -(66) HashAggregate [codegen id : 18] -Input [5]: [sales_price#88, profit#89, return_amt#90, net_loss#91, web_site_id#109] -Keys [1]: [web_site_id#109] -Functions [4]: [partial_sum(UnscaledValue(sales_price#88)), partial_sum(UnscaledValue(return_amt#90)), partial_sum(UnscaledValue(profit#89)), partial_sum(UnscaledValue(net_loss#91))] -Aggregate Attributes [4]: [sum#110, sum#111, sum#112, sum#113] -Results [5]: [web_site_id#109, sum#114, sum#115, sum#116, sum#117] - -(67) Exchange -Input [5]: [web_site_id#109, sum#114, sum#115, sum#116, sum#117] -Arguments: hashpartitioning(web_site_id#109, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(68) HashAggregate [codegen id : 19] -Input [5]: [web_site_id#109, sum#114, sum#115, sum#116, sum#117] -Keys [1]: [web_site_id#109] -Functions [4]: [sum(UnscaledValue(sales_price#88)), sum(UnscaledValue(return_amt#90)), sum(UnscaledValue(profit#89)), sum(UnscaledValue(net_loss#91))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#88))#118, sum(UnscaledValue(return_amt#90))#119, sum(UnscaledValue(profit#89))#120, sum(UnscaledValue(net_loss#91))#121] -Results [5]: [web channel AS channel#122, concat(web_site, web_site_id#109) AS id#123, MakeDecimal(sum(UnscaledValue(sales_price#88))#118,17,2) AS sales#124, MakeDecimal(sum(UnscaledValue(return_amt#90))#119,17,2) AS returns#125, (MakeDecimal(sum(UnscaledValue(profit#89))#120,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#91))#121,17,2)) AS profit#126] - -(69) Union - -(70) HashAggregate [codegen id : 20] -Input [5]: [channel#37, id#38, sales#39, returns#40, profit#41] -Keys [2]: [channel#37, id#38] -Functions [3]: [partial_sum(sales#39), partial_sum(returns#40), partial_sum(profit#41)] -Aggregate Attributes [6]: [sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Results [8]: [channel#37, id#38, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] - -(71) Exchange -Input [8]: [channel#37, id#38, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] -Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, [plan_id=8] - -(72) HashAggregate [codegen id : 21] -Input [8]: [channel#37, id#38, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] -Keys [2]: [channel#37, id#38] -Functions [3]: [sum(sales#39), sum(returns#40), sum(profit#41)] -Aggregate Attributes [3]: [sum(sales#39)#139, sum(returns#40)#140, sum(profit#41)#141] -Results [5]: [channel#37, id#38, cast(sum(sales#39)#139 as decimal(37,2)) AS sales#142, cast(sum(returns#40)#140 as decimal(37,2)) AS returns#143, cast(sum(profit#41)#141 as decimal(38,2)) AS profit#144] - -(73) ReusedExchange [Reuses operator id: 71] -Output [8]: [channel#145, id#146, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] - -(74) HashAggregate [codegen id : 42] -Input [8]: [channel#145, id#146, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] -Keys [2]: [channel#145, id#146] -Functions [3]: [sum(sales#153), sum(returns#154), sum(profit#155)] -Aggregate Attributes [3]: [sum(sales#153)#139, sum(returns#154)#140, sum(profit#155)#141] -Results [4]: [channel#145, sum(sales#153)#139 AS sales#156, sum(returns#154)#140 AS returns#157, sum(profit#155)#141 AS profit#158] - -(75) HashAggregate [codegen id : 42] -Input [4]: [channel#145, sales#156, returns#157, profit#158] -Keys [1]: [channel#145] -Functions [3]: [partial_sum(sales#156), partial_sum(returns#157), partial_sum(profit#158)] -Aggregate Attributes [6]: [sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] -Results [7]: [channel#145, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] +(74) Project [codegen id : 21] +Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#112] +Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#111, web_site_id#112] -(76) Exchange -Input [7]: [channel#145, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] -Arguments: hashpartitioning(channel#145, 5), ENSURE_REQUIREMENTS, [plan_id=9] +(75) HashAggregate [codegen id : 21] +Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#112] +Keys [1]: [web_site_id#112] +Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] +Aggregate Attributes [4]: [sum#113, sum#114, sum#115, sum#116] +Results [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] -(77) HashAggregate [codegen id : 43] -Input [7]: [channel#145, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] -Keys [1]: [channel#145] +(76) Exchange +Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] +Arguments: hashpartitioning(web_site_id#112, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(77) HashAggregate [codegen id : 22] +Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] +Keys [1]: [web_site_id#112] +Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#90))#121, sum(UnscaledValue(return_amt#92))#122, sum(UnscaledValue(profit#91))#123, sum(UnscaledValue(net_loss#93))#124] +Results [5]: [web channel AS channel#125, concat(web_site, web_site_id#112) AS id#126, MakeDecimal(sum(UnscaledValue(sales_price#90))#121,17,2) AS sales#127, MakeDecimal(sum(UnscaledValue(return_amt#92))#122,17,2) AS returns#128, (MakeDecimal(sum(UnscaledValue(profit#91))#123,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#93))#124,17,2)) AS profit#129] + +(78) Union + +(79) HashAggregate [codegen id : 23] +Input [5]: [channel#38, id#39, sales#40, returns#41, profit#42] +Keys [2]: [channel#38, id#39] +Functions [3]: [partial_sum(sales#40), partial_sum(returns#41), partial_sum(profit#42)] +Aggregate Attributes [6]: [sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135] +Results [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] + +(80) Exchange +Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] +Arguments: hashpartitioning(channel#38, id#39, 5), ENSURE_REQUIREMENTS, [plan_id=8] + +(81) HashAggregate [codegen id : 24] +Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] +Keys [2]: [channel#38, id#39] +Functions [3]: [sum(sales#40), sum(returns#41), sum(profit#42)] +Aggregate Attributes [3]: [sum(sales#40)#142, sum(returns#41)#143, sum(profit#42)#144] +Results [5]: [channel#38, id#39, cast(sum(sales#40)#142 as decimal(37,2)) AS sales#145, cast(sum(returns#41)#143 as decimal(37,2)) AS returns#146, cast(sum(profit#42)#144 as decimal(38,2)) AS profit#147] + +(82) ReusedExchange [Reuses operator id: 80] +Output [8]: [channel#148, id#149, sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155] + +(83) HashAggregate [codegen id : 48] +Input [8]: [channel#148, id#149, sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155] +Keys [2]: [channel#148, id#149] Functions [3]: [sum(sales#156), sum(returns#157), sum(profit#158)] -Aggregate Attributes [3]: [sum(sales#156)#171, sum(returns#157)#172, sum(profit#158)#173] -Results [5]: [channel#145, null AS id#174, sum(sales#156)#171 AS sum(sales)#175, sum(returns#157)#172 AS sum(returns)#176, sum(profit#158)#173 AS sum(profit)#177] +Aggregate Attributes [3]: [sum(sales#156)#142, sum(returns#157)#143, sum(profit#158)#144] +Results [4]: [channel#148, sum(sales#156)#142 AS sales#159, sum(returns#157)#143 AS returns#160, sum(profit#158)#144 AS profit#161] -(78) ReusedExchange [Reuses operator id: 71] -Output [8]: [channel#178, id#179, sum#180, isEmpty#181, sum#182, isEmpty#183, sum#184, isEmpty#185] +(84) HashAggregate [codegen id : 48] +Input [4]: [channel#148, sales#159, returns#160, profit#161] +Keys [1]: [channel#148] +Functions [3]: [partial_sum(sales#159), partial_sum(returns#160), partial_sum(profit#161)] +Aggregate Attributes [6]: [sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] +Results [7]: [channel#148, sum#168, isEmpty#169, sum#170, isEmpty#171, sum#172, isEmpty#173] -(79) HashAggregate [codegen id : 64] -Input [8]: [channel#178, id#179, sum#180, isEmpty#181, sum#182, isEmpty#183, sum#184, isEmpty#185] -Keys [2]: [channel#178, id#179] -Functions [3]: [sum(sales#186), sum(returns#187), sum(profit#188)] -Aggregate Attributes [3]: [sum(sales#186)#139, sum(returns#187)#140, sum(profit#188)#141] -Results [3]: [sum(sales#186)#139 AS sales#189, sum(returns#187)#140 AS returns#190, sum(profit#188)#141 AS profit#191] +(85) Exchange +Input [7]: [channel#148, sum#168, isEmpty#169, sum#170, isEmpty#171, sum#172, isEmpty#173] +Arguments: hashpartitioning(channel#148, 5), ENSURE_REQUIREMENTS, [plan_id=9] + +(86) HashAggregate [codegen id : 49] +Input [7]: [channel#148, sum#168, isEmpty#169, sum#170, isEmpty#171, sum#172, isEmpty#173] +Keys [1]: [channel#148] +Functions [3]: [sum(sales#159), sum(returns#160), sum(profit#161)] +Aggregate Attributes [3]: [sum(sales#159)#174, sum(returns#160)#175, sum(profit#161)#176] +Results [5]: [channel#148, null AS id#177, sum(sales#159)#174 AS sum(sales)#178, sum(returns#160)#175 AS sum(returns)#179, sum(profit#161)#176 AS sum(profit)#180] + +(87) ReusedExchange [Reuses operator id: 80] +Output [8]: [channel#181, id#182, sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188] + +(88) HashAggregate [codegen id : 73] +Input [8]: [channel#181, id#182, sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188] +Keys [2]: [channel#181, id#182] +Functions [3]: [sum(sales#189), sum(returns#190), sum(profit#191)] +Aggregate Attributes [3]: [sum(sales#189)#142, sum(returns#190)#143, sum(profit#191)#144] +Results [3]: [sum(sales#189)#142 AS sales#192, sum(returns#190)#143 AS returns#193, sum(profit#191)#144 AS profit#194] -(80) HashAggregate [codegen id : 64] -Input [3]: [sales#189, returns#190, profit#191] +(89) HashAggregate [codegen id : 73] +Input [3]: [sales#192, returns#193, profit#194] Keys: [] -Functions [3]: [partial_sum(sales#189), partial_sum(returns#190), partial_sum(profit#191)] -Aggregate Attributes [6]: [sum#192, isEmpty#193, sum#194, isEmpty#195, sum#196, isEmpty#197] -Results [6]: [sum#198, isEmpty#199, sum#200, isEmpty#201, sum#202, isEmpty#203] +Functions [3]: [partial_sum(sales#192), partial_sum(returns#193), partial_sum(profit#194)] +Aggregate Attributes [6]: [sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200] +Results [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] -(81) Exchange -Input [6]: [sum#198, isEmpty#199, sum#200, isEmpty#201, sum#202, isEmpty#203] +(90) Exchange +Input [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] -(82) HashAggregate [codegen id : 65] -Input [6]: [sum#198, isEmpty#199, sum#200, isEmpty#201, sum#202, isEmpty#203] +(91) HashAggregate [codegen id : 74] +Input [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] Keys: [] -Functions [3]: [sum(sales#189), sum(returns#190), sum(profit#191)] -Aggregate Attributes [3]: [sum(sales#189)#204, sum(returns#190)#205, sum(profit#191)#206] -Results [5]: [null AS channel#207, null AS id#208, sum(sales#189)#204 AS sum(sales)#209, sum(returns#190)#205 AS sum(returns)#210, sum(profit#191)#206 AS sum(profit)#211] +Functions [3]: [sum(sales#192), sum(returns#193), sum(profit#194)] +Aggregate Attributes [3]: [sum(sales#192)#207, sum(returns#193)#208, sum(profit#194)#209] +Results [5]: [null AS channel#210, null AS id#211, sum(sales#192)#207 AS sum(sales)#212, sum(returns#193)#208 AS sum(returns)#213, sum(profit#194)#209 AS sum(profit)#214] -(83) Union +(92) Union -(84) HashAggregate [codegen id : 66] -Input [5]: [channel#37, id#38, sales#142, returns#143, profit#144] -Keys [5]: [channel#37, id#38, sales#142, returns#143, profit#144] +(93) HashAggregate [codegen id : 75] +Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] +Keys [5]: [channel#38, id#39, sales#145, returns#146, profit#147] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#37, id#38, sales#142, returns#143, profit#144] +Results [5]: [channel#38, id#39, sales#145, returns#146, profit#147] -(85) Exchange -Input [5]: [channel#37, id#38, sales#142, returns#143, profit#144] -Arguments: hashpartitioning(channel#37, id#38, sales#142, returns#143, profit#144, 5), ENSURE_REQUIREMENTS, [plan_id=11] +(94) Exchange +Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] +Arguments: hashpartitioning(channel#38, id#39, sales#145, returns#146, profit#147, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(86) HashAggregate [codegen id : 67] -Input [5]: [channel#37, id#38, sales#142, returns#143, profit#144] -Keys [5]: [channel#37, id#38, sales#142, returns#143, profit#144] +(95) HashAggregate [codegen id : 76] +Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] +Keys [5]: [channel#38, id#39, sales#145, returns#146, profit#147] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#37, id#38, sales#142, returns#143, profit#144] +Results [5]: [channel#38, id#39, sales#145, returns#146, profit#147] -(87) TakeOrderedAndProject -Input [5]: [channel#37, id#38, sales#142, returns#143, profit#144] -Arguments: 100, [channel#37 ASC NULLS FIRST, id#38 ASC NULLS FIRST], [channel#37, id#38, sales#142, returns#143, profit#144] +(96) TakeOrderedAndProject +Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] +Arguments: 100, [channel#38 ASC NULLS FIRST, id#39 ASC NULLS FIRST], [channel#38, id#39, sales#145, returns#146, profit#147] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (92) -+- * Project (91) - +- * Filter (90) - +- * ColumnarToRow (89) - +- Scan parquet spark_catalog.default.date_dim (88) +BroadcastExchange (101) ++- * Project (100) + +- * Filter (99) + +- * ColumnarToRow (98) + +- Scan parquet spark_catalog.default.date_dim (97) -(88) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#212] +(97) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#12, d_date#215] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] ReadSchema: struct -(89) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#22, d_date#212] +(98) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#12, d_date#215] -(90) Filter [codegen id : 1] -Input [2]: [d_date_sk#22, d_date#212] -Condition : (((isnotnull(d_date#212) AND (d_date#212 >= 1998-08-04)) AND (d_date#212 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) +(99) Filter [codegen id : 1] +Input [2]: [d_date_sk#12, d_date#215] +Condition : (((isnotnull(d_date#215) AND (d_date#215 >= 1998-08-04)) AND (d_date#215 <= 1998-08-18)) AND isnotnull(d_date_sk#12)) -(91) Project [codegen id : 1] -Output [1]: [d_date_sk#22] -Input [2]: [d_date_sk#22, d_date#212] +(100) Project [codegen id : 1] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_date#215] -(92) BroadcastExchange -Input [1]: [d_date_sk#22] +(101) BroadcastExchange +Input [1]: [d_date_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -Subquery:2 Hosting operator id = 5 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 8 Hosting Expression = sr_returned_date_sk#16 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 25 Hosting Expression = cs_sold_date_sk#46 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#55 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 32 Hosting Expression = cr_returned_date_sk#57 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#85 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 49 Hosting Expression = ws_sold_date_sk#87 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 47 Hosting Expression = wr_returned_date_sk#96 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 56 Hosting Expression = wr_returned_date_sk#99 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt index c7d72bb6c5bd7..70578c526ddcc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt @@ -1,33 +1,33 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (67) + WholeStageCodegen (76) HashAggregate [channel,id,sales,returns,profit] InputAdapter Exchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (66) + WholeStageCodegen (75) HashAggregate [channel,id,sales,returns,profit] InputAdapter Union - WholeStageCodegen (21) + WholeStageCodegen (24) HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel,id] #2 - WholeStageCodegen (20) + WholeStageCodegen (23) HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (6) + WholeStageCodegen (7) HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] InputAdapter Exchange [s_store_id] #3 - WholeStageCodegen (5) + WholeStageCodegen (6) HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] Project [sales_price,profit,return_amt,net_loss,s_store_id] BroadcastHashJoin [store_sk,s_store_sk] - Project [store_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (1) + InputAdapter + Union + WholeStageCodegen (2) + Project [store_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] Project [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] Filter [ss_store_sk] ColumnarToRow @@ -41,82 +41,94 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - WholeStageCodegen (2) + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (4) + Project [store_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] Project [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] Filter [sr_store_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 + InputAdapter + ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #5 - WholeStageCodegen (4) + WholeStageCodegen (5) Filter [s_store_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store [s_store_sk,s_store_id] - WholeStageCodegen (12) + WholeStageCodegen (14) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] InputAdapter Exchange [cp_catalog_page_id] #6 - WholeStageCodegen (11) + WholeStageCodegen (13) HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] BroadcastHashJoin [page_sk,cp_catalog_page_sk] - Project [page_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (7) + InputAdapter + Union + WholeStageCodegen (9) + Project [page_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] Project [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] Filter [cs_catalog_page_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (8) + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (11) + Project [page_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] Project [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] Filter [cr_catalog_page_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 + InputAdapter + ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #7 - WholeStageCodegen (10) + WholeStageCodegen (12) Filter [cp_catalog_page_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (19) + WholeStageCodegen (22) HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] InputAdapter Exchange [web_site_id] #8 - WholeStageCodegen (18) + WholeStageCodegen (21) HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] Project [sales_price,profit,return_amt,net_loss,web_site_id] BroadcastHashJoin [wsr_web_site_sk,web_site_sk] - Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (13) + InputAdapter + Union + WholeStageCodegen (16) + Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] Project [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] Filter [ws_web_site_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (15) + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (19) + Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] Project [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] InputAdapter BroadcastExchange #9 - WholeStageCodegen (14) + WholeStageCodegen (17) ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] @@ -126,29 +138,29 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #4 + InputAdapter + ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #10 - WholeStageCodegen (17) + WholeStageCodegen (20) Filter [web_site_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - WholeStageCodegen (43) + WholeStageCodegen (49) HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel] #11 - WholeStageCodegen (42) + WholeStageCodegen (48) HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (65) + WholeStageCodegen (74) HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange #12 - WholeStageCodegen (64) + WholeStageCodegen (73) HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter diff --git a/sql/core/src/test/scala/org/apache/spark/sql/PushDownJoinThroughUnionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/PushDownJoinThroughUnionSuite.scala new file mode 100644 index 0000000000000..9ee1100425e66 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/PushDownJoinThroughUnionSuite.scala @@ -0,0 +1,207 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession + +class PushDownJoinThroughUnionSuite extends QueryTest with SharedSparkSession { + import testImplicits._ + + test("UNION ALL + broadcast JOIN produces correct results") { + withTempView("fact1", "fact2", "dim") { + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "10485760") { + val fact1 = Seq((1, "a"), (2, "b")).toDF("id", "val1") + val fact2 = Seq((3, "c"), (4, "d")).toDF("id", "val1") + val dim = Seq((1, "x"), (2, "y"), (3, "z")).toDF("id", "label") + + fact1.createOrReplaceTempView("fact1") + fact2.createOrReplaceTempView("fact2") + dim.createOrReplaceTempView("dim") + + val result = sql( + """SELECT f.id, f.val1, d.label + |FROM (SELECT * FROM fact1 UNION ALL SELECT * FROM fact2) f + |JOIN dim d ON f.id = d.id + """.stripMargin) + + checkAnswer(result, Seq( + Row(1, "a", "x"), + Row(2, "b", "y"), + Row(3, "c", "z") + )) + } + } + } + + test("3-way UNION ALL + broadcast JOIN (TPC-DS pattern)") { + withTempView("fact1", "fact2", "fact3", "dim") { + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "10485760") { + val fact1 = Seq((1, 10), (2, 20)).toDF("id", "amount") + val fact2 = Seq((3, 30), (4, 40)).toDF("id", "amount") + val fact3 = Seq((1, 50), (5, 60)).toDF("id", "amount") + val dim = Seq((1, "web"), (2, "store"), (3, "catalog"), (5, "other")) + .toDF("id", "channel") + + fact1.createOrReplaceTempView("fact1") + fact2.createOrReplaceTempView("fact2") + fact3.createOrReplaceTempView("fact3") + dim.createOrReplaceTempView("dim") + + val result = sql( + """SELECT f.id, f.amount, d.channel + |FROM ( + | SELECT * FROM fact1 + | UNION ALL SELECT * FROM fact2 + | UNION ALL SELECT * FROM fact3 + |) f + |JOIN dim d ON f.id = d.id + """.stripMargin) + + checkAnswer(result, Seq( + Row(1, 10, "web"), + Row(2, 20, "store"), + Row(3, 30, "catalog"), + Row(1, 50, "web"), + Row(5, 60, "other") + )) + } + } + } + + test("LeftOuter Join through UNION ALL produces correct results") { + withTempView("fact1", "fact2", "dim") { + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "10485760") { + val fact1 = Seq((1, "a"), (2, "b")).toDF("id", "val1") + val fact2 = Seq((3, "c"), (99, "d")).toDF("id", "val1") + val dim = Seq((1, "x"), (2, "y"), (3, "z")).toDF("id", "label") + + fact1.createOrReplaceTempView("fact1") + fact2.createOrReplaceTempView("fact2") + dim.createOrReplaceTempView("dim") + + val result = sql( + """SELECT f.id, f.val1, d.label + |FROM (SELECT * FROM fact1 UNION ALL SELECT * FROM fact2) f + |LEFT OUTER JOIN dim d ON f.id = d.id + """.stripMargin) + + checkAnswer(result, Seq( + Row(1, "a", "x"), + Row(2, "b", "y"), + Row(3, "c", "z"), + Row(99, "d", null) + )) + } + } + } + + test("Optimization disabled produces same results") { + withTempView("fact1", "fact2", "dim") { + val fact1 = Seq((1, "a"), (2, "b")).toDF("id", "val1") + val fact2 = Seq((3, "c"), (4, "d")).toDF("id", "val1") + val dim = Seq((1, "x"), (2, "y"), (3, "z")).toDF("id", "label") + + fact1.createOrReplaceTempView("fact1") + fact2.createOrReplaceTempView("fact2") + dim.createOrReplaceTempView("dim") + + val query = + """SELECT f.id, f.val1, d.label + |FROM (SELECT * FROM fact1 UNION ALL SELECT * FROM fact2) f + |JOIN dim d ON f.id = d.id + """.stripMargin + + val expected = Seq( + Row(1, "a", "x"), + Row(2, "b", "y"), + Row(3, "c", "z") + ) + + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "10485760") { + checkAnswer(sql(query), expected) + } + + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "10485760", + SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> + "org.apache.spark.sql.catalyst.optimizer.PushDownJoinThroughUnion") { + checkAnswer(sql(query), expected) + } + } + } + + test("ColumnPruning works after join push down") { + withTempView("fact1", "fact2", "dim") { + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "10485760") { + val fact1 = Seq((1, "a", 100), (2, "b", 200)).toDF("id", "val1", "val2") + val fact2 = Seq((3, "c", 300), (4, "d", 400)).toDF("id", "val1", "val2") + val dim = Seq((1, "x", "extra1"), (2, "y", "extra2"), (3, "z", "extra3")) + .toDF("id", "label", "info") + + fact1.createOrReplaceTempView("fact1") + fact2.createOrReplaceTempView("fact2") + dim.createOrReplaceTempView("dim") + + val result = sql( + """SELECT f.id, d.label + |FROM (SELECT * FROM fact1 UNION ALL SELECT * FROM fact2) f + |JOIN dim d ON f.id = d.id + """.stripMargin) + + checkAnswer(result, Seq( + Row(1, "x"), + Row(2, "y"), + Row(3, "z") + )) + } + } + } + + test("PushPredicateThroughJoin works after join push down") { + withTempView("fact1", "fact2", "dim") { + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "10485760") { + val fact1 = Seq((1, "a"), (2, "b")).toDF("id", "val1") + val fact2 = Seq((3, "c"), (4, "d")).toDF("id", "val1") + val dim = Seq((1, "x"), (2, "y"), (3, "z"), (4, "w")).toDF("id", "label") + + fact1.createOrReplaceTempView("fact1") + fact2.createOrReplaceTempView("fact2") + dim.createOrReplaceTempView("dim") + + val result = sql( + """SELECT f.id, f.val1, d.label + |FROM (SELECT * FROM fact1 UNION ALL SELECT * FROM fact2) f + |JOIN dim d ON f.id = d.id + |WHERE d.label IN ('x', 'z') + """.stripMargin) + + checkAnswer(result, Seq( + Row(1, "a", "x"), + Row(3, "c", "z") + )) + } + } + } +}