From 84ba7238993466376b178219f2621fd0c32d5a48 Mon Sep 17 00:00:00 2001 From: "tanel.kiis@gmail.com" Date: Mon, 12 Oct 2020 23:19:43 +0300 Subject: [PATCH 01/22] RemoveRedundantAggregates --- .../sql/catalyst/optimizer/Optimizer.scala | 19 + .../plans/logical/basicLogicalOperators.scala | 2 +- .../RemoveRedundantAggregatesSuite.scala | 83 ++ .../q14a.sf100/explain.txt | 518 +++++----- .../q14a.sf100/simplified.txt | 134 ++- .../approved-plans-v1_4/q14a/explain.txt | 470 +++++---- .../approved-plans-v1_4/q14a/simplified.txt | 116 ++- .../q14b.sf100/explain.txt | 468 +++++---- .../q14b.sf100/simplified.txt | 134 ++- .../approved-plans-v1_4/q14b/explain.txt | 444 ++++----- .../approved-plans-v1_4/q14b/simplified.txt | 116 ++- .../approved-plans-v1_4/q38.sf100/explain.txt | 168 ++-- .../q38.sf100/simplified.txt | 194 ++-- .../approved-plans-v1_4/q38/explain.txt | 142 ++- .../approved-plans-v1_4/q38/simplified.txt | 124 ++- .../approved-plans-v1_4/q87.sf100/explain.txt | 164 ++-- .../q87.sf100/simplified.txt | 194 ++-- .../approved-plans-v1_4/q87/explain.txt | 138 +-- .../approved-plans-v1_4/q87/simplified.txt | 124 ++- .../approved-plans-v2_7/q14.sf100/explain.txt | 468 +++++---- .../q14.sf100/simplified.txt | 134 ++- .../approved-plans-v2_7/q14/explain.txt | 444 ++++----- .../approved-plans-v2_7/q14/simplified.txt | 116 ++- .../q14a.sf100/explain.txt | 922 +++++++++--------- .../q14a.sf100/simplified.txt | 134 ++- .../approved-plans-v2_7/q14a/explain.txt | 874 ++++++++--------- .../approved-plans-v2_7/q14a/simplified.txt | 116 ++- 27 files changed, 3387 insertions(+), 3573 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala 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 5bdaa504a3be..1b22a57ecc16 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 @@ -108,6 +108,7 @@ abstract class Optimizer(catalogManager: CatalogManager) RewriteCorrelatedScalarSubquery, EliminateSerialization, RemoveRedundantAliases, + RemoveRedundantAggregates, UnwrapCastInBinaryComparison, RemoveNoopOperators, CombineUpdateFields, @@ -477,6 +478,24 @@ object RemoveRedundantAliases extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = removeRedundantAliases(plan, AttributeSet.empty) } +/** + * Remove redundant aggregates from a query plan. A redundant aggregate is an aggregate whose + * only goal is to keep distinct values, while its parent aggregate would ignore duplicate values. + */ +object RemoveRedundantAggregates extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { + case upper @ Aggregate(_, _, lower: Aggregate) if isRedundant(upper, lower) => + upper.copy(child = lower.child) + } + + private def isRedundant(upper: Aggregate, lower: Aggregate): Boolean = { + val referencesOnlyGrouping = upper.references.subsetOf(AttributeSet(lower.groupingExpressions)) + val hasAggregateExpressions = upper.aggregateExpressions + .exists(_.find(_.isInstanceOf[AggregateExpression]).nonEmpty) + referencesOnlyGrouping && !hasAggregateExpressions + } +} + /** * Remove no-op operators from the query plan that do not make any modifications. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 223ef652d2f8..76064097afe1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -563,7 +563,7 @@ case class Range( * * @param groupingExpressions expressions for grouping keys * @param aggregateExpressions expressions for a project list, which could contain - * [[AggregateFunction]]s. + * [[AggregateExpression]]s. * * Note: Currently, aggregateExpressions is the project list of this Group by operator. Before * separating projection from grouping and aggregate, we should avoid expression-level optimization diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala new file mode 100644 index 000000000000..50b0451e9ad9 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala @@ -0,0 +1,83 @@ +/* + * 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.plans.PlanTest +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.RuleExecutor + +class RemoveRedundantAggregatesSuite extends PlanTest { + + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = Batch("RemoveRedundantAggregates", FixedPoint(10), + RemoveRedundantAggregates) :: Nil + } + + test("Remove redundant aggregate") { + val relation = LocalRelation('a.int, 'b.int) + val query = relation + .groupBy('a)('a, count('b)) + .groupBy('a)('a) + .analyze + val expected = relation + .groupBy('a)('a) + .analyze + val optimized = Optimize.execute(query) + comparePlans(optimized, expected) + } + + test("Remove 2 redundant aggregates") { + val relation = LocalRelation('a.int, 'b.int) + val query = relation + .groupBy('a)('a, count('b)) + .groupBy('a)('a) + .groupBy('a)('a) + .analyze + val expected = relation + .groupBy('a)('a) + .analyze + val optimized = Optimize.execute(query) + comparePlans(optimized, expected) + } + + test("Remove redundant aggregate with different grouping") { + val relation = LocalRelation('a.int, 'b.int) + val query = relation + .groupBy('a, 'b)('a) + .groupBy('a)('a) + .analyze + val expected = relation + .groupBy('a)('a) + .analyze + val optimized = Optimize.execute(query) + comparePlans(optimized, expected) + } + + test("Keep non-redundant aggregate") { + val relation = LocalRelation('a.int, 'b.int) + val query = relation + .groupBy('a)('a, first('b) as 'b) + // The count would change if we remove the first aggregate + .groupBy('a)('a, count('b)) + .analyze + val optimized = Optimize.execute(query) + comparePlans(optimized, query) + } +} diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt index b346701fa314..2facb5e5d237 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt @@ -1,135 +1,133 @@ == Physical Plan == -TakeOrderedAndProject (131) -+- * HashAggregate (130) - +- Exchange (129) - +- * HashAggregate (128) - +- * Expand (127) - +- Union (126) - :- * Project (87) - : +- * Filter (86) - : +- * HashAggregate (85) - : +- Exchange (84) - : +- * HashAggregate (83) - : +- * Project (82) - : +- * BroadcastHashJoin Inner BuildRight (81) - : :- * Project (71) - : : +- * BroadcastHashJoin Inner BuildRight (70) - : : :- SortMergeJoin LeftSemi (64) +TakeOrderedAndProject (129) ++- * HashAggregate (128) + +- Exchange (127) + +- * HashAggregate (126) + +- * Expand (125) + +- Union (124) + :- * Project (85) + : +- * Filter (84) + : +- * HashAggregate (83) + : +- Exchange (82) + : +- * HashAggregate (81) + : +- * Project (80) + : +- * BroadcastHashJoin Inner BuildRight (79) + : :- * Project (69) + : : +- * BroadcastHashJoin Inner BuildRight (68) + : : :- SortMergeJoin LeftSemi (62) : : : :- * Sort (5) : : : : +- Exchange (4) : : : : +- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.store_sales (1) - : : : +- * Sort (63) - : : : +- Exchange (62) - : : : +- * Project (61) - : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : +- * Sort (61) + : : : +- Exchange (60) + : : : +- * Project (59) + : : : +- * BroadcastHashJoin Inner BuildRight (58) : : : :- * Filter (8) : : : : +- * ColumnarToRow (7) : : : : +- Scan parquet default.item (6) - : : : +- BroadcastExchange (59) - : : : +- * HashAggregate (58) - : : : +- * HashAggregate (57) - : : : +- * HashAggregate (56) - : : : +- Exchange (55) - : : : +- * HashAggregate (54) - : : : +- SortMergeJoin LeftSemi (53) - : : : :- SortMergeJoin LeftSemi (41) - : : : : :- * Sort (26) - : : : : : +- Exchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (18) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : : :- * Filter (11) - : : : : : : : +- * ColumnarToRow (10) - : : : : : : : +- Scan parquet default.store_sales (9) - : : : : : : +- BroadcastExchange (16) - : : : : : : +- * Project (15) - : : : : : : +- * Filter (14) - : : : : : : +- * ColumnarToRow (13) - : : : : : : +- Scan parquet default.date_dim (12) - : : : : : +- BroadcastExchange (22) - : : : : : +- * Filter (21) - : : : : : +- * ColumnarToRow (20) - : : : : : +- Scan parquet default.item (19) - : : : : +- * Sort (40) - : : : : +- Exchange (39) - : : : : +- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Project (32) - : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : :- * Filter (29) - : : : : : : +- * ColumnarToRow (28) - : : : : : : +- Scan parquet default.catalog_sales (27) - : : : : : +- ReusedExchange (30) - : : : : +- BroadcastExchange (36) - : : : : +- * Filter (35) - : : : : +- * ColumnarToRow (34) - : : : : +- Scan parquet default.item (33) - : : : +- * Sort (52) - : : : +- Exchange (51) - : : : +- * Project (50) - : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * Project (47) - : : : : +- * BroadcastHashJoin Inner BuildRight (46) - : : : : :- * Filter (44) - : : : : : +- * ColumnarToRow (43) - : : : : : +- Scan parquet default.web_sales (42) - : : : : +- ReusedExchange (45) - : : : +- ReusedExchange (48) - : : +- BroadcastExchange (69) - : : +- * Project (68) - : : +- * Filter (67) - : : +- * ColumnarToRow (66) - : : +- Scan parquet default.date_dim (65) - : +- BroadcastExchange (80) - : +- SortMergeJoin LeftSemi (79) - : :- * Sort (76) - : : +- Exchange (75) - : : +- * Filter (74) - : : +- * ColumnarToRow (73) - : : +- Scan parquet default.item (72) - : +- * Sort (78) - : +- ReusedExchange (77) - :- * Project (106) - : +- * Filter (105) - : +- * HashAggregate (104) - : +- Exchange (103) - : +- * HashAggregate (102) - : +- * Project (101) - : +- * BroadcastHashJoin Inner BuildRight (100) - : :- * Project (98) - : : +- * BroadcastHashJoin Inner BuildRight (97) - : : :- SortMergeJoin LeftSemi (95) - : : : :- * Sort (92) - : : : : +- Exchange (91) - : : : : +- * Filter (90) - : : : : +- * ColumnarToRow (89) - : : : : +- Scan parquet default.catalog_sales (88) - : : : +- * Sort (94) - : : : +- ReusedExchange (93) - : : +- ReusedExchange (96) - : +- ReusedExchange (99) - +- * Project (125) - +- * Filter (124) - +- * HashAggregate (123) - +- Exchange (122) - +- * HashAggregate (121) - +- * Project (120) - +- * BroadcastHashJoin Inner BuildRight (119) - :- * Project (117) - : +- * BroadcastHashJoin Inner BuildRight (116) - : :- SortMergeJoin LeftSemi (114) - : : :- * Sort (111) - : : : +- Exchange (110) - : : : +- * Filter (109) - : : : +- * ColumnarToRow (108) - : : : +- Scan parquet default.web_sales (107) - : : +- * Sort (113) - : : +- ReusedExchange (112) - : +- ReusedExchange (115) - +- ReusedExchange (118) + : : : +- BroadcastExchange (57) + : : : +- * HashAggregate (56) + : : : +- Exchange (55) + : : : +- * HashAggregate (54) + : : : +- SortMergeJoin LeftSemi (53) + : : : :- SortMergeJoin LeftSemi (41) + : : : : :- * Sort (26) + : : : : : +- Exchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * Project (18) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : : :- * Filter (11) + : : : : : : : +- * ColumnarToRow (10) + : : : : : : : +- Scan parquet default.store_sales (9) + : : : : : : +- BroadcastExchange (16) + : : : : : : +- * Project (15) + : : : : : : +- * Filter (14) + : : : : : : +- * ColumnarToRow (13) + : : : : : : +- Scan parquet default.date_dim (12) + : : : : : +- BroadcastExchange (22) + : : : : : +- * Filter (21) + : : : : : +- * ColumnarToRow (20) + : : : : : +- Scan parquet default.item (19) + : : : : +- * Sort (40) + : : : : +- Exchange (39) + : : : : +- * Project (38) + : : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : : :- * Project (32) + : : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : : :- * Filter (29) + : : : : : : +- * ColumnarToRow (28) + : : : : : : +- Scan parquet default.catalog_sales (27) + : : : : : +- ReusedExchange (30) + : : : : +- BroadcastExchange (36) + : : : : +- * Filter (35) + : : : : +- * ColumnarToRow (34) + : : : : +- Scan parquet default.item (33) + : : : +- * Sort (52) + : : : +- Exchange (51) + : : : +- * Project (50) + : : : +- * BroadcastHashJoin Inner BuildRight (49) + : : : :- * Project (47) + : : : : +- * BroadcastHashJoin Inner BuildRight (46) + : : : : :- * Filter (44) + : : : : : +- * ColumnarToRow (43) + : : : : : +- Scan parquet default.web_sales (42) + : : : : +- ReusedExchange (45) + : : : +- ReusedExchange (48) + : : +- BroadcastExchange (67) + : : +- * Project (66) + : : +- * Filter (65) + : : +- * ColumnarToRow (64) + : : +- Scan parquet default.date_dim (63) + : +- BroadcastExchange (78) + : +- SortMergeJoin LeftSemi (77) + : :- * Sort (74) + : : +- Exchange (73) + : : +- * Filter (72) + : : +- * ColumnarToRow (71) + : : +- Scan parquet default.item (70) + : +- * Sort (76) + : +- ReusedExchange (75) + :- * Project (104) + : +- * Filter (103) + : +- * HashAggregate (102) + : +- Exchange (101) + : +- * HashAggregate (100) + : +- * Project (99) + : +- * BroadcastHashJoin Inner BuildRight (98) + : :- * Project (96) + : : +- * BroadcastHashJoin Inner BuildRight (95) + : : :- SortMergeJoin LeftSemi (93) + : : : :- * Sort (90) + : : : : +- Exchange (89) + : : : : +- * Filter (88) + : : : : +- * ColumnarToRow (87) + : : : : +- Scan parquet default.catalog_sales (86) + : : : +- * Sort (92) + : : : +- ReusedExchange (91) + : : +- ReusedExchange (94) + : +- ReusedExchange (97) + +- * Project (123) + +- * Filter (122) + +- * HashAggregate (121) + +- Exchange (120) + +- * HashAggregate (119) + +- * Project (118) + +- * BroadcastHashJoin Inner BuildRight (117) + :- * Project (115) + : +- * BroadcastHashJoin Inner BuildRight (114) + : :- SortMergeJoin LeftSemi (112) + : : :- * Sort (109) + : : : +- Exchange (108) + : : : +- * Filter (107) + : : : +- * ColumnarToRow (106) + : : : +- Scan parquet default.web_sales (105) + : : +- * Sort (111) + : : +- ReusedExchange (110) + : +- ReusedExchange (113) + +- ReusedExchange (116) (1) Scan parquet default.store_sales @@ -383,496 +381,482 @@ Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#14, class_id#15, category_id#16] -(57) HashAggregate [codegen id : 16] -Input [3]: [brand_id#14, class_id#15, category_id#16] -Keys [3]: [brand_id#14, class_id#15, category_id#16] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#14, class_id#15, category_id#16] - -(58) HashAggregate [codegen id : 16] -Input [3]: [brand_id#14, class_id#15, category_id#16] -Keys [3]: [brand_id#14, class_id#15, category_id#16] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#14, class_id#15, category_id#16] - -(59) BroadcastExchange +(57) BroadcastExchange Input [3]: [brand_id#14, class_id#15, category_id#16] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#26] -(60) BroadcastHashJoin [codegen id : 17] +(58) BroadcastHashJoin [codegen id : 17] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#14, class_id#15, category_id#16] Join condition: None -(61) Project [codegen id : 17] +(59) Project [codegen id : 17] Output [1]: [i_item_sk#6 AS ss_item_sk#27] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#14, class_id#15, category_id#16] -(62) Exchange +(60) Exchange Input [1]: [ss_item_sk#27] Arguments: hashpartitioning(ss_item_sk#27, 5), true, [id=#28] -(63) Sort [codegen id : 18] +(61) Sort [codegen id : 18] Input [1]: [ss_item_sk#27] Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 -(64) SortMergeJoin +(62) SortMergeJoin Left keys [1]: [ss_item_sk#2] Right keys [1]: [ss_item_sk#27] Join condition: None -(65) Scan parquet default.date_dim +(63) Scan parquet default.date_dim Output [3]: [d_date_sk#10, d_year#11, d_moy#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(66) ColumnarToRow [codegen id : 19] +(64) ColumnarToRow [codegen id : 19] Input [3]: [d_date_sk#10, d_year#11, d_moy#29] -(67) Filter [codegen id : 19] +(65) Filter [codegen id : 19] Input [3]: [d_date_sk#10, d_year#11, d_moy#29] Condition : ((((isnotnull(d_year#11) AND isnotnull(d_moy#29)) AND (d_year#11 = 2001)) AND (d_moy#29 = 11)) AND isnotnull(d_date_sk#10)) -(68) Project [codegen id : 19] +(66) Project [codegen id : 19] Output [1]: [d_date_sk#10] Input [3]: [d_date_sk#10, d_year#11, d_moy#29] -(69) BroadcastExchange +(67) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] -(70) BroadcastHashJoin [codegen id : 38] +(68) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(71) Project [codegen id : 38] +(69) Project [codegen id : 38] Output [3]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4] Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(72) Scan parquet default.item +(70) Scan parquet default.item Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(73) ColumnarToRow [codegen id : 20] +(71) ColumnarToRow [codegen id : 20] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(74) Filter [codegen id : 20] +(72) Filter [codegen id : 20] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : isnotnull(i_item_sk#6) -(75) Exchange +(73) Exchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: hashpartitioning(i_item_sk#6, 5), true, [id=#31] -(76) Sort [codegen id : 21] +(74) Sort [codegen id : 21] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: [i_item_sk#6 ASC NULLS FIRST], false, 0 -(77) ReusedExchange [Reuses operator id: 62] +(75) ReusedExchange [Reuses operator id: 60] Output [1]: [ss_item_sk#27] -(78) Sort [codegen id : 37] +(76) Sort [codegen id : 37] Input [1]: [ss_item_sk#27] Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 -(79) SortMergeJoin +(77) SortMergeJoin Left keys [1]: [i_item_sk#6] Right keys [1]: [ss_item_sk#27] Join condition: None -(80) BroadcastExchange +(78) BroadcastExchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#32] -(81) BroadcastHashJoin [codegen id : 38] +(79) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#6] Join condition: None -(82) Project [codegen id : 38] +(80) Project [codegen id : 38] Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#7, i_class_id#8, i_category_id#9] Input [7]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(83) HashAggregate [codegen id : 38] +(81) HashAggregate [codegen id : 38] Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#7, i_class_id#8, i_category_id#9] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#33, isEmpty#34, count#35] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#36, isEmpty#37, count#38] -(84) Exchange +(82) Exchange Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#36, isEmpty#37, count#38] Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#39] -(85) HashAggregate [codegen id : 39] +(83) HashAggregate [codegen id : 39] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#36, isEmpty#37, count#38] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40, count(1)#41] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40 AS sales#42, count(1)#41 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44] -(86) Filter [codegen id : 39] +(84) Filter [codegen id : 39] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44 as decimal(32,6)) > cast(Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(87) Project [codegen id : 39] +(85) Project [codegen id : 39] Output [6]: [sales#42, number_sales#43, store AS channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44] -(88) Scan parquet default.catalog_sales +(86) Scan parquet default.catalog_sales Output [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(89) ColumnarToRow [codegen id : 40] +(87) ColumnarToRow [codegen id : 40] Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] -(90) Filter [codegen id : 40] +(88) Filter [codegen id : 40] Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] Condition : (isnotnull(cs_item_sk#19) AND isnotnull(cs_sold_date_sk#18)) -(91) Exchange +(89) Exchange Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] Arguments: hashpartitioning(cs_item_sk#19, 5), true, [id=#50] -(92) Sort [codegen id : 41] +(90) Sort [codegen id : 41] Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] Arguments: [cs_item_sk#19 ASC NULLS FIRST], false, 0 -(93) ReusedExchange [Reuses operator id: 62] +(91) ReusedExchange [Reuses operator id: 60] Output [1]: [ss_item_sk#27] -(94) Sort [codegen id : 57] +(92) Sort [codegen id : 57] Input [1]: [ss_item_sk#27] Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 -(95) SortMergeJoin +(93) SortMergeJoin Left keys [1]: [cs_item_sk#19] Right keys [1]: [ss_item_sk#27] Join condition: None -(96) ReusedExchange [Reuses operator id: 69] +(94) ReusedExchange [Reuses operator id: 67] Output [1]: [d_date_sk#10] -(97) BroadcastHashJoin [codegen id : 77] +(95) BroadcastHashJoin [codegen id : 77] Left keys [1]: [cs_sold_date_sk#18] Right keys [1]: [d_date_sk#10] Join condition: None -(98) Project [codegen id : 77] +(96) Project [codegen id : 77] Output [3]: [cs_item_sk#19, cs_quantity#48, cs_list_price#49] Input [5]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49, d_date_sk#10] -(99) ReusedExchange [Reuses operator id: 80] +(97) ReusedExchange [Reuses operator id: 78] Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(100) BroadcastHashJoin [codegen id : 77] +(98) BroadcastHashJoin [codegen id : 77] Left keys [1]: [cs_item_sk#19] Right keys [1]: [i_item_sk#6] Join condition: None -(101) Project [codegen id : 77] +(99) Project [codegen id : 77] Output [5]: [cs_quantity#48, cs_list_price#49, i_brand_id#7, i_class_id#8, i_category_id#9] Input [7]: [cs_item_sk#19, cs_quantity#48, cs_list_price#49, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(102) HashAggregate [codegen id : 77] +(100) HashAggregate [codegen id : 77] Input [5]: [cs_quantity#48, cs_list_price#49, i_brand_id#7, i_class_id#8, i_category_id#9] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#51, isEmpty#52, count#53] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#54, isEmpty#55, count#56] -(103) Exchange +(101) Exchange Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#54, isEmpty#55, count#56] Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#57] -(104) HashAggregate [codegen id : 78] +(102) HashAggregate [codegen id : 78] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#54, isEmpty#55, count#56] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58, count(1)#59] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sales#60, count(1)#59 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62] -(105) Filter [codegen id : 78] +(103) Filter [codegen id : 78] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(106) Project [codegen id : 78] +(104) Project [codegen id : 78] Output [6]: [sales#60, number_sales#61, catalog AS channel#63, i_brand_id#7, i_class_id#8, i_category_id#9] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62] -(107) Scan parquet default.web_sales +(105) Scan parquet default.web_sales Output [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(108) ColumnarToRow [codegen id : 79] +(106) ColumnarToRow [codegen id : 79] Input [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] -(109) Filter [codegen id : 79] +(107) Filter [codegen id : 79] Input [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] Condition : (isnotnull(ws_item_sk#23) AND isnotnull(ws_sold_date_sk#22)) -(110) Exchange +(108) Exchange Input [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] Arguments: hashpartitioning(ws_item_sk#23, 5), true, [id=#66] -(111) Sort [codegen id : 80] +(109) Sort [codegen id : 80] Input [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] Arguments: [ws_item_sk#23 ASC NULLS FIRST], false, 0 -(112) ReusedExchange [Reuses operator id: 62] +(110) ReusedExchange [Reuses operator id: 60] Output [1]: [ss_item_sk#27] -(113) Sort [codegen id : 96] +(111) Sort [codegen id : 96] Input [1]: [ss_item_sk#27] Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 -(114) SortMergeJoin +(112) SortMergeJoin Left keys [1]: [ws_item_sk#23] Right keys [1]: [ss_item_sk#27] Join condition: None -(115) ReusedExchange [Reuses operator id: 69] +(113) ReusedExchange [Reuses operator id: 67] Output [1]: [d_date_sk#10] -(116) BroadcastHashJoin [codegen id : 116] +(114) BroadcastHashJoin [codegen id : 116] Left keys [1]: [ws_sold_date_sk#22] Right keys [1]: [d_date_sk#10] Join condition: None -(117) Project [codegen id : 116] +(115) Project [codegen id : 116] Output [3]: [ws_item_sk#23, ws_quantity#64, ws_list_price#65] Input [5]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65, d_date_sk#10] -(118) ReusedExchange [Reuses operator id: 80] +(116) ReusedExchange [Reuses operator id: 78] Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(119) BroadcastHashJoin [codegen id : 116] +(117) BroadcastHashJoin [codegen id : 116] Left keys [1]: [ws_item_sk#23] Right keys [1]: [i_item_sk#6] Join condition: None -(120) Project [codegen id : 116] +(118) Project [codegen id : 116] Output [5]: [ws_quantity#64, ws_list_price#65, i_brand_id#7, i_class_id#8, i_category_id#9] Input [7]: [ws_item_sk#23, ws_quantity#64, ws_list_price#65, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(121) HashAggregate [codegen id : 116] +(119) HashAggregate [codegen id : 116] Input [5]: [ws_quantity#64, ws_list_price#65, i_brand_id#7, i_class_id#8, i_category_id#9] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#67, isEmpty#68, count#69] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#70, isEmpty#71, count#72] -(122) Exchange +(120) Exchange Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#70, isEmpty#71, count#72] Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#73] -(123) HashAggregate [codegen id : 117] +(121) HashAggregate [codegen id : 117] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#70, isEmpty#71, count#72] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74, count(1)#75] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74 AS sales#76, count(1)#75 AS number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78] -(124) Filter [codegen id : 117] +(122) Filter [codegen id : 117] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(125) Project [codegen id : 117] +(123) Project [codegen id : 117] Output [6]: [sales#76, number_sales#77, web AS channel#79, i_brand_id#7, i_class_id#8, i_category_id#9] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78] -(126) Union +(124) Union -(127) Expand [codegen id : 118] +(125) Expand [codegen id : 118] Input [6]: [sales#42, number_sales#43, channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: [List(sales#42, number_sales#43, channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, 0), List(sales#42, number_sales#43, channel#47, i_brand_id#7, i_class_id#8, null, 1), List(sales#42, number_sales#43, channel#47, i_brand_id#7, null, null, 3), List(sales#42, number_sales#43, channel#47, null, null, null, 7), List(sales#42, number_sales#43, null, null, null, null, 15)], [sales#42, number_sales#43, channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84] -(128) HashAggregate [codegen id : 118] +(126) HashAggregate [codegen id : 118] Input [7]: [sales#42, number_sales#43, channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84] Keys [5]: [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84] Functions [2]: [partial_sum(sales#42), partial_sum(number_sales#43)] Aggregate Attributes [3]: [sum#85, isEmpty#86, sum#87] Results [8]: [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84, sum#88, isEmpty#89, sum#90] -(129) Exchange +(127) Exchange Input [8]: [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84, sum#88, isEmpty#89, sum#90] Arguments: hashpartitioning(channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84, 5), true, [id=#91] -(130) HashAggregate [codegen id : 119] +(128) HashAggregate [codegen id : 119] Input [8]: [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84, sum#88, isEmpty#89, sum#90] Keys [5]: [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, spark_grouping_id#84] Functions [2]: [sum(sales#42), sum(number_sales#43)] Aggregate Attributes [2]: [sum(sales#42)#92, sum(number_sales#43)#93] Results [6]: [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, sum(sales#42)#92 AS sum(sales)#94, sum(number_sales#43)#93 AS sum(number_sales)#95] -(131) TakeOrderedAndProject +(129) TakeOrderedAndProject Input [6]: [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, sum(sales)#94, sum(number_sales)#95] Arguments: 100, [channel#80 ASC NULLS FIRST, i_brand_id#81 ASC NULLS FIRST, i_class_id#82 ASC NULLS FIRST, i_category_id#83 ASC NULLS FIRST], [channel#80, i_brand_id#81, i_class_id#82, i_category_id#83, sum(sales)#94, sum(number_sales)#95] ===== Subqueries ===== -Subquery:1 Hosting operator id = 86 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* HashAggregate (157) -+- Exchange (156) - +- * HashAggregate (155) - +- Union (154) - :- * Project (141) - : +- * BroadcastHashJoin Inner BuildRight (140) - : :- * Filter (134) - : : +- * ColumnarToRow (133) - : : +- Scan parquet default.store_sales (132) - : +- BroadcastExchange (139) - : +- * Project (138) - : +- * Filter (137) - : +- * ColumnarToRow (136) - : +- Scan parquet default.date_dim (135) - :- * Project (147) - : +- * BroadcastHashJoin Inner BuildRight (146) - : :- * Filter (144) - : : +- * ColumnarToRow (143) - : : +- Scan parquet default.catalog_sales (142) - : +- ReusedExchange (145) - +- * Project (153) - +- * BroadcastHashJoin Inner BuildRight (152) - :- * Filter (150) - : +- * ColumnarToRow (149) - : +- Scan parquet default.web_sales (148) - +- ReusedExchange (151) - - -(132) Scan parquet default.store_sales +Subquery:1 Hosting operator id = 84 Hosting Expression = Subquery scalar-subquery#45, [id=#46] +* HashAggregate (155) ++- Exchange (154) + +- * HashAggregate (153) + +- Union (152) + :- * Project (139) + : +- * BroadcastHashJoin Inner BuildRight (138) + : :- * Filter (132) + : : +- * ColumnarToRow (131) + : : +- Scan parquet default.store_sales (130) + : +- BroadcastExchange (137) + : +- * Project (136) + : +- * Filter (135) + : +- * ColumnarToRow (134) + : +- Scan parquet default.date_dim (133) + :- * Project (145) + : +- * BroadcastHashJoin Inner BuildRight (144) + : :- * Filter (142) + : : +- * ColumnarToRow (141) + : : +- Scan parquet default.catalog_sales (140) + : +- ReusedExchange (143) + +- * Project (151) + +- * BroadcastHashJoin Inner BuildRight (150) + :- * Filter (148) + : +- * ColumnarToRow (147) + : +- Scan parquet default.web_sales (146) + +- ReusedExchange (149) + + +(130) Scan parquet default.store_sales Output [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(133) ColumnarToRow [codegen id : 2] +(131) ColumnarToRow [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] -(134) Filter [codegen id : 2] +(132) Filter [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Condition : isnotnull(ss_sold_date_sk#1) -(135) Scan parquet default.date_dim +(133) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_year#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(136) ColumnarToRow [codegen id : 1] +(134) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] -(137) Filter [codegen id : 1] +(135) Filter [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] Condition : (((isnotnull(d_year#11) AND (d_year#11 >= 1999)) AND (d_year#11 <= 2001)) AND isnotnull(d_date_sk#10)) -(138) Project [codegen id : 1] +(136) Project [codegen id : 1] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_year#11] -(139) BroadcastExchange +(137) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#96] -(140) BroadcastHashJoin [codegen id : 2] +(138) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(141) Project [codegen id : 2] +(139) Project [codegen id : 2] Output [2]: [ss_quantity#3 AS quantity#97, ss_list_price#4 AS list_price#98] Input [4]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(142) Scan parquet default.catalog_sales +(140) Scan parquet default.catalog_sales Output [3]: [cs_sold_date_sk#18, cs_quantity#48, cs_list_price#49] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(143) ColumnarToRow [codegen id : 4] +(141) ColumnarToRow [codegen id : 4] Input [3]: [cs_sold_date_sk#18, cs_quantity#48, cs_list_price#49] -(144) Filter [codegen id : 4] +(142) Filter [codegen id : 4] Input [3]: [cs_sold_date_sk#18, cs_quantity#48, cs_list_price#49] Condition : isnotnull(cs_sold_date_sk#18) -(145) ReusedExchange [Reuses operator id: 139] +(143) ReusedExchange [Reuses operator id: 137] Output [1]: [d_date_sk#10] -(146) BroadcastHashJoin [codegen id : 4] +(144) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#18] Right keys [1]: [d_date_sk#10] Join condition: None -(147) Project [codegen id : 4] +(145) Project [codegen id : 4] Output [2]: [cs_quantity#48 AS quantity#99, cs_list_price#49 AS list_price#100] Input [4]: [cs_sold_date_sk#18, cs_quantity#48, cs_list_price#49, d_date_sk#10] -(148) Scan parquet default.web_sales +(146) Scan parquet default.web_sales Output [3]: [ws_sold_date_sk#22, ws_quantity#64, ws_list_price#65] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(149) ColumnarToRow [codegen id : 6] +(147) ColumnarToRow [codegen id : 6] Input [3]: [ws_sold_date_sk#22, ws_quantity#64, ws_list_price#65] -(150) Filter [codegen id : 6] +(148) Filter [codegen id : 6] Input [3]: [ws_sold_date_sk#22, ws_quantity#64, ws_list_price#65] Condition : isnotnull(ws_sold_date_sk#22) -(151) ReusedExchange [Reuses operator id: 139] +(149) ReusedExchange [Reuses operator id: 137] Output [1]: [d_date_sk#10] -(152) BroadcastHashJoin [codegen id : 6] +(150) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#22] Right keys [1]: [d_date_sk#10] Join condition: None -(153) Project [codegen id : 6] +(151) Project [codegen id : 6] Output [2]: [ws_quantity#64 AS quantity#101, ws_list_price#65 AS list_price#102] Input [4]: [ws_sold_date_sk#22, ws_quantity#64, ws_list_price#65, d_date_sk#10] -(154) Union +(152) Union -(155) HashAggregate [codegen id : 7] +(153) HashAggregate [codegen id : 7] Input [2]: [quantity#97, list_price#98] Keys: [] Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#97 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#98 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [2]: [sum#103, count#104] Results [2]: [sum#105, count#106] -(156) Exchange +(154) Exchange Input [2]: [sum#105, count#106] Arguments: SinglePartition, true, [id=#107] -(157) HashAggregate [codegen id : 8] +(155) HashAggregate [codegen id : 8] Input [2]: [sum#105, count#106] Keys: [] Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#97 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#98 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#97 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#98 as decimal(12,2)))), DecimalType(18,2), true))#108] Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#97 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#98 as decimal(12,2)))), DecimalType(18,2), true))#108 AS average_sales#109] -Subquery:2 Hosting operator id = 105 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:2 Hosting operator id = 103 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:3 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:3 Hosting operator id = 122 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt index 5b93392d023d..04bc0c86a16f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt @@ -88,82 +88,80 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastExchange #5 WholeStageCodegen (16) HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (15) - HashAggregate [brand_id,class_id,category_id] - InputAdapter - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (6) - Sort [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] + InputAdapter + Exchange [brand_id,class_id,category_id] #6 + WholeStageCodegen (15) + HashAggregate [brand_id,class_id,category_id] + InputAdapter + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (6) + Sort [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #7 + WholeStageCodegen (5) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk,ss_sold_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (10) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #10 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] + BroadcastExchange #8 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (10) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #10 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk,cs_sold_date_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #8 + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] InputAdapter - BroadcastExchange #11 - WholeStageCodegen (8) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (14) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #12 - WholeStageCodegen (13) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk,ws_sold_date_sk] + ReusedExchange [d_date_sk] #8 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (8) + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (14) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #12 + WholeStageCodegen (13) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk,ws_sold_date_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #8 + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + ReusedExchange [d_date_sk] #8 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 InputAdapter BroadcastExchange #13 WholeStageCodegen (19) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt index 3f0cc9e7acb1..28a9589165f2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt @@ -1,119 +1,117 @@ == Physical Plan == -TakeOrderedAndProject (115) -+- * HashAggregate (114) - +- Exchange (113) - +- * HashAggregate (112) - +- * Expand (111) - +- Union (110) - :- * Project (77) - : +- * Filter (76) - : +- * HashAggregate (75) - : +- Exchange (74) - : +- * HashAggregate (73) - : +- * Project (72) - : +- * BroadcastHashJoin Inner BuildRight (71) - : :- * Project (65) - : : +- * BroadcastHashJoin Inner BuildRight (64) - : : :- * BroadcastHashJoin LeftSemi BuildRight (57) +TakeOrderedAndProject (113) ++- * HashAggregate (112) + +- Exchange (111) + +- * HashAggregate (110) + +- * Expand (109) + +- Union (108) + :- * Project (75) + : +- * Filter (74) + : +- * HashAggregate (73) + : +- Exchange (72) + : +- * HashAggregate (71) + : +- * Project (70) + : +- * BroadcastHashJoin Inner BuildRight (69) + : :- * Project (63) + : : +- * BroadcastHashJoin Inner BuildRight (62) + : : :- * BroadcastHashJoin LeftSemi BuildRight (55) : : : :- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (56) - : : : +- * Project (55) - : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : +- BroadcastExchange (54) + : : : +- * Project (53) + : : : +- * BroadcastHashJoin Inner BuildRight (52) : : : :- * Filter (6) : : : : +- * ColumnarToRow (5) : : : : +- Scan parquet default.item (4) - : : : +- BroadcastExchange (53) - : : : +- * HashAggregate (52) - : : : +- * HashAggregate (51) - : : : +- * HashAggregate (50) - : : : +- Exchange (49) - : : : +- * HashAggregate (48) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (36) - : : : : :- * Project (22) - : : : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : : : :- * Project (15) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : :- * Filter (9) - : : : : : : : +- * ColumnarToRow (8) - : : : : : : : +- Scan parquet default.store_sales (7) - : : : : : : +- BroadcastExchange (13) - : : : : : : +- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet default.item (10) - : : : : : +- BroadcastExchange (20) - : : : : : +- * Project (19) - : : : : : +- * Filter (18) - : : : : : +- * ColumnarToRow (17) - : : : : : +- Scan parquet default.date_dim (16) - : : : : +- BroadcastExchange (35) - : : : : +- * Project (34) - : : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : : :- * Project (31) - : : : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : : : :- * Filter (25) - : : : : : : +- * ColumnarToRow (24) - : : : : : : +- Scan parquet default.catalog_sales (23) - : : : : : +- BroadcastExchange (29) - : : : : : +- * Filter (28) - : : : : : +- * ColumnarToRow (27) - : : : : : +- Scan parquet default.item (26) - : : : : +- ReusedExchange (32) - : : : +- BroadcastExchange (46) - : : : +- * Project (45) - : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : :- * Project (42) - : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : :- * Filter (39) - : : : : : +- * ColumnarToRow (38) - : : : : : +- Scan parquet default.web_sales (37) - : : : : +- ReusedExchange (40) - : : : +- ReusedExchange (43) - : : +- BroadcastExchange (63) - : : +- * BroadcastHashJoin LeftSemi BuildRight (62) - : : :- * Filter (60) - : : : +- * ColumnarToRow (59) - : : : +- Scan parquet default.item (58) - : : +- ReusedExchange (61) - : +- BroadcastExchange (70) - : +- * Project (69) - : +- * Filter (68) - : +- * ColumnarToRow (67) - : +- Scan parquet default.date_dim (66) - :- * Project (93) - : +- * Filter (92) - : +- * HashAggregate (91) - : +- Exchange (90) - : +- * HashAggregate (89) - : +- * Project (88) - : +- * BroadcastHashJoin Inner BuildRight (87) - : :- * Project (85) - : : +- * BroadcastHashJoin Inner BuildRight (84) - : : :- * BroadcastHashJoin LeftSemi BuildRight (82) - : : : :- * Filter (80) - : : : : +- * ColumnarToRow (79) - : : : : +- Scan parquet default.catalog_sales (78) - : : : +- ReusedExchange (81) - : : +- ReusedExchange (83) - : +- ReusedExchange (86) - +- * Project (109) - +- * Filter (108) - +- * HashAggregate (107) - +- Exchange (106) - +- * HashAggregate (105) - +- * Project (104) - +- * BroadcastHashJoin Inner BuildRight (103) - :- * Project (101) - : +- * BroadcastHashJoin Inner BuildRight (100) - : :- * BroadcastHashJoin LeftSemi BuildRight (98) - : : :- * Filter (96) - : : : +- * ColumnarToRow (95) - : : : +- Scan parquet default.web_sales (94) - : : +- ReusedExchange (97) - : +- ReusedExchange (99) - +- ReusedExchange (102) + : : : +- BroadcastExchange (51) + : : : +- * HashAggregate (50) + : : : +- Exchange (49) + : : : +- * HashAggregate (48) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (36) + : : : : :- * Project (22) + : : : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : : : :- * Project (15) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : : :- * Filter (9) + : : : : : : : +- * ColumnarToRow (8) + : : : : : : : +- Scan parquet default.store_sales (7) + : : : : : : +- BroadcastExchange (13) + : : : : : : +- * Filter (12) + : : : : : : +- * ColumnarToRow (11) + : : : : : : +- Scan parquet default.item (10) + : : : : : +- BroadcastExchange (20) + : : : : : +- * Project (19) + : : : : : +- * Filter (18) + : : : : : +- * ColumnarToRow (17) + : : : : : +- Scan parquet default.date_dim (16) + : : : : +- BroadcastExchange (35) + : : : : +- * Project (34) + : : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : : :- * Project (31) + : : : : : +- * BroadcastHashJoin Inner BuildRight (30) + : : : : : :- * Filter (25) + : : : : : : +- * ColumnarToRow (24) + : : : : : : +- Scan parquet default.catalog_sales (23) + : : : : : +- BroadcastExchange (29) + : : : : : +- * Filter (28) + : : : : : +- * ColumnarToRow (27) + : : : : : +- Scan parquet default.item (26) + : : : : +- ReusedExchange (32) + : : : +- BroadcastExchange (46) + : : : +- * Project (45) + : : : +- * BroadcastHashJoin Inner BuildRight (44) + : : : :- * Project (42) + : : : : +- * BroadcastHashJoin Inner BuildRight (41) + : : : : :- * Filter (39) + : : : : : +- * ColumnarToRow (38) + : : : : : +- Scan parquet default.web_sales (37) + : : : : +- ReusedExchange (40) + : : : +- ReusedExchange (43) + : : +- BroadcastExchange (61) + : : +- * BroadcastHashJoin LeftSemi BuildRight (60) + : : :- * Filter (58) + : : : +- * ColumnarToRow (57) + : : : +- Scan parquet default.item (56) + : : +- ReusedExchange (59) + : +- BroadcastExchange (68) + : +- * Project (67) + : +- * Filter (66) + : +- * ColumnarToRow (65) + : +- Scan parquet default.date_dim (64) + :- * Project (91) + : +- * Filter (90) + : +- * HashAggregate (89) + : +- Exchange (88) + : +- * HashAggregate (87) + : +- * Project (86) + : +- * BroadcastHashJoin Inner BuildRight (85) + : :- * Project (83) + : : +- * BroadcastHashJoin Inner BuildRight (82) + : : :- * BroadcastHashJoin LeftSemi BuildRight (80) + : : : :- * Filter (78) + : : : : +- * ColumnarToRow (77) + : : : : +- Scan parquet default.catalog_sales (76) + : : : +- ReusedExchange (79) + : : +- ReusedExchange (81) + : +- ReusedExchange (84) + +- * Project (107) + +- * Filter (106) + +- * HashAggregate (105) + +- Exchange (104) + +- * HashAggregate (103) + +- * Project (102) + +- * BroadcastHashJoin Inner BuildRight (101) + :- * Project (99) + : +- * BroadcastHashJoin Inner BuildRight (98) + : :- * BroadcastHashJoin LeftSemi BuildRight (96) + : : :- * Filter (94) + : : : +- * ColumnarToRow (93) + : : : +- Scan parquet default.web_sales (92) + : : +- ReusedExchange (95) + : +- ReusedExchange (97) + +- ReusedExchange (100) (1) Scan parquet default.store_sales @@ -343,456 +341,442 @@ Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#13, class_id#14, category_id#15] -(51) HashAggregate [codegen id : 10] -Input [3]: [brand_id#13, class_id#14, category_id#15] -Keys [3]: [brand_id#13, class_id#14, category_id#15] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#13, class_id#14, category_id#15] - -(52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#13, class_id#14, category_id#15] -Keys [3]: [brand_id#13, class_id#14, category_id#15] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#13, class_id#14, category_id#15] - -(53) BroadcastExchange +(51) BroadcastExchange Input [3]: [brand_id#13, class_id#14, category_id#15] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#24] -(54) BroadcastHashJoin [codegen id : 11] +(52) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Right keys [3]: [brand_id#13, class_id#14, category_id#15] Join condition: None -(55) Project [codegen id : 11] +(53) Project [codegen id : 11] Output [1]: [i_item_sk#5 AS ss_item_sk#25] Input [7]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8, brand_id#13, class_id#14, category_id#15] -(56) BroadcastExchange +(54) BroadcastExchange Input [1]: [ss_item_sk#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] -(57) BroadcastHashJoin [codegen id : 25] +(55) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#2] Right keys [1]: [ss_item_sk#25] Join condition: None -(58) Scan parquet default.item +(56) Scan parquet default.item Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(59) ColumnarToRow [codegen id : 23] +(57) ColumnarToRow [codegen id : 23] Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(60) Filter [codegen id : 23] +(58) Filter [codegen id : 23] Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Condition : isnotnull(i_item_sk#5) -(61) ReusedExchange [Reuses operator id: 56] +(59) ReusedExchange [Reuses operator id: 54] Output [1]: [ss_item_sk#25] -(62) BroadcastHashJoin [codegen id : 23] +(60) BroadcastHashJoin [codegen id : 23] Left keys [1]: [i_item_sk#5] Right keys [1]: [ss_item_sk#25] Join condition: None -(63) BroadcastExchange +(61) BroadcastExchange Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] -(64) BroadcastHashJoin [codegen id : 25] +(62) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#5] Join condition: None -(65) Project [codegen id : 25] +(63) Project [codegen id : 25] Output [6]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8] Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(66) Scan parquet default.date_dim +(64) Scan parquet default.date_dim Output [3]: [d_date_sk#10, d_year#11, d_moy#28] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(67) ColumnarToRow [codegen id : 24] +(65) ColumnarToRow [codegen id : 24] Input [3]: [d_date_sk#10, d_year#11, d_moy#28] -(68) Filter [codegen id : 24] +(66) Filter [codegen id : 24] Input [3]: [d_date_sk#10, d_year#11, d_moy#28] Condition : ((((isnotnull(d_year#11) AND isnotnull(d_moy#28)) AND (d_year#11 = 2001)) AND (d_moy#28 = 11)) AND isnotnull(d_date_sk#10)) -(69) Project [codegen id : 24] +(67) Project [codegen id : 24] Output [1]: [d_date_sk#10] Input [3]: [d_date_sk#10, d_year#11, d_moy#28] -(70) BroadcastExchange +(68) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] -(71) BroadcastHashJoin [codegen id : 25] +(69) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(72) Project [codegen id : 25] +(70) Project [codegen id : 25] Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8] Input [7]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8, d_date_sk#10] -(73) HashAggregate [codegen id : 25] +(71) HashAggregate [codegen id : 25] Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#30, isEmpty#31, count#32] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#33, isEmpty#34, count#35] -(74) Exchange +(72) Exchange Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#33, isEmpty#34, count#35] Arguments: hashpartitioning(i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#36] -(75) HashAggregate [codegen id : 26] +(73) HashAggregate [codegen id : 26] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#33, isEmpty#34, count#35] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37, count(1)#38] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sales#39, count(1)#38 AS number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41] -(76) Filter [codegen id : 26] +(74) Filter [codegen id : 26] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41 as decimal(32,6)) > cast(Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(77) Project [codegen id : 26] +(75) Project [codegen id : 26] Output [6]: [sales#39, number_sales#40, store AS channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41] -(78) Scan parquet default.catalog_sales +(76) Scan parquet default.catalog_sales Output [4]: [cs_sold_date_sk#16, cs_item_sk#17, cs_quantity#45, cs_list_price#46] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(79) ColumnarToRow [codegen id : 51] +(77) ColumnarToRow [codegen id : 51] Input [4]: [cs_sold_date_sk#16, cs_item_sk#17, cs_quantity#45, cs_list_price#46] -(80) Filter [codegen id : 51] +(78) Filter [codegen id : 51] Input [4]: [cs_sold_date_sk#16, cs_item_sk#17, cs_quantity#45, cs_list_price#46] Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_sold_date_sk#16)) -(81) ReusedExchange [Reuses operator id: 56] +(79) ReusedExchange [Reuses operator id: 54] Output [1]: [ss_item_sk#25] -(82) BroadcastHashJoin [codegen id : 51] +(80) BroadcastHashJoin [codegen id : 51] Left keys [1]: [cs_item_sk#17] Right keys [1]: [ss_item_sk#25] Join condition: None -(83) ReusedExchange [Reuses operator id: 63] +(81) ReusedExchange [Reuses operator id: 61] Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(84) BroadcastHashJoin [codegen id : 51] +(82) BroadcastHashJoin [codegen id : 51] Left keys [1]: [cs_item_sk#17] Right keys [1]: [i_item_sk#5] Join condition: None -(85) Project [codegen id : 51] +(83) Project [codegen id : 51] Output [6]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46, i_brand_id#6, i_class_id#7, i_category_id#8] Input [8]: [cs_sold_date_sk#16, cs_item_sk#17, cs_quantity#45, cs_list_price#46, i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(86) ReusedExchange [Reuses operator id: 70] +(84) ReusedExchange [Reuses operator id: 68] Output [1]: [d_date_sk#10] -(87) BroadcastHashJoin [codegen id : 51] +(85) BroadcastHashJoin [codegen id : 51] Left keys [1]: [cs_sold_date_sk#16] Right keys [1]: [d_date_sk#10] Join condition: None -(88) Project [codegen id : 51] +(86) Project [codegen id : 51] Output [5]: [cs_quantity#45, cs_list_price#46, i_brand_id#6, i_class_id#7, i_category_id#8] Input [7]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46, i_brand_id#6, i_class_id#7, i_category_id#8, d_date_sk#10] -(89) HashAggregate [codegen id : 51] +(87) HashAggregate [codegen id : 51] Input [5]: [cs_quantity#45, cs_list_price#46, i_brand_id#6, i_class_id#7, i_category_id#8] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#47, isEmpty#48, count#49] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#50, isEmpty#51, count#52] -(90) Exchange +(88) Exchange Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#50, isEmpty#51, count#52] Arguments: hashpartitioning(i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#53] -(91) HashAggregate [codegen id : 52] +(89) HashAggregate [codegen id : 52] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#50, isEmpty#51, count#52] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54, count(1)#55] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sales#56, count(1)#55 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58] -(92) Filter [codegen id : 52] +(90) Filter [codegen id : 52] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(93) Project [codegen id : 52] +(91) Project [codegen id : 52] Output [6]: [sales#56, number_sales#57, catalog AS channel#59, i_brand_id#6, i_class_id#7, i_category_id#8] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58] -(94) Scan parquet default.web_sales +(92) Scan parquet default.web_sales Output [4]: [ws_sold_date_sk#20, ws_item_sk#21, ws_quantity#60, ws_list_price#61] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(95) ColumnarToRow [codegen id : 77] +(93) ColumnarToRow [codegen id : 77] Input [4]: [ws_sold_date_sk#20, ws_item_sk#21, ws_quantity#60, ws_list_price#61] -(96) Filter [codegen id : 77] +(94) Filter [codegen id : 77] Input [4]: [ws_sold_date_sk#20, ws_item_sk#21, ws_quantity#60, ws_list_price#61] Condition : (isnotnull(ws_item_sk#21) AND isnotnull(ws_sold_date_sk#20)) -(97) ReusedExchange [Reuses operator id: 56] +(95) ReusedExchange [Reuses operator id: 54] Output [1]: [ss_item_sk#25] -(98) BroadcastHashJoin [codegen id : 77] +(96) BroadcastHashJoin [codegen id : 77] Left keys [1]: [ws_item_sk#21] Right keys [1]: [ss_item_sk#25] Join condition: None -(99) ReusedExchange [Reuses operator id: 63] +(97) ReusedExchange [Reuses operator id: 61] Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(100) BroadcastHashJoin [codegen id : 77] +(98) BroadcastHashJoin [codegen id : 77] Left keys [1]: [ws_item_sk#21] Right keys [1]: [i_item_sk#5] Join condition: None -(101) Project [codegen id : 77] +(99) Project [codegen id : 77] Output [6]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61, i_brand_id#6, i_class_id#7, i_category_id#8] Input [8]: [ws_sold_date_sk#20, ws_item_sk#21, ws_quantity#60, ws_list_price#61, i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(102) ReusedExchange [Reuses operator id: 70] +(100) ReusedExchange [Reuses operator id: 68] Output [1]: [d_date_sk#10] -(103) BroadcastHashJoin [codegen id : 77] +(101) BroadcastHashJoin [codegen id : 77] Left keys [1]: [ws_sold_date_sk#20] Right keys [1]: [d_date_sk#10] Join condition: None -(104) Project [codegen id : 77] +(102) Project [codegen id : 77] Output [5]: [ws_quantity#60, ws_list_price#61, i_brand_id#6, i_class_id#7, i_category_id#8] Input [7]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61, i_brand_id#6, i_class_id#7, i_category_id#8, d_date_sk#10] -(105) HashAggregate [codegen id : 77] +(103) HashAggregate [codegen id : 77] Input [5]: [ws_quantity#60, ws_list_price#61, i_brand_id#6, i_class_id#7, i_category_id#8] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#62, isEmpty#63, count#64] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#65, isEmpty#66, count#67] -(106) Exchange +(104) Exchange Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#65, isEmpty#66, count#67] Arguments: hashpartitioning(i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#68] -(107) HashAggregate [codegen id : 78] +(105) HashAggregate [codegen id : 78] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#65, isEmpty#66, count#67] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69, count(1)#70] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sales#71, count(1)#70 AS number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73] -(108) Filter [codegen id : 78] +(106) Filter [codegen id : 78] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(109) Project [codegen id : 78] +(107) Project [codegen id : 78] Output [6]: [sales#71, number_sales#72, web AS channel#74, i_brand_id#6, i_class_id#7, i_category_id#8] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73] -(110) Union +(108) Union -(111) Expand [codegen id : 79] +(109) Expand [codegen id : 79] Input [6]: [sales#39, number_sales#40, channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] Arguments: [List(sales#39, number_sales#40, channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, 0), List(sales#39, number_sales#40, channel#44, i_brand_id#6, i_class_id#7, null, 1), List(sales#39, number_sales#40, channel#44, i_brand_id#6, null, null, 3), List(sales#39, number_sales#40, channel#44, null, null, null, 7), List(sales#39, number_sales#40, null, null, null, null, 15)], [sales#39, number_sales#40, channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79] -(112) HashAggregate [codegen id : 79] +(110) HashAggregate [codegen id : 79] Input [7]: [sales#39, number_sales#40, channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79] Keys [5]: [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79] Functions [2]: [partial_sum(sales#39), partial_sum(number_sales#40)] Aggregate Attributes [3]: [sum#80, isEmpty#81, sum#82] Results [8]: [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79, sum#83, isEmpty#84, sum#85] -(113) Exchange +(111) Exchange Input [8]: [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79, sum#83, isEmpty#84, sum#85] Arguments: hashpartitioning(channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79, 5), true, [id=#86] -(114) HashAggregate [codegen id : 80] +(112) HashAggregate [codegen id : 80] Input [8]: [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79, sum#83, isEmpty#84, sum#85] Keys [5]: [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, spark_grouping_id#79] Functions [2]: [sum(sales#39), sum(number_sales#40)] Aggregate Attributes [2]: [sum(sales#39)#87, sum(number_sales#40)#88] Results [6]: [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, sum(sales#39)#87 AS sum(sales)#89, sum(number_sales#40)#88 AS sum(number_sales)#90] -(115) TakeOrderedAndProject +(113) TakeOrderedAndProject Input [6]: [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, sum(sales)#89, sum(number_sales)#90] Arguments: 100, [channel#75 ASC NULLS FIRST, i_brand_id#76 ASC NULLS FIRST, i_class_id#77 ASC NULLS FIRST, i_category_id#78 ASC NULLS FIRST], [channel#75, i_brand_id#76, i_class_id#77, i_category_id#78, sum(sales)#89, sum(number_sales)#90] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#42, [id=#43] -* HashAggregate (141) -+- Exchange (140) - +- * HashAggregate (139) - +- Union (138) - :- * Project (125) - : +- * BroadcastHashJoin Inner BuildRight (124) - : :- * Filter (118) - : : +- * ColumnarToRow (117) - : : +- Scan parquet default.store_sales (116) - : +- BroadcastExchange (123) - : +- * Project (122) - : +- * Filter (121) - : +- * ColumnarToRow (120) - : +- Scan parquet default.date_dim (119) - :- * Project (131) - : +- * BroadcastHashJoin Inner BuildRight (130) - : :- * Filter (128) - : : +- * ColumnarToRow (127) - : : +- Scan parquet default.catalog_sales (126) - : +- ReusedExchange (129) - +- * Project (137) - +- * BroadcastHashJoin Inner BuildRight (136) - :- * Filter (134) - : +- * ColumnarToRow (133) - : +- Scan parquet default.web_sales (132) - +- ReusedExchange (135) - - -(116) Scan parquet default.store_sales +Subquery:1 Hosting operator id = 74 Hosting Expression = Subquery scalar-subquery#42, [id=#43] +* HashAggregate (139) ++- Exchange (138) + +- * HashAggregate (137) + +- Union (136) + :- * Project (123) + : +- * BroadcastHashJoin Inner BuildRight (122) + : :- * Filter (116) + : : +- * ColumnarToRow (115) + : : +- Scan parquet default.store_sales (114) + : +- BroadcastExchange (121) + : +- * Project (120) + : +- * Filter (119) + : +- * ColumnarToRow (118) + : +- Scan parquet default.date_dim (117) + :- * Project (129) + : +- * BroadcastHashJoin Inner BuildRight (128) + : :- * Filter (126) + : : +- * ColumnarToRow (125) + : : +- Scan parquet default.catalog_sales (124) + : +- ReusedExchange (127) + +- * Project (135) + +- * BroadcastHashJoin Inner BuildRight (134) + :- * Filter (132) + : +- * ColumnarToRow (131) + : +- Scan parquet default.web_sales (130) + +- ReusedExchange (133) + + +(114) Scan parquet default.store_sales Output [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(117) ColumnarToRow [codegen id : 2] +(115) ColumnarToRow [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] -(118) Filter [codegen id : 2] +(116) Filter [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Condition : isnotnull(ss_sold_date_sk#1) -(119) Scan parquet default.date_dim +(117) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_year#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(120) ColumnarToRow [codegen id : 1] +(118) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] -(121) Filter [codegen id : 1] +(119) Filter [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] Condition : (((isnotnull(d_year#11) AND (d_year#11 >= 1999)) AND (d_year#11 <= 2001)) AND isnotnull(d_date_sk#10)) -(122) Project [codegen id : 1] +(120) Project [codegen id : 1] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_year#11] -(123) BroadcastExchange +(121) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#91] -(124) BroadcastHashJoin [codegen id : 2] +(122) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(125) Project [codegen id : 2] +(123) Project [codegen id : 2] Output [2]: [ss_quantity#3 AS quantity#92, ss_list_price#4 AS list_price#93] Input [4]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(126) Scan parquet default.catalog_sales +(124) Scan parquet default.catalog_sales Output [3]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(127) ColumnarToRow [codegen id : 4] +(125) ColumnarToRow [codegen id : 4] Input [3]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46] -(128) Filter [codegen id : 4] +(126) Filter [codegen id : 4] Input [3]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46] Condition : isnotnull(cs_sold_date_sk#16) -(129) ReusedExchange [Reuses operator id: 123] +(127) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#10] -(130) BroadcastHashJoin [codegen id : 4] +(128) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#16] Right keys [1]: [d_date_sk#10] Join condition: None -(131) Project [codegen id : 4] +(129) Project [codegen id : 4] Output [2]: [cs_quantity#45 AS quantity#94, cs_list_price#46 AS list_price#95] Input [4]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46, d_date_sk#10] -(132) Scan parquet default.web_sales +(130) Scan parquet default.web_sales Output [3]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(133) ColumnarToRow [codegen id : 6] +(131) ColumnarToRow [codegen id : 6] Input [3]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61] -(134) Filter [codegen id : 6] +(132) Filter [codegen id : 6] Input [3]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61] Condition : isnotnull(ws_sold_date_sk#20) -(135) ReusedExchange [Reuses operator id: 123] +(133) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#10] -(136) BroadcastHashJoin [codegen id : 6] +(134) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#20] Right keys [1]: [d_date_sk#10] Join condition: None -(137) Project [codegen id : 6] +(135) Project [codegen id : 6] Output [2]: [ws_quantity#60 AS quantity#96, ws_list_price#61 AS list_price#97] Input [4]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61, d_date_sk#10] -(138) Union +(136) Union -(139) HashAggregate [codegen id : 7] +(137) HashAggregate [codegen id : 7] Input [2]: [quantity#92, list_price#93] Keys: [] Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#93 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [2]: [sum#98, count#99] Results [2]: [sum#100, count#101] -(140) Exchange +(138) Exchange Input [2]: [sum#100, count#101] Arguments: SinglePartition, true, [id=#102] -(141) HashAggregate [codegen id : 8] +(139) HashAggregate [codegen id : 8] Input [2]: [sum#100, count#101] Keys: [] Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#93 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#103] Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#103 AS average_sales#104] -Subquery:2 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:2 Hosting operator id = 90 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:3 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:3 Hosting operator id = 106 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt index dfa8c1bcc157..a159ad12af02 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt @@ -80,72 +80,70 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastExchange #4 WholeStageCodegen (10) HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (9) - HashAggregate [brand_id,class_id,category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] + InputAdapter + Exchange [brand_id,class_id,category_id] #5 + WholeStageCodegen (9) + HashAggregate [brand_id,class_id,category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk,ss_sold_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] + BroadcastExchange #6 + WholeStageCodegen (1) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] + BroadcastExchange #7 + WholeStageCodegen (2) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (5) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk,cs_sold_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #9 - WholeStageCodegen (3) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] InputAdapter - ReusedExchange [d_date_sk] #7 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk,ws_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] + BroadcastExchange #9 + WholeStageCodegen (3) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [d_date_sk] #7 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (8) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk,ws_sold_date_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] InputAdapter - ReusedExchange [d_date_sk] #7 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + InputAdapter + ReusedExchange [d_date_sk] #7 InputAdapter BroadcastExchange #11 WholeStageCodegen (23) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt index 2d2b56e32bdb..26595d29697b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt @@ -1,114 +1,112 @@ == Physical Plan == -TakeOrderedAndProject (110) -+- * BroadcastHashJoin Inner BuildRight (109) - :- * Project (87) - : +- * Filter (86) - : +- * HashAggregate (85) - : +- Exchange (84) - : +- * HashAggregate (83) - : +- * Project (82) - : +- * BroadcastHashJoin Inner BuildRight (81) - : :- * Project (71) - : : +- * BroadcastHashJoin Inner BuildRight (70) - : : :- SortMergeJoin LeftSemi (64) +TakeOrderedAndProject (108) ++- * BroadcastHashJoin Inner BuildRight (107) + :- * Project (85) + : +- * Filter (84) + : +- * HashAggregate (83) + : +- Exchange (82) + : +- * HashAggregate (81) + : +- * Project (80) + : +- * BroadcastHashJoin Inner BuildRight (79) + : :- * Project (69) + : : +- * BroadcastHashJoin Inner BuildRight (68) + : : :- SortMergeJoin LeftSemi (62) : : : :- * Sort (5) : : : : +- Exchange (4) : : : : +- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.store_sales (1) - : : : +- * Sort (63) - : : : +- Exchange (62) - : : : +- * Project (61) - : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : +- * Sort (61) + : : : +- Exchange (60) + : : : +- * Project (59) + : : : +- * BroadcastHashJoin Inner BuildRight (58) : : : :- * Filter (8) : : : : +- * ColumnarToRow (7) : : : : +- Scan parquet default.item (6) - : : : +- BroadcastExchange (59) - : : : +- * HashAggregate (58) - : : : +- * HashAggregate (57) - : : : +- * HashAggregate (56) - : : : +- Exchange (55) - : : : +- * HashAggregate (54) - : : : +- SortMergeJoin LeftSemi (53) - : : : :- SortMergeJoin LeftSemi (41) - : : : : :- * Sort (26) - : : : : : +- Exchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (18) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : : :- * Filter (11) - : : : : : : : +- * ColumnarToRow (10) - : : : : : : : +- Scan parquet default.store_sales (9) - : : : : : : +- BroadcastExchange (16) - : : : : : : +- * Project (15) - : : : : : : +- * Filter (14) - : : : : : : +- * ColumnarToRow (13) - : : : : : : +- Scan parquet default.date_dim (12) - : : : : : +- BroadcastExchange (22) - : : : : : +- * Filter (21) - : : : : : +- * ColumnarToRow (20) - : : : : : +- Scan parquet default.item (19) - : : : : +- * Sort (40) - : : : : +- Exchange (39) - : : : : +- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Project (32) - : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : :- * Filter (29) - : : : : : : +- * ColumnarToRow (28) - : : : : : : +- Scan parquet default.catalog_sales (27) - : : : : : +- ReusedExchange (30) - : : : : +- BroadcastExchange (36) - : : : : +- * Filter (35) - : : : : +- * ColumnarToRow (34) - : : : : +- Scan parquet default.item (33) - : : : +- * Sort (52) - : : : +- Exchange (51) - : : : +- * Project (50) - : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * Project (47) - : : : : +- * BroadcastHashJoin Inner BuildRight (46) - : : : : :- * Filter (44) - : : : : : +- * ColumnarToRow (43) - : : : : : +- Scan parquet default.web_sales (42) - : : : : +- ReusedExchange (45) - : : : +- ReusedExchange (48) - : : +- BroadcastExchange (69) - : : +- * Project (68) - : : +- * Filter (67) - : : +- * ColumnarToRow (66) - : : +- Scan parquet default.date_dim (65) - : +- BroadcastExchange (80) - : +- SortMergeJoin LeftSemi (79) - : :- * Sort (76) - : : +- Exchange (75) - : : +- * Filter (74) - : : +- * ColumnarToRow (73) - : : +- Scan parquet default.item (72) - : +- * Sort (78) - : +- ReusedExchange (77) - +- BroadcastExchange (108) - +- * Project (107) - +- * Filter (106) - +- * HashAggregate (105) - +- Exchange (104) - +- * HashAggregate (103) - +- * Project (102) - +- * BroadcastHashJoin Inner BuildRight (101) - :- * Project (99) - : +- * BroadcastHashJoin Inner BuildRight (98) - : :- SortMergeJoin LeftSemi (92) - : : :- * Sort (89) - : : : +- ReusedExchange (88) - : : +- * Sort (91) - : : +- ReusedExchange (90) - : +- BroadcastExchange (97) - : +- * Project (96) - : +- * Filter (95) - : +- * ColumnarToRow (94) - : +- Scan parquet default.date_dim (93) - +- ReusedExchange (100) + : : : +- BroadcastExchange (57) + : : : +- * HashAggregate (56) + : : : +- Exchange (55) + : : : +- * HashAggregate (54) + : : : +- SortMergeJoin LeftSemi (53) + : : : :- SortMergeJoin LeftSemi (41) + : : : : :- * Sort (26) + : : : : : +- Exchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * Project (18) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : : :- * Filter (11) + : : : : : : : +- * ColumnarToRow (10) + : : : : : : : +- Scan parquet default.store_sales (9) + : : : : : : +- BroadcastExchange (16) + : : : : : : +- * Project (15) + : : : : : : +- * Filter (14) + : : : : : : +- * ColumnarToRow (13) + : : : : : : +- Scan parquet default.date_dim (12) + : : : : : +- BroadcastExchange (22) + : : : : : +- * Filter (21) + : : : : : +- * ColumnarToRow (20) + : : : : : +- Scan parquet default.item (19) + : : : : +- * Sort (40) + : : : : +- Exchange (39) + : : : : +- * Project (38) + : : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : : :- * Project (32) + : : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : : :- * Filter (29) + : : : : : : +- * ColumnarToRow (28) + : : : : : : +- Scan parquet default.catalog_sales (27) + : : : : : +- ReusedExchange (30) + : : : : +- BroadcastExchange (36) + : : : : +- * Filter (35) + : : : : +- * ColumnarToRow (34) + : : : : +- Scan parquet default.item (33) + : : : +- * Sort (52) + : : : +- Exchange (51) + : : : +- * Project (50) + : : : +- * BroadcastHashJoin Inner BuildRight (49) + : : : :- * Project (47) + : : : : +- * BroadcastHashJoin Inner BuildRight (46) + : : : : :- * Filter (44) + : : : : : +- * ColumnarToRow (43) + : : : : : +- Scan parquet default.web_sales (42) + : : : : +- ReusedExchange (45) + : : : +- ReusedExchange (48) + : : +- BroadcastExchange (67) + : : +- * Project (66) + : : +- * Filter (65) + : : +- * ColumnarToRow (64) + : : +- Scan parquet default.date_dim (63) + : +- BroadcastExchange (78) + : +- SortMergeJoin LeftSemi (77) + : :- * Sort (74) + : : +- Exchange (73) + : : +- * Filter (72) + : : +- * ColumnarToRow (71) + : : +- Scan parquet default.item (70) + : +- * Sort (76) + : +- ReusedExchange (75) + +- BroadcastExchange (106) + +- * Project (105) + +- * Filter (104) + +- * HashAggregate (103) + +- Exchange (102) + +- * HashAggregate (101) + +- * Project (100) + +- * BroadcastHashJoin Inner BuildRight (99) + :- * Project (97) + : +- * BroadcastHashJoin Inner BuildRight (96) + : :- SortMergeJoin LeftSemi (90) + : : :- * Sort (87) + : : : +- ReusedExchange (86) + : : +- * Sort (89) + : : +- ReusedExchange (88) + : +- BroadcastExchange (95) + : +- * Project (94) + : +- * Filter (93) + : +- * ColumnarToRow (92) + : +- Scan parquet default.date_dim (91) + +- ReusedExchange (98) (1) Scan parquet default.store_sales @@ -362,448 +360,434 @@ Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#14, class_id#15, category_id#16] -(57) HashAggregate [codegen id : 16] -Input [3]: [brand_id#14, class_id#15, category_id#16] -Keys [3]: [brand_id#14, class_id#15, category_id#16] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#14, class_id#15, category_id#16] - -(58) HashAggregate [codegen id : 16] -Input [3]: [brand_id#14, class_id#15, category_id#16] -Keys [3]: [brand_id#14, class_id#15, category_id#16] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#14, class_id#15, category_id#16] - -(59) BroadcastExchange +(57) BroadcastExchange Input [3]: [brand_id#14, class_id#15, category_id#16] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#26] -(60) BroadcastHashJoin [codegen id : 17] +(58) BroadcastHashJoin [codegen id : 17] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#14, class_id#15, category_id#16] Join condition: None -(61) Project [codegen id : 17] +(59) Project [codegen id : 17] Output [1]: [i_item_sk#6 AS ss_item_sk#27] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#14, class_id#15, category_id#16] -(62) Exchange +(60) Exchange Input [1]: [ss_item_sk#27] Arguments: hashpartitioning(ss_item_sk#27, 5), true, [id=#28] -(63) Sort [codegen id : 18] +(61) Sort [codegen id : 18] Input [1]: [ss_item_sk#27] Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 -(64) SortMergeJoin +(62) SortMergeJoin Left keys [1]: [ss_item_sk#2] Right keys [1]: [ss_item_sk#27] Join condition: None -(65) Scan parquet default.date_dim +(63) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_week_seq#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(66) ColumnarToRow [codegen id : 19] +(64) ColumnarToRow [codegen id : 19] Input [2]: [d_date_sk#10, d_week_seq#29] -(67) Filter [codegen id : 19] +(65) Filter [codegen id : 19] Input [2]: [d_date_sk#10, d_week_seq#29] Condition : ((isnotnull(d_week_seq#29) AND (d_week_seq#29 = Subquery scalar-subquery#30, [id=#31])) AND isnotnull(d_date_sk#10)) -(68) Project [codegen id : 19] +(66) Project [codegen id : 19] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_week_seq#29] -(69) BroadcastExchange +(67) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#32] -(70) BroadcastHashJoin [codegen id : 38] +(68) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(71) Project [codegen id : 38] +(69) Project [codegen id : 38] Output [3]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4] Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(72) Scan parquet default.item +(70) Scan parquet default.item Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(73) ColumnarToRow [codegen id : 20] +(71) ColumnarToRow [codegen id : 20] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(74) Filter [codegen id : 20] +(72) Filter [codegen id : 20] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) -(75) Exchange +(73) Exchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: hashpartitioning(i_item_sk#6, 5), true, [id=#33] -(76) Sort [codegen id : 21] +(74) Sort [codegen id : 21] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: [i_item_sk#6 ASC NULLS FIRST], false, 0 -(77) ReusedExchange [Reuses operator id: 62] +(75) ReusedExchange [Reuses operator id: 60] Output [1]: [ss_item_sk#27] -(78) Sort [codegen id : 37] +(76) Sort [codegen id : 37] Input [1]: [ss_item_sk#27] Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 -(79) SortMergeJoin +(77) SortMergeJoin Left keys [1]: [i_item_sk#6] Right keys [1]: [ss_item_sk#27] Join condition: None -(80) BroadcastExchange +(78) BroadcastExchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] -(81) BroadcastHashJoin [codegen id : 38] +(79) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#6] Join condition: None -(82) Project [codegen id : 38] +(80) Project [codegen id : 38] Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#7, i_class_id#8, i_category_id#9] Input [7]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(83) HashAggregate [codegen id : 38] +(81) HashAggregate [codegen id : 38] Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#7, i_class_id#8, i_category_id#9] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#35, isEmpty#36, count#37] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#38, isEmpty#39, count#40] -(84) Exchange +(82) Exchange Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#38, isEmpty#39, count#40] Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#41] -(85) HashAggregate [codegen id : 78] +(83) HashAggregate [codegen id : 78] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#38, isEmpty#39, count#40] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42, count(1)#43] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 AS sales#44, count(1)#43 AS number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46] -(86) Filter [codegen id : 78] +(84) Filter [codegen id : 78] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46 as decimal(32,6)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(32,6)))) -(87) Project [codegen id : 78] +(85) Project [codegen id : 78] Output [6]: [store AS channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46] -(88) ReusedExchange [Reuses operator id: 4] +(86) ReusedExchange [Reuses operator id: 4] Output [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] -(89) Sort [codegen id : 40] +(87) Sort [codegen id : 40] Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] Arguments: [ss_item_sk#2 ASC NULLS FIRST], false, 0 -(90) ReusedExchange [Reuses operator id: 62] +(88) ReusedExchange [Reuses operator id: 60] Output [1]: [ss_item_sk#27] -(91) Sort [codegen id : 56] +(89) Sort [codegen id : 56] Input [1]: [ss_item_sk#27] Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 -(92) SortMergeJoin +(90) SortMergeJoin Left keys [1]: [ss_item_sk#2] Right keys [1]: [ss_item_sk#27] Join condition: None -(93) Scan parquet default.date_dim +(91) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_week_seq#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(94) ColumnarToRow [codegen id : 57] +(92) ColumnarToRow [codegen id : 57] Input [2]: [d_date_sk#10, d_week_seq#29] -(95) Filter [codegen id : 57] +(93) Filter [codegen id : 57] Input [2]: [d_date_sk#10, d_week_seq#29] Condition : ((isnotnull(d_week_seq#29) AND (d_week_seq#29 = Subquery scalar-subquery#50, [id=#51])) AND isnotnull(d_date_sk#10)) -(96) Project [codegen id : 57] +(94) Project [codegen id : 57] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_week_seq#29] -(97) BroadcastExchange +(95) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#52] -(98) BroadcastHashJoin [codegen id : 76] +(96) BroadcastHashJoin [codegen id : 76] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(99) Project [codegen id : 76] +(97) Project [codegen id : 76] Output [3]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4] Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(100) ReusedExchange [Reuses operator id: 80] +(98) ReusedExchange [Reuses operator id: 78] Output [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -(101) BroadcastHashJoin [codegen id : 76] +(99) BroadcastHashJoin [codegen id : 76] Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#53] Join condition: None -(102) Project [codegen id : 76] +(100) Project [codegen id : 76] Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#54, i_class_id#55, i_category_id#56] Input [7]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -(103) HashAggregate [codegen id : 76] +(101) HashAggregate [codegen id : 76] Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#54, i_class_id#55, i_category_id#56] Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#57, isEmpty#58, count#59] Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#60, isEmpty#61, count#62] -(104) Exchange +(102) Exchange Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#60, isEmpty#61, count#62] Arguments: hashpartitioning(i_brand_id#54, i_class_id#55, i_category_id#56, 5), true, [id=#63] -(105) HashAggregate [codegen id : 77] +(103) HashAggregate [codegen id : 77] Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#60, isEmpty#61, count#62] Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64, count(1)#65] Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sales#66, count(1)#65 AS number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68] -(106) Filter [codegen id : 77] +(104) Filter [codegen id : 77] Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#47, [id=#48] as decimal(32,6)))) -(107) Project [codegen id : 77] +(105) Project [codegen id : 77] Output [6]: [store AS channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67] Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68] -(108) BroadcastExchange +(106) BroadcastExchange Input [6]: [channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#70] -(109) BroadcastHashJoin [codegen id : 78] +(107) BroadcastHashJoin [codegen id : 78] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] Join condition: None -(110) TakeOrderedAndProject +(108) TakeOrderedAndProject Input [12]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67] Arguments: 100, [i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67] ===== Subqueries ===== -Subquery:1 Hosting operator id = 86 Hosting Expression = Subquery scalar-subquery#47, [id=#48] -* HashAggregate (136) -+- Exchange (135) - +- * HashAggregate (134) - +- Union (133) - :- * Project (120) - : +- * BroadcastHashJoin Inner BuildRight (119) - : :- * Filter (113) - : : +- * ColumnarToRow (112) - : : +- Scan parquet default.store_sales (111) - : +- BroadcastExchange (118) - : +- * Project (117) - : +- * Filter (116) - : +- * ColumnarToRow (115) - : +- Scan parquet default.date_dim (114) - :- * Project (126) - : +- * BroadcastHashJoin Inner BuildRight (125) - : :- * Filter (123) - : : +- * ColumnarToRow (122) - : : +- Scan parquet default.catalog_sales (121) - : +- ReusedExchange (124) - +- * Project (132) - +- * BroadcastHashJoin Inner BuildRight (131) - :- * Filter (129) - : +- * ColumnarToRow (128) - : +- Scan parquet default.web_sales (127) - +- ReusedExchange (130) - - -(111) Scan parquet default.store_sales +Subquery:1 Hosting operator id = 84 Hosting Expression = Subquery scalar-subquery#47, [id=#48] +* HashAggregate (134) ++- Exchange (133) + +- * HashAggregate (132) + +- Union (131) + :- * Project (118) + : +- * BroadcastHashJoin Inner BuildRight (117) + : :- * Filter (111) + : : +- * ColumnarToRow (110) + : : +- Scan parquet default.store_sales (109) + : +- BroadcastExchange (116) + : +- * Project (115) + : +- * Filter (114) + : +- * ColumnarToRow (113) + : +- Scan parquet default.date_dim (112) + :- * Project (124) + : +- * BroadcastHashJoin Inner BuildRight (123) + : :- * Filter (121) + : : +- * ColumnarToRow (120) + : : +- Scan parquet default.catalog_sales (119) + : +- ReusedExchange (122) + +- * Project (130) + +- * BroadcastHashJoin Inner BuildRight (129) + :- * Filter (127) + : +- * ColumnarToRow (126) + : +- Scan parquet default.web_sales (125) + +- ReusedExchange (128) + + +(109) Scan parquet default.store_sales Output [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(112) ColumnarToRow [codegen id : 2] +(110) ColumnarToRow [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] -(113) Filter [codegen id : 2] +(111) Filter [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Condition : isnotnull(ss_sold_date_sk#1) -(114) Scan parquet default.date_dim +(112) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_year#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(115) ColumnarToRow [codegen id : 1] +(113) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] -(116) Filter [codegen id : 1] +(114) Filter [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] Condition : (((isnotnull(d_year#11) AND (d_year#11 >= 1999)) AND (d_year#11 <= 2001)) AND isnotnull(d_date_sk#10)) -(117) Project [codegen id : 1] +(115) Project [codegen id : 1] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_year#11] -(118) BroadcastExchange +(116) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#71] -(119) BroadcastHashJoin [codegen id : 2] +(117) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(120) Project [codegen id : 2] +(118) Project [codegen id : 2] Output [2]: [ss_quantity#3 AS quantity#72, ss_list_price#4 AS list_price#73] Input [4]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(121) Scan parquet default.catalog_sales +(119) Scan parquet default.catalog_sales Output [3]: [cs_sold_date_sk#18, cs_quantity#74, cs_list_price#75] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(122) ColumnarToRow [codegen id : 4] +(120) ColumnarToRow [codegen id : 4] Input [3]: [cs_sold_date_sk#18, cs_quantity#74, cs_list_price#75] -(123) Filter [codegen id : 4] +(121) Filter [codegen id : 4] Input [3]: [cs_sold_date_sk#18, cs_quantity#74, cs_list_price#75] Condition : isnotnull(cs_sold_date_sk#18) -(124) ReusedExchange [Reuses operator id: 118] +(122) ReusedExchange [Reuses operator id: 116] Output [1]: [d_date_sk#10] -(125) BroadcastHashJoin [codegen id : 4] +(123) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#18] Right keys [1]: [d_date_sk#10] Join condition: None -(126) Project [codegen id : 4] +(124) Project [codegen id : 4] Output [2]: [cs_quantity#74 AS quantity#76, cs_list_price#75 AS list_price#77] Input [4]: [cs_sold_date_sk#18, cs_quantity#74, cs_list_price#75, d_date_sk#10] -(127) Scan parquet default.web_sales +(125) Scan parquet default.web_sales Output [3]: [ws_sold_date_sk#22, ws_quantity#78, ws_list_price#79] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(128) ColumnarToRow [codegen id : 6] +(126) ColumnarToRow [codegen id : 6] Input [3]: [ws_sold_date_sk#22, ws_quantity#78, ws_list_price#79] -(129) Filter [codegen id : 6] +(127) Filter [codegen id : 6] Input [3]: [ws_sold_date_sk#22, ws_quantity#78, ws_list_price#79] Condition : isnotnull(ws_sold_date_sk#22) -(130) ReusedExchange [Reuses operator id: 118] +(128) ReusedExchange [Reuses operator id: 116] Output [1]: [d_date_sk#10] -(131) BroadcastHashJoin [codegen id : 6] +(129) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#22] Right keys [1]: [d_date_sk#10] Join condition: None -(132) Project [codegen id : 6] +(130) Project [codegen id : 6] Output [2]: [ws_quantity#78 AS quantity#80, ws_list_price#79 AS list_price#81] Input [4]: [ws_sold_date_sk#22, ws_quantity#78, ws_list_price#79, d_date_sk#10] -(133) Union +(131) Union -(134) HashAggregate [codegen id : 7] +(132) HashAggregate [codegen id : 7] Input [2]: [quantity#72, list_price#73] Keys: [] Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#73 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [2]: [sum#82, count#83] Results [2]: [sum#84, count#85] -(135) Exchange +(133) Exchange Input [2]: [sum#84, count#85] Arguments: SinglePartition, true, [id=#86] -(136) HashAggregate [codegen id : 8] +(134) HashAggregate [codegen id : 8] Input [2]: [sum#84, count#85] Keys: [] Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#73 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#73 as decimal(12,2)))), DecimalType(18,2), true))#87] Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#73 as decimal(12,2)))), DecimalType(18,2), true))#87 AS average_sales#88] -Subquery:2 Hosting operator id = 67 Hosting Expression = Subquery scalar-subquery#30, [id=#31] -* Project (140) -+- * Filter (139) - +- * ColumnarToRow (138) - +- Scan parquet default.date_dim (137) +Subquery:2 Hosting operator id = 65 Hosting Expression = Subquery scalar-subquery#30, [id=#31] +* Project (138) ++- * Filter (137) + +- * ColumnarToRow (136) + +- Scan parquet default.date_dim (135) -(137) Scan parquet default.date_dim +(135) Scan parquet default.date_dim Output [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct -(138) ColumnarToRow [codegen id : 1] +(136) ColumnarToRow [codegen id : 1] Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] -(139) Filter [codegen id : 1] +(137) Filter [codegen id : 1] Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#89)) AND isnotnull(d_dom#90)) AND (d_year#11 = 2000)) AND (d_moy#89 = 12)) AND (d_dom#90 = 11)) -(140) Project [codegen id : 1] +(138) Project [codegen id : 1] Output [1]: [d_week_seq#29] Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] -Subquery:3 Hosting operator id = 106 Hosting Expression = ReusedSubquery Subquery scalar-subquery#47, [id=#48] +Subquery:3 Hosting operator id = 104 Hosting Expression = ReusedSubquery Subquery scalar-subquery#47, [id=#48] -Subquery:4 Hosting operator id = 95 Hosting Expression = Subquery scalar-subquery#50, [id=#51] -* Project (144) -+- * Filter (143) - +- * ColumnarToRow (142) - +- Scan parquet default.date_dim (141) +Subquery:4 Hosting operator id = 93 Hosting Expression = Subquery scalar-subquery#50, [id=#51] +* Project (142) ++- * Filter (141) + +- * ColumnarToRow (140) + +- Scan parquet default.date_dim (139) -(141) Scan parquet default.date_dim +(139) Scan parquet default.date_dim Output [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct -(142) ColumnarToRow [codegen id : 1] +(140) ColumnarToRow [codegen id : 1] Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] -(143) Filter [codegen id : 1] +(141) Filter [codegen id : 1] Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#89)) AND isnotnull(d_dom#90)) AND (d_year#11 = 1999)) AND (d_moy#89 = 12)) AND (d_dom#90 = 11)) -(144) Project [codegen id : 1] +(142) Project [codegen id : 1] Output [1]: [d_week_seq#29] Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt index d6b8ba4395d2..1e0c188cd48c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt @@ -80,82 +80,80 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastExchange #4 WholeStageCodegen (16) HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (15) - HashAggregate [brand_id,class_id,category_id] - InputAdapter - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (6) - Sort [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] + InputAdapter + Exchange [brand_id,class_id,category_id] #5 + WholeStageCodegen (15) + HashAggregate [brand_id,class_id,category_id] + InputAdapter + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (6) + Sort [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #6 + WholeStageCodegen (5) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk,ss_sold_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (10) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #9 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] + BroadcastExchange #7 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (10) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #9 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk,cs_sold_date_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #7 + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (14) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #11 - WholeStageCodegen (13) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk,ws_sold_date_sk] + ReusedExchange [d_date_sk] #7 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (8) + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (14) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #11 + WholeStageCodegen (13) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk,ws_sold_date_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #7 + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + ReusedExchange [d_date_sk] #7 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 InputAdapter BroadcastExchange #12 WholeStageCodegen (19) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt index 1f31ded51f1e..1e075920689f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt @@ -1,104 +1,102 @@ == Physical Plan == -TakeOrderedAndProject (100) -+- * BroadcastHashJoin Inner BuildRight (99) - :- * Project (77) - : +- * Filter (76) - : +- * HashAggregate (75) - : +- Exchange (74) - : +- * HashAggregate (73) - : +- * Project (72) - : +- * BroadcastHashJoin Inner BuildRight (71) - : :- * Project (65) - : : +- * BroadcastHashJoin Inner BuildRight (64) - : : :- * BroadcastHashJoin LeftSemi BuildRight (57) +TakeOrderedAndProject (98) ++- * BroadcastHashJoin Inner BuildRight (97) + :- * Project (75) + : +- * Filter (74) + : +- * HashAggregate (73) + : +- Exchange (72) + : +- * HashAggregate (71) + : +- * Project (70) + : +- * BroadcastHashJoin Inner BuildRight (69) + : :- * Project (63) + : : +- * BroadcastHashJoin Inner BuildRight (62) + : : :- * BroadcastHashJoin LeftSemi BuildRight (55) : : : :- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (56) - : : : +- * Project (55) - : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : +- BroadcastExchange (54) + : : : +- * Project (53) + : : : +- * BroadcastHashJoin Inner BuildRight (52) : : : :- * Filter (6) : : : : +- * ColumnarToRow (5) : : : : +- Scan parquet default.item (4) - : : : +- BroadcastExchange (53) - : : : +- * HashAggregate (52) - : : : +- * HashAggregate (51) - : : : +- * HashAggregate (50) - : : : +- Exchange (49) - : : : +- * HashAggregate (48) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (36) - : : : : :- * Project (22) - : : : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : : : :- * Project (15) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : :- * Filter (9) - : : : : : : : +- * ColumnarToRow (8) - : : : : : : : +- Scan parquet default.store_sales (7) - : : : : : : +- BroadcastExchange (13) - : : : : : : +- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet default.item (10) - : : : : : +- BroadcastExchange (20) - : : : : : +- * Project (19) - : : : : : +- * Filter (18) - : : : : : +- * ColumnarToRow (17) - : : : : : +- Scan parquet default.date_dim (16) - : : : : +- BroadcastExchange (35) - : : : : +- * Project (34) - : : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : : :- * Project (31) - : : : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : : : :- * Filter (25) - : : : : : : +- * ColumnarToRow (24) - : : : : : : +- Scan parquet default.catalog_sales (23) - : : : : : +- BroadcastExchange (29) - : : : : : +- * Filter (28) - : : : : : +- * ColumnarToRow (27) - : : : : : +- Scan parquet default.item (26) - : : : : +- ReusedExchange (32) - : : : +- BroadcastExchange (46) - : : : +- * Project (45) - : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : :- * Project (42) - : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : :- * Filter (39) - : : : : : +- * ColumnarToRow (38) - : : : : : +- Scan parquet default.web_sales (37) - : : : : +- ReusedExchange (40) - : : : +- ReusedExchange (43) - : : +- BroadcastExchange (63) - : : +- * BroadcastHashJoin LeftSemi BuildRight (62) - : : :- * Filter (60) - : : : +- * ColumnarToRow (59) - : : : +- Scan parquet default.item (58) - : : +- ReusedExchange (61) - : +- BroadcastExchange (70) - : +- * Project (69) - : +- * Filter (68) - : +- * ColumnarToRow (67) - : +- Scan parquet default.date_dim (66) - +- BroadcastExchange (98) - +- * Project (97) - +- * Filter (96) - +- * HashAggregate (95) - +- Exchange (94) - +- * HashAggregate (93) - +- * Project (92) - +- * BroadcastHashJoin Inner BuildRight (91) - :- * Project (85) - : +- * BroadcastHashJoin Inner BuildRight (84) - : :- * BroadcastHashJoin LeftSemi BuildRight (82) - : : :- * Filter (80) - : : : +- * ColumnarToRow (79) - : : : +- Scan parquet default.store_sales (78) - : : +- ReusedExchange (81) - : +- ReusedExchange (83) - +- BroadcastExchange (90) - +- * Project (89) - +- * Filter (88) - +- * ColumnarToRow (87) - +- Scan parquet default.date_dim (86) + : : : +- BroadcastExchange (51) + : : : +- * HashAggregate (50) + : : : +- Exchange (49) + : : : +- * HashAggregate (48) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (36) + : : : : :- * Project (22) + : : : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : : : :- * Project (15) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : : :- * Filter (9) + : : : : : : : +- * ColumnarToRow (8) + : : : : : : : +- Scan parquet default.store_sales (7) + : : : : : : +- BroadcastExchange (13) + : : : : : : +- * Filter (12) + : : : : : : +- * ColumnarToRow (11) + : : : : : : +- Scan parquet default.item (10) + : : : : : +- BroadcastExchange (20) + : : : : : +- * Project (19) + : : : : : +- * Filter (18) + : : : : : +- * ColumnarToRow (17) + : : : : : +- Scan parquet default.date_dim (16) + : : : : +- BroadcastExchange (35) + : : : : +- * Project (34) + : : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : : :- * Project (31) + : : : : : +- * BroadcastHashJoin Inner BuildRight (30) + : : : : : :- * Filter (25) + : : : : : : +- * ColumnarToRow (24) + : : : : : : +- Scan parquet default.catalog_sales (23) + : : : : : +- BroadcastExchange (29) + : : : : : +- * Filter (28) + : : : : : +- * ColumnarToRow (27) + : : : : : +- Scan parquet default.item (26) + : : : : +- ReusedExchange (32) + : : : +- BroadcastExchange (46) + : : : +- * Project (45) + : : : +- * BroadcastHashJoin Inner BuildRight (44) + : : : :- * Project (42) + : : : : +- * BroadcastHashJoin Inner BuildRight (41) + : : : : :- * Filter (39) + : : : : : +- * ColumnarToRow (38) + : : : : : +- Scan parquet default.web_sales (37) + : : : : +- ReusedExchange (40) + : : : +- ReusedExchange (43) + : : +- BroadcastExchange (61) + : : +- * BroadcastHashJoin LeftSemi BuildRight (60) + : : :- * Filter (58) + : : : +- * ColumnarToRow (57) + : : : +- Scan parquet default.item (56) + : : +- ReusedExchange (59) + : +- BroadcastExchange (68) + : +- * Project (67) + : +- * Filter (66) + : +- * ColumnarToRow (65) + : +- Scan parquet default.date_dim (64) + +- BroadcastExchange (96) + +- * Project (95) + +- * Filter (94) + +- * HashAggregate (93) + +- Exchange (92) + +- * HashAggregate (91) + +- * Project (90) + +- * BroadcastHashJoin Inner BuildRight (89) + :- * Project (83) + : +- * BroadcastHashJoin Inner BuildRight (82) + : :- * BroadcastHashJoin LeftSemi BuildRight (80) + : : :- * Filter (78) + : : : +- * ColumnarToRow (77) + : : : +- Scan parquet default.store_sales (76) + : : +- ReusedExchange (79) + : +- ReusedExchange (81) + +- BroadcastExchange (88) + +- * Project (87) + +- * Filter (86) + +- * ColumnarToRow (85) + +- Scan parquet default.date_dim (84) (1) Scan parquet default.store_sales @@ -328,435 +326,421 @@ Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#13, class_id#14, category_id#15] -(51) HashAggregate [codegen id : 10] -Input [3]: [brand_id#13, class_id#14, category_id#15] -Keys [3]: [brand_id#13, class_id#14, category_id#15] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#13, class_id#14, category_id#15] - -(52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#13, class_id#14, category_id#15] -Keys [3]: [brand_id#13, class_id#14, category_id#15] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#13, class_id#14, category_id#15] - -(53) BroadcastExchange +(51) BroadcastExchange Input [3]: [brand_id#13, class_id#14, category_id#15] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#24] -(54) BroadcastHashJoin [codegen id : 11] +(52) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Right keys [3]: [brand_id#13, class_id#14, category_id#15] Join condition: None -(55) Project [codegen id : 11] +(53) Project [codegen id : 11] Output [1]: [i_item_sk#5 AS ss_item_sk#25] Input [7]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8, brand_id#13, class_id#14, category_id#15] -(56) BroadcastExchange +(54) BroadcastExchange Input [1]: [ss_item_sk#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] -(57) BroadcastHashJoin [codegen id : 25] +(55) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#2] Right keys [1]: [ss_item_sk#25] Join condition: None -(58) Scan parquet default.item +(56) Scan parquet default.item Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(59) ColumnarToRow [codegen id : 23] +(57) ColumnarToRow [codegen id : 23] Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(60) Filter [codegen id : 23] +(58) Filter [codegen id : 23] Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Condition : (((isnotnull(i_item_sk#5) AND isnotnull(i_brand_id#6)) AND isnotnull(i_class_id#7)) AND isnotnull(i_category_id#8)) -(61) ReusedExchange [Reuses operator id: 56] +(59) ReusedExchange [Reuses operator id: 54] Output [1]: [ss_item_sk#25] -(62) BroadcastHashJoin [codegen id : 23] +(60) BroadcastHashJoin [codegen id : 23] Left keys [1]: [i_item_sk#5] Right keys [1]: [ss_item_sk#25] Join condition: None -(63) BroadcastExchange +(61) BroadcastExchange Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] -(64) BroadcastHashJoin [codegen id : 25] +(62) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#5] Join condition: None -(65) Project [codegen id : 25] +(63) Project [codegen id : 25] Output [6]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8] Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(66) Scan parquet default.date_dim +(64) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_week_seq#28] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(67) ColumnarToRow [codegen id : 24] +(65) ColumnarToRow [codegen id : 24] Input [2]: [d_date_sk#10, d_week_seq#28] -(68) Filter [codegen id : 24] +(66) Filter [codegen id : 24] Input [2]: [d_date_sk#10, d_week_seq#28] Condition : ((isnotnull(d_week_seq#28) AND (d_week_seq#28 = Subquery scalar-subquery#29, [id=#30])) AND isnotnull(d_date_sk#10)) -(69) Project [codegen id : 24] +(67) Project [codegen id : 24] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_week_seq#28] -(70) BroadcastExchange +(68) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] -(71) BroadcastHashJoin [codegen id : 25] +(69) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(72) Project [codegen id : 25] +(70) Project [codegen id : 25] Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8] Input [7]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8, d_date_sk#10] -(73) HashAggregate [codegen id : 25] +(71) HashAggregate [codegen id : 25] Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#32, isEmpty#33, count#34] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#35, isEmpty#36, count#37] -(74) Exchange +(72) Exchange Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#35, isEmpty#36, count#37] Arguments: hashpartitioning(i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#38] -(75) HashAggregate [codegen id : 52] +(73) HashAggregate [codegen id : 52] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#35, isEmpty#36, count#37] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#39, count(1)#40] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sales#41, count(1)#40 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43] -(76) Filter [codegen id : 52] +(74) Filter [codegen id : 52] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43 as decimal(32,6)) > cast(Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) -(77) Project [codegen id : 52] +(75) Project [codegen id : 52] Output [6]: [store AS channel#46, i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43] -(78) Scan parquet default.store_sales +(76) Scan parquet default.store_sales Output [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(79) ColumnarToRow [codegen id : 50] +(77) ColumnarToRow [codegen id : 50] Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] -(80) Filter [codegen id : 50] +(78) Filter [codegen id : 50] Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] Condition : (isnotnull(ss_item_sk#2) AND isnotnull(ss_sold_date_sk#1)) -(81) ReusedExchange [Reuses operator id: 56] +(79) ReusedExchange [Reuses operator id: 54] Output [1]: [ss_item_sk#25] -(82) BroadcastHashJoin [codegen id : 50] +(80) BroadcastHashJoin [codegen id : 50] Left keys [1]: [ss_item_sk#2] Right keys [1]: [ss_item_sk#25] Join condition: None -(83) ReusedExchange [Reuses operator id: 63] +(81) ReusedExchange [Reuses operator id: 61] Output [4]: [i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] -(84) BroadcastHashJoin [codegen id : 50] +(82) BroadcastHashJoin [codegen id : 50] Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#47] Join condition: None -(85) Project [codegen id : 50] +(83) Project [codegen id : 50] Output [6]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, i_brand_id#48, i_class_id#49, i_category_id#50] Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] -(86) Scan parquet default.date_dim +(84) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_week_seq#28] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(87) ColumnarToRow [codegen id : 49] +(85) ColumnarToRow [codegen id : 49] Input [2]: [d_date_sk#10, d_week_seq#28] -(88) Filter [codegen id : 49] +(86) Filter [codegen id : 49] Input [2]: [d_date_sk#10, d_week_seq#28] Condition : ((isnotnull(d_week_seq#28) AND (d_week_seq#28 = Subquery scalar-subquery#51, [id=#52])) AND isnotnull(d_date_sk#10)) -(89) Project [codegen id : 49] +(87) Project [codegen id : 49] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_week_seq#28] -(90) BroadcastExchange +(88) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] -(91) BroadcastHashJoin [codegen id : 50] +(89) BroadcastHashJoin [codegen id : 50] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(92) Project [codegen id : 50] +(90) Project [codegen id : 50] Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#48, i_class_id#49, i_category_id#50] Input [7]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, i_brand_id#48, i_class_id#49, i_category_id#50, d_date_sk#10] -(93) HashAggregate [codegen id : 50] +(91) HashAggregate [codegen id : 50] Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#48, i_class_id#49, i_category_id#50] Keys [3]: [i_brand_id#48, i_class_id#49, i_category_id#50] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#54, isEmpty#55, count#56] Results [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum#57, isEmpty#58, count#59] -(94) Exchange +(92) Exchange Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum#57, isEmpty#58, count#59] Arguments: hashpartitioning(i_brand_id#48, i_class_id#49, i_category_id#50, 5), true, [id=#60] -(95) HashAggregate [codegen id : 51] +(93) HashAggregate [codegen id : 51] Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum#57, isEmpty#58, count#59] Keys [3]: [i_brand_id#48, i_class_id#49, i_category_id#50] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#61, count(1)#62] Results [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sales#63, count(1)#62 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65] -(96) Filter [codegen id : 51] +(94) Filter [codegen id : 51] Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) -(97) Project [codegen id : 51] +(95) Project [codegen id : 51] Output [6]: [store AS channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64] Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65] -(98) BroadcastExchange +(96) BroadcastExchange Input [6]: [channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#67] -(99) BroadcastHashJoin [codegen id : 52] +(97) BroadcastHashJoin [codegen id : 52] Left keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Right keys [3]: [i_brand_id#48, i_class_id#49, i_category_id#50] Join condition: None -(100) TakeOrderedAndProject +(98) TakeOrderedAndProject Input [12]: [channel#46, i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64] Arguments: 100, [i_brand_id#6 ASC NULLS FIRST, i_class_id#7 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST], [channel#46, i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#44, [id=#45] -* HashAggregate (126) -+- Exchange (125) - +- * HashAggregate (124) - +- Union (123) - :- * Project (110) - : +- * BroadcastHashJoin Inner BuildRight (109) - : :- * Filter (103) - : : +- * ColumnarToRow (102) - : : +- Scan parquet default.store_sales (101) - : +- BroadcastExchange (108) - : +- * Project (107) - : +- * Filter (106) - : +- * ColumnarToRow (105) - : +- Scan parquet default.date_dim (104) - :- * Project (116) - : +- * BroadcastHashJoin Inner BuildRight (115) - : :- * Filter (113) - : : +- * ColumnarToRow (112) - : : +- Scan parquet default.catalog_sales (111) - : +- ReusedExchange (114) - +- * Project (122) - +- * BroadcastHashJoin Inner BuildRight (121) - :- * Filter (119) - : +- * ColumnarToRow (118) - : +- Scan parquet default.web_sales (117) - +- ReusedExchange (120) - - -(101) Scan parquet default.store_sales +Subquery:1 Hosting operator id = 74 Hosting Expression = Subquery scalar-subquery#44, [id=#45] +* HashAggregate (124) ++- Exchange (123) + +- * HashAggregate (122) + +- Union (121) + :- * Project (108) + : +- * BroadcastHashJoin Inner BuildRight (107) + : :- * Filter (101) + : : +- * ColumnarToRow (100) + : : +- Scan parquet default.store_sales (99) + : +- BroadcastExchange (106) + : +- * Project (105) + : +- * Filter (104) + : +- * ColumnarToRow (103) + : +- Scan parquet default.date_dim (102) + :- * Project (114) + : +- * BroadcastHashJoin Inner BuildRight (113) + : :- * Filter (111) + : : +- * ColumnarToRow (110) + : : +- Scan parquet default.catalog_sales (109) + : +- ReusedExchange (112) + +- * Project (120) + +- * BroadcastHashJoin Inner BuildRight (119) + :- * Filter (117) + : +- * ColumnarToRow (116) + : +- Scan parquet default.web_sales (115) + +- ReusedExchange (118) + + +(99) Scan parquet default.store_sales Output [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(102) ColumnarToRow [codegen id : 2] +(100) ColumnarToRow [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] -(103) Filter [codegen id : 2] +(101) Filter [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Condition : isnotnull(ss_sold_date_sk#1) -(104) Scan parquet default.date_dim +(102) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_year#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(105) ColumnarToRow [codegen id : 1] +(103) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] -(106) Filter [codegen id : 1] +(104) Filter [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] Condition : (((isnotnull(d_year#11) AND (d_year#11 >= 1999)) AND (d_year#11 <= 2001)) AND isnotnull(d_date_sk#10)) -(107) Project [codegen id : 1] +(105) Project [codegen id : 1] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_year#11] -(108) BroadcastExchange +(106) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#68] -(109) BroadcastHashJoin [codegen id : 2] +(107) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(110) Project [codegen id : 2] +(108) Project [codegen id : 2] Output [2]: [ss_quantity#3 AS quantity#69, ss_list_price#4 AS list_price#70] Input [4]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(111) Scan parquet default.catalog_sales +(109) Scan parquet default.catalog_sales Output [3]: [cs_sold_date_sk#16, cs_quantity#71, cs_list_price#72] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(112) ColumnarToRow [codegen id : 4] +(110) ColumnarToRow [codegen id : 4] Input [3]: [cs_sold_date_sk#16, cs_quantity#71, cs_list_price#72] -(113) Filter [codegen id : 4] +(111) Filter [codegen id : 4] Input [3]: [cs_sold_date_sk#16, cs_quantity#71, cs_list_price#72] Condition : isnotnull(cs_sold_date_sk#16) -(114) ReusedExchange [Reuses operator id: 108] +(112) ReusedExchange [Reuses operator id: 106] Output [1]: [d_date_sk#10] -(115) BroadcastHashJoin [codegen id : 4] +(113) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#16] Right keys [1]: [d_date_sk#10] Join condition: None -(116) Project [codegen id : 4] +(114) Project [codegen id : 4] Output [2]: [cs_quantity#71 AS quantity#73, cs_list_price#72 AS list_price#74] Input [4]: [cs_sold_date_sk#16, cs_quantity#71, cs_list_price#72, d_date_sk#10] -(117) Scan parquet default.web_sales +(115) Scan parquet default.web_sales Output [3]: [ws_sold_date_sk#20, ws_quantity#75, ws_list_price#76] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(118) ColumnarToRow [codegen id : 6] +(116) ColumnarToRow [codegen id : 6] Input [3]: [ws_sold_date_sk#20, ws_quantity#75, ws_list_price#76] -(119) Filter [codegen id : 6] +(117) Filter [codegen id : 6] Input [3]: [ws_sold_date_sk#20, ws_quantity#75, ws_list_price#76] Condition : isnotnull(ws_sold_date_sk#20) -(120) ReusedExchange [Reuses operator id: 108] +(118) ReusedExchange [Reuses operator id: 106] Output [1]: [d_date_sk#10] -(121) BroadcastHashJoin [codegen id : 6] +(119) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#20] Right keys [1]: [d_date_sk#10] Join condition: None -(122) Project [codegen id : 6] +(120) Project [codegen id : 6] Output [2]: [ws_quantity#75 AS quantity#77, ws_list_price#76 AS list_price#78] Input [4]: [ws_sold_date_sk#20, ws_quantity#75, ws_list_price#76, d_date_sk#10] -(123) Union +(121) Union -(124) HashAggregate [codegen id : 7] +(122) HashAggregate [codegen id : 7] Input [2]: [quantity#69, list_price#70] Keys: [] Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#70 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [2]: [sum#79, count#80] Results [2]: [sum#81, count#82] -(125) Exchange +(123) Exchange Input [2]: [sum#81, count#82] Arguments: SinglePartition, true, [id=#83] -(126) HashAggregate [codegen id : 8] +(124) HashAggregate [codegen id : 8] Input [2]: [sum#81, count#82] Keys: [] Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#70 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#84] Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#84 AS average_sales#85] -Subquery:2 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#29, [id=#30] -* Project (130) -+- * Filter (129) - +- * ColumnarToRow (128) - +- Scan parquet default.date_dim (127) +Subquery:2 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#29, [id=#30] +* Project (128) ++- * Filter (127) + +- * ColumnarToRow (126) + +- Scan parquet default.date_dim (125) -(127) Scan parquet default.date_dim +(125) Scan parquet default.date_dim Output [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct -(128) ColumnarToRow [codegen id : 1] +(126) ColumnarToRow [codegen id : 1] Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] -(129) Filter [codegen id : 1] +(127) Filter [codegen id : 1] Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#86)) AND isnotnull(d_dom#87)) AND (d_year#11 = 2000)) AND (d_moy#86 = 12)) AND (d_dom#87 = 11)) -(130) Project [codegen id : 1] +(128) Project [codegen id : 1] Output [1]: [d_week_seq#28] Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] -Subquery:3 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:3 Hosting operator id = 94 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -Subquery:4 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#51, [id=#52] -* Project (134) -+- * Filter (133) - +- * ColumnarToRow (132) - +- Scan parquet default.date_dim (131) +Subquery:4 Hosting operator id = 86 Hosting Expression = Subquery scalar-subquery#51, [id=#52] +* Project (132) ++- * Filter (131) + +- * ColumnarToRow (130) + +- Scan parquet default.date_dim (129) -(131) Scan parquet default.date_dim +(129) Scan parquet default.date_dim Output [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct -(132) ColumnarToRow [codegen id : 1] +(130) ColumnarToRow [codegen id : 1] Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] -(133) Filter [codegen id : 1] +(131) Filter [codegen id : 1] Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#86)) AND isnotnull(d_dom#87)) AND (d_year#11 = 1999)) AND (d_moy#86 = 12)) AND (d_dom#87 = 11)) -(134) Project [codegen id : 1] +(132) Project [codegen id : 1] Output [1]: [d_week_seq#28] Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt index 7bbf83e3de70..e58043e1de3a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt @@ -72,72 +72,70 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastExchange #3 WholeStageCodegen (10) HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #4 - WholeStageCodegen (9) - HashAggregate [brand_id,class_id,category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] + InputAdapter + Exchange [brand_id,class_id,category_id] #4 + WholeStageCodegen (9) + HashAggregate [brand_id,class_id,category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk,ss_sold_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] + BroadcastExchange #5 + WholeStageCodegen (1) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] + BroadcastExchange #6 + WholeStageCodegen (2) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (5) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk,cs_sold_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (8) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk,ws_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] + BroadcastExchange #8 + WholeStageCodegen (3) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (8) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk,ws_sold_date_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8 + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] InputAdapter - ReusedExchange [d_date_sk] #6 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8 + InputAdapter + ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #10 WholeStageCodegen (23) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt index 92b9c26825e5..62fbf0bdc57b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt @@ -1,72 +1,68 @@ == Physical Plan == -CollectLimit (68) -+- * HashAggregate (67) - +- Exchange (66) - +- * HashAggregate (65) - +- * HashAggregate (64) - +- * HashAggregate (63) - +- * HashAggregate (62) - +- * HashAggregate (61) - +- * HashAggregate (60) - +- Exchange (59) - +- * HashAggregate (58) - +- SortMergeJoin LeftSemi (57) - :- SortMergeJoin LeftSemi (39) - : :- * Sort (21) - : : +- Exchange (20) - : : +- * Project (19) - : : +- * SortMergeJoin Inner (18) - : : :- * Sort (12) - : : : +- Exchange (11) - : : : +- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.date_dim (4) - : : +- * Sort (17) - : : +- Exchange (16) - : : +- * Filter (15) - : : +- * ColumnarToRow (14) - : : +- Scan parquet default.customer (13) - : +- * Sort (38) - : +- Exchange (37) - : +- * HashAggregate (36) - : +- Exchange (35) - : +- * HashAggregate (34) - : +- * Project (33) - : +- * SortMergeJoin Inner (32) - : :- * Sort (29) - : : +- Exchange (28) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Filter (24) - : : : +- * ColumnarToRow (23) - : : : +- Scan parquet default.catalog_sales (22) - : : +- ReusedExchange (25) - : +- * Sort (31) - : +- ReusedExchange (30) - +- * Sort (56) - +- Exchange (55) - +- * HashAggregate (54) - +- Exchange (53) - +- * HashAggregate (52) - +- * Project (51) - +- * SortMergeJoin Inner (50) - :- * Sort (47) - : +- Exchange (46) - : +- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Filter (42) - : : +- * ColumnarToRow (41) - : : +- Scan parquet default.web_sales (40) - : +- ReusedExchange (43) - +- * Sort (49) - +- ReusedExchange (48) +CollectLimit (64) ++- * HashAggregate (63) + +- Exchange (62) + +- * HashAggregate (61) + +- * HashAggregate (60) + +- Exchange (59) + +- * HashAggregate (58) + +- SortMergeJoin LeftSemi (57) + :- SortMergeJoin LeftSemi (39) + : :- * Sort (21) + : : +- Exchange (20) + : : +- * Project (19) + : : +- * SortMergeJoin Inner (18) + : : :- * Sort (12) + : : : +- Exchange (11) + : : : +- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Project (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.date_dim (4) + : : +- * Sort (17) + : : +- Exchange (16) + : : +- * Filter (15) + : : +- * ColumnarToRow (14) + : : +- Scan parquet default.customer (13) + : +- * Sort (38) + : +- Exchange (37) + : +- * HashAggregate (36) + : +- Exchange (35) + : +- * HashAggregate (34) + : +- * Project (33) + : +- * SortMergeJoin Inner (32) + : :- * Sort (29) + : : +- Exchange (28) + : : +- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * Filter (24) + : : : +- * ColumnarToRow (23) + : : : +- Scan parquet default.catalog_sales (22) + : : +- ReusedExchange (25) + : +- * Sort (31) + : +- ReusedExchange (30) + +- * Sort (56) + +- Exchange (55) + +- * HashAggregate (54) + +- Exchange (53) + +- * HashAggregate (52) + +- * Project (51) + +- * SortMergeJoin Inner (50) + :- * Sort (47) + : +- Exchange (46) + : +- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) + : :- * Filter (42) + : : +- * ColumnarToRow (41) + : : +- Scan parquet default.web_sales (40) + : +- ReusedExchange (43) + +- * Sort (49) + +- ReusedExchange (48) (1) Scan parquet default.store_sales @@ -339,55 +335,27 @@ Input [3]: [c_last_name#10, c_first_name#9, d_date#4] Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#10, c_first_name#9, d_date#4] - -(61) HashAggregate [codegen id : 25] -Input [3]: [c_last_name#10, c_first_name#9, d_date#4] -Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#10, c_first_name#9, d_date#4] - -(62) HashAggregate [codegen id : 25] -Input [3]: [c_last_name#10, c_first_name#9, d_date#4] -Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#10, c_first_name#9, d_date#4] - -(63) HashAggregate [codegen id : 25] -Input [3]: [c_last_name#10, c_first_name#9, d_date#4] -Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#10, c_first_name#9, d_date#4] - -(64) HashAggregate [codegen id : 25] -Input [3]: [c_last_name#10, c_first_name#9, d_date#4] -Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] -Functions: [] -Aggregate Attributes: [] Results: [] -(65) HashAggregate [codegen id : 25] +(61) HashAggregate [codegen id : 25] Input: [] Keys: [] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#34] Results [1]: [count#35] -(66) Exchange +(62) Exchange Input [1]: [count#35] Arguments: SinglePartition, true, [id=#36] -(67) HashAggregate [codegen id : 26] +(63) HashAggregate [codegen id : 26] Input [1]: [count#35] Keys: [] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#37] Results [1]: [count(1)#37 AS count(1)#38] -(68) CollectLimit +(64) CollectLimit Input [1]: [count(1)#38] Arguments: 100 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/simplified.txt index 5bcd7dbb9302..d930496e41e2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/simplified.txt @@ -6,113 +6,109 @@ CollectLimit WholeStageCodegen (25) HashAggregate [count,count] HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #2 + WholeStageCodegen (24) HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (24) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - WholeStageCodegen (7) - Sort [c_last_name,c_first_name,d_date] + SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + WholeStageCodegen (7) + Sort [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #3 + WholeStageCodegen (6) + Project [d_date,c_first_name,c_last_name] + SortMergeJoin [ss_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (3) + Sort [ss_customer_sk] + InputAdapter + Exchange [ss_customer_sk] #4 + WholeStageCodegen (2) + Project [ss_customer_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + Project [d_date_sk,d_date] + Filter [d_month_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + WholeStageCodegen (5) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #6 + WholeStageCodegen (4) + Filter [c_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] + WholeStageCodegen (15) + Sort [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #7 + WholeStageCodegen (14) + HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_last_name,c_first_name,d_date] #3 - WholeStageCodegen (6) - Project [d_date,c_first_name,c_last_name] - SortMergeJoin [ss_customer_sk,c_customer_sk] + Exchange [c_last_name,c_first_name,d_date] #8 + WholeStageCodegen (13) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (10) + Sort [cs_bill_customer_sk] + InputAdapter + Exchange [cs_bill_customer_sk] #9 + WholeStageCodegen (9) + Project [cs_bill_customer_sk,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_sold_date_sk,cs_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] + InputAdapter + ReusedExchange [d_date_sk,d_date] #5 + InputAdapter + WholeStageCodegen (12) + Sort [c_customer_sk] + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #6 + WholeStageCodegen (23) + Sort [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #10 + WholeStageCodegen (22) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #11 + WholeStageCodegen (21) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (3) - Sort [ss_customer_sk] + WholeStageCodegen (18) + Sort [ws_bill_customer_sk] InputAdapter - Exchange [ss_customer_sk] #4 - WholeStageCodegen (2) - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_customer_sk] + Exchange [ws_bill_customer_sk] #12 + WholeStageCodegen (17) + Project [ws_bill_customer_sk,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Project [d_date_sk,d_date] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] + ReusedExchange [d_date_sk,d_date] #5 InputAdapter - WholeStageCodegen (5) + WholeStageCodegen (20) Sort [c_customer_sk] InputAdapter - Exchange [c_customer_sk] #6 - WholeStageCodegen (4) - Filter [c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] - WholeStageCodegen (15) - Sort [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #7 - WholeStageCodegen (14) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (13) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - SortMergeJoin [cs_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (10) - Sort [cs_bill_customer_sk] - InputAdapter - Exchange [cs_bill_customer_sk] #9 - WholeStageCodegen (9) - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] - InputAdapter - ReusedExchange [d_date_sk,d_date] #5 - InputAdapter - WholeStageCodegen (12) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #6 - WholeStageCodegen (23) - Sort [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #10 - WholeStageCodegen (22) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #11 - WholeStageCodegen (21) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - SortMergeJoin [ws_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (18) - Sort [ws_bill_customer_sk] - InputAdapter - Exchange [ws_bill_customer_sk] #12 - WholeStageCodegen (17) - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_sold_date_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] - InputAdapter - ReusedExchange [d_date_sk,d_date] #5 - InputAdapter - WholeStageCodegen (20) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #6 + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt index 09ab60c7cf65..84aa770ce667 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt @@ -1,59 +1,55 @@ == Physical Plan == -CollectLimit (55) -+- * HashAggregate (54) - +- Exchange (53) - +- * HashAggregate (52) - +- * HashAggregate (51) - +- * HashAggregate (50) - +- * HashAggregate (49) - +- * HashAggregate (48) - +- * HashAggregate (47) - +- Exchange (46) - +- * HashAggregate (45) - +- * BroadcastHashJoin LeftSemi BuildRight (44) - :- * BroadcastHashJoin LeftSemi BuildRight (30) - : :- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.date_dim (4) - : : +- BroadcastExchange (14) - : : +- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.customer (11) - : +- BroadcastExchange (29) - : +- * HashAggregate (28) - : +- Exchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * Filter (19) - : : : +- * ColumnarToRow (18) - : : : +- Scan parquet default.catalog_sales (17) - : : +- ReusedExchange (20) - : +- ReusedExchange (23) - +- BroadcastExchange (43) - +- * HashAggregate (42) - +- Exchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * Filter (33) - : : +- * ColumnarToRow (32) - : : +- Scan parquet default.web_sales (31) - : +- ReusedExchange (34) - +- ReusedExchange (37) +CollectLimit (51) ++- * HashAggregate (50) + +- Exchange (49) + +- * HashAggregate (48) + +- * HashAggregate (47) + +- Exchange (46) + +- * HashAggregate (45) + +- * BroadcastHashJoin LeftSemi BuildRight (44) + :- * BroadcastHashJoin LeftSemi BuildRight (30) + : :- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Project (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.date_dim (4) + : : +- BroadcastExchange (14) + : : +- * Filter (13) + : : +- * ColumnarToRow (12) + : : +- Scan parquet default.customer (11) + : +- BroadcastExchange (29) + : +- * HashAggregate (28) + : +- Exchange (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) + : : :- * Filter (19) + : : : +- * ColumnarToRow (18) + : : : +- Scan parquet default.catalog_sales (17) + : : +- ReusedExchange (20) + : +- ReusedExchange (23) + +- BroadcastExchange (43) + +- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) + :- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * Filter (33) + : : +- * ColumnarToRow (32) + : : +- Scan parquet default.web_sales (31) + : +- ReusedExchange (34) + +- ReusedExchange (37) (1) Scan parquet default.store_sales @@ -274,55 +270,27 @@ Input [3]: [c_last_name#9, c_first_name#8, d_date#4] Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#9, c_first_name#8, d_date#4] - -(48) HashAggregate [codegen id : 12] -Input [3]: [c_last_name#9, c_first_name#8, d_date#4] -Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#9, c_first_name#8, d_date#4] - -(49) HashAggregate [codegen id : 12] -Input [3]: [c_last_name#9, c_first_name#8, d_date#4] -Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#9, c_first_name#8, d_date#4] - -(50) HashAggregate [codegen id : 12] -Input [3]: [c_last_name#9, c_first_name#8, d_date#4] -Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#9, c_first_name#8, d_date#4] - -(51) HashAggregate [codegen id : 12] -Input [3]: [c_last_name#9, c_first_name#8, d_date#4] -Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] -Functions: [] -Aggregate Attributes: [] Results: [] -(52) HashAggregate [codegen id : 12] +(48) HashAggregate [codegen id : 12] Input: [] Keys: [] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#30] Results [1]: [count#31] -(53) Exchange +(49) Exchange Input [1]: [count#31] Arguments: SinglePartition, true, [id=#32] -(54) HashAggregate [codegen id : 13] +(50) HashAggregate [codegen id : 13] Input [1]: [count#31] Keys: [] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#33] Results [1]: [count(1)#33 AS count(1)#34] -(55) CollectLimit +(51) CollectLimit Input [1]: [count(1)#34] Arguments: 100 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt index 10a2166ce761..c87811331e23 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt @@ -6,76 +6,72 @@ CollectLimit WholeStageCodegen (12) HashAggregate [count,count] HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #2 + WholeStageCodegen (11) HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (11) - HashAggregate [c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - Project [d_date,c_first_name,c_last_name] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_customer_sk] + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + Project [d_date,c_first_name,c_last_name] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Project [d_date_sk,d_date] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk,d_date] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Filter [c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] + Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + Filter [c_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (6) + HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) + Exchange [c_last_name,c_first_name,d_date] #6 + WholeStageCodegen (5) HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #6 - WholeStageCodegen (5) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_sold_date_sk,cs_bill_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] + InputAdapter + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (10) + HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (10) + Exchange [c_last_name,c_first_name,d_date] #8 + WholeStageCodegen (9) HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (9) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_sold_date_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + Project [ws_bill_customer_sk,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_bill_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] + InputAdapter + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/explain.txt index 377bd36f520e..895715312c40 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/explain.txt @@ -1,71 +1,67 @@ == Physical Plan == -* HashAggregate (67) -+- Exchange (66) - +- * HashAggregate (65) - +- * HashAggregate (64) - +- * HashAggregate (63) - +- * HashAggregate (62) - +- * HashAggregate (61) - +- * HashAggregate (60) - +- Exchange (59) - +- * HashAggregate (58) - +- SortMergeJoin LeftAnti (57) - :- SortMergeJoin LeftAnti (39) - : :- * Sort (21) - : : +- Exchange (20) - : : +- * Project (19) - : : +- * SortMergeJoin Inner (18) - : : :- * Sort (12) - : : : +- Exchange (11) - : : : +- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.date_dim (4) - : : +- * Sort (17) - : : +- Exchange (16) - : : +- * Filter (15) - : : +- * ColumnarToRow (14) - : : +- Scan parquet default.customer (13) - : +- * Sort (38) - : +- Exchange (37) - : +- * HashAggregate (36) - : +- Exchange (35) - : +- * HashAggregate (34) - : +- * Project (33) - : +- * SortMergeJoin Inner (32) - : :- * Sort (29) - : : +- Exchange (28) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Filter (24) - : : : +- * ColumnarToRow (23) - : : : +- Scan parquet default.catalog_sales (22) - : : +- ReusedExchange (25) - : +- * Sort (31) - : +- ReusedExchange (30) - +- * Sort (56) - +- Exchange (55) - +- * HashAggregate (54) - +- Exchange (53) - +- * HashAggregate (52) - +- * Project (51) - +- * SortMergeJoin Inner (50) - :- * Sort (47) - : +- Exchange (46) - : +- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Filter (42) - : : +- * ColumnarToRow (41) - : : +- Scan parquet default.web_sales (40) - : +- ReusedExchange (43) - +- * Sort (49) - +- ReusedExchange (48) +* HashAggregate (63) ++- Exchange (62) + +- * HashAggregate (61) + +- * HashAggregate (60) + +- Exchange (59) + +- * HashAggregate (58) + +- SortMergeJoin LeftAnti (57) + :- SortMergeJoin LeftAnti (39) + : :- * Sort (21) + : : +- Exchange (20) + : : +- * Project (19) + : : +- * SortMergeJoin Inner (18) + : : :- * Sort (12) + : : : +- Exchange (11) + : : : +- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Project (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.date_dim (4) + : : +- * Sort (17) + : : +- Exchange (16) + : : +- * Filter (15) + : : +- * ColumnarToRow (14) + : : +- Scan parquet default.customer (13) + : +- * Sort (38) + : +- Exchange (37) + : +- * HashAggregate (36) + : +- Exchange (35) + : +- * HashAggregate (34) + : +- * Project (33) + : +- * SortMergeJoin Inner (32) + : :- * Sort (29) + : : +- Exchange (28) + : : +- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * Filter (24) + : : : +- * ColumnarToRow (23) + : : : +- Scan parquet default.catalog_sales (22) + : : +- ReusedExchange (25) + : +- * Sort (31) + : +- ReusedExchange (30) + +- * Sort (56) + +- Exchange (55) + +- * HashAggregate (54) + +- Exchange (53) + +- * HashAggregate (52) + +- * Project (51) + +- * SortMergeJoin Inner (50) + :- * Sort (47) + : +- Exchange (46) + : +- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) + : :- * Filter (42) + : : +- * ColumnarToRow (41) + : : +- Scan parquet default.web_sales (40) + : +- ReusedExchange (43) + +- * Sort (49) + +- ReusedExchange (48) (1) Scan parquet default.store_sales @@ -338,48 +334,20 @@ Input [3]: [c_last_name#10, c_first_name#9, d_date#4] Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#10, c_first_name#9, d_date#4] - -(61) HashAggregate [codegen id : 25] -Input [3]: [c_last_name#10, c_first_name#9, d_date#4] -Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#10, c_first_name#9, d_date#4] - -(62) HashAggregate [codegen id : 25] -Input [3]: [c_last_name#10, c_first_name#9, d_date#4] -Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#10, c_first_name#9, d_date#4] - -(63) HashAggregate [codegen id : 25] -Input [3]: [c_last_name#10, c_first_name#9, d_date#4] -Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#10, c_first_name#9, d_date#4] - -(64) HashAggregate [codegen id : 25] -Input [3]: [c_last_name#10, c_first_name#9, d_date#4] -Keys [3]: [c_last_name#10, c_first_name#9, d_date#4] -Functions: [] -Aggregate Attributes: [] Results: [] -(65) HashAggregate [codegen id : 25] +(61) HashAggregate [codegen id : 25] Input: [] Keys: [] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#34] Results [1]: [count#35] -(66) Exchange +(62) Exchange Input [1]: [count#35] Arguments: SinglePartition, true, [id=#36] -(67) HashAggregate [codegen id : 26] +(63) HashAggregate [codegen id : 26] Input [1]: [count#35] Keys: [] Functions [1]: [count(1)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/simplified.txt index 8dd59340cf06..6ce40106eea4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/simplified.txt @@ -5,113 +5,109 @@ WholeStageCodegen (26) WholeStageCodegen (25) HashAggregate [count,count] HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #2 + WholeStageCodegen (24) HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (24) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - WholeStageCodegen (7) - Sort [c_last_name,c_first_name,d_date] + SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + WholeStageCodegen (7) + Sort [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #3 + WholeStageCodegen (6) + Project [d_date,c_first_name,c_last_name] + SortMergeJoin [ss_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (3) + Sort [ss_customer_sk] + InputAdapter + Exchange [ss_customer_sk] #4 + WholeStageCodegen (2) + Project [ss_customer_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + Project [d_date_sk,d_date] + Filter [d_month_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + WholeStageCodegen (5) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #6 + WholeStageCodegen (4) + Filter [c_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] + WholeStageCodegen (15) + Sort [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #7 + WholeStageCodegen (14) + HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_last_name,c_first_name,d_date] #3 - WholeStageCodegen (6) - Project [d_date,c_first_name,c_last_name] - SortMergeJoin [ss_customer_sk,c_customer_sk] + Exchange [c_last_name,c_first_name,d_date] #8 + WholeStageCodegen (13) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (10) + Sort [cs_bill_customer_sk] + InputAdapter + Exchange [cs_bill_customer_sk] #9 + WholeStageCodegen (9) + Project [cs_bill_customer_sk,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_sold_date_sk,cs_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] + InputAdapter + ReusedExchange [d_date_sk,d_date] #5 + InputAdapter + WholeStageCodegen (12) + Sort [c_customer_sk] + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #6 + WholeStageCodegen (23) + Sort [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #10 + WholeStageCodegen (22) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #11 + WholeStageCodegen (21) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (3) - Sort [ss_customer_sk] + WholeStageCodegen (18) + Sort [ws_bill_customer_sk] InputAdapter - Exchange [ss_customer_sk] #4 - WholeStageCodegen (2) - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_customer_sk] + Exchange [ws_bill_customer_sk] #12 + WholeStageCodegen (17) + Project [ws_bill_customer_sk,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Project [d_date_sk,d_date] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] + ReusedExchange [d_date_sk,d_date] #5 InputAdapter - WholeStageCodegen (5) + WholeStageCodegen (20) Sort [c_customer_sk] InputAdapter - Exchange [c_customer_sk] #6 - WholeStageCodegen (4) - Filter [c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] - WholeStageCodegen (15) - Sort [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #7 - WholeStageCodegen (14) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (13) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - SortMergeJoin [cs_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (10) - Sort [cs_bill_customer_sk] - InputAdapter - Exchange [cs_bill_customer_sk] #9 - WholeStageCodegen (9) - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] - InputAdapter - ReusedExchange [d_date_sk,d_date] #5 - InputAdapter - WholeStageCodegen (12) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #6 - WholeStageCodegen (23) - Sort [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #10 - WholeStageCodegen (22) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #11 - WholeStageCodegen (21) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - SortMergeJoin [ws_bill_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (18) - Sort [ws_bill_customer_sk] - InputAdapter - Exchange [ws_bill_customer_sk] #12 - WholeStageCodegen (17) - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_sold_date_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] - InputAdapter - ReusedExchange [d_date_sk,d_date] #5 - InputAdapter - WholeStageCodegen (20) - Sort [c_customer_sk] - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #6 + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt index 3d59a670b7e8..5fa95b952c55 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt @@ -1,58 +1,54 @@ == Physical Plan == -* HashAggregate (54) -+- Exchange (53) - +- * HashAggregate (52) - +- * HashAggregate (51) - +- * HashAggregate (50) - +- * HashAggregate (49) - +- * HashAggregate (48) - +- * HashAggregate (47) - +- Exchange (46) - +- * HashAggregate (45) - +- * BroadcastHashJoin LeftAnti BuildRight (44) - :- * BroadcastHashJoin LeftAnti BuildRight (30) - : :- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.date_dim (4) - : : +- BroadcastExchange (14) - : : +- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.customer (11) - : +- BroadcastExchange (29) - : +- * HashAggregate (28) - : +- Exchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * Filter (19) - : : : +- * ColumnarToRow (18) - : : : +- Scan parquet default.catalog_sales (17) - : : +- ReusedExchange (20) - : +- ReusedExchange (23) - +- BroadcastExchange (43) - +- * HashAggregate (42) - +- Exchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * Filter (33) - : : +- * ColumnarToRow (32) - : : +- Scan parquet default.web_sales (31) - : +- ReusedExchange (34) - +- ReusedExchange (37) +* HashAggregate (50) ++- Exchange (49) + +- * HashAggregate (48) + +- * HashAggregate (47) + +- Exchange (46) + +- * HashAggregate (45) + +- * BroadcastHashJoin LeftAnti BuildRight (44) + :- * BroadcastHashJoin LeftAnti BuildRight (30) + : :- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Project (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.date_dim (4) + : : +- BroadcastExchange (14) + : : +- * Filter (13) + : : +- * ColumnarToRow (12) + : : +- Scan parquet default.customer (11) + : +- BroadcastExchange (29) + : +- * HashAggregate (28) + : +- Exchange (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) + : : :- * Filter (19) + : : : +- * ColumnarToRow (18) + : : : +- Scan parquet default.catalog_sales (17) + : : +- ReusedExchange (20) + : +- ReusedExchange (23) + +- BroadcastExchange (43) + +- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) + :- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * Filter (33) + : : +- * ColumnarToRow (32) + : : +- Scan parquet default.web_sales (31) + : +- ReusedExchange (34) + +- ReusedExchange (37) (1) Scan parquet default.store_sales @@ -273,48 +269,20 @@ Input [3]: [c_last_name#9, c_first_name#8, d_date#4] Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#9, c_first_name#8, d_date#4] - -(48) HashAggregate [codegen id : 12] -Input [3]: [c_last_name#9, c_first_name#8, d_date#4] -Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#9, c_first_name#8, d_date#4] - -(49) HashAggregate [codegen id : 12] -Input [3]: [c_last_name#9, c_first_name#8, d_date#4] -Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#9, c_first_name#8, d_date#4] - -(50) HashAggregate [codegen id : 12] -Input [3]: [c_last_name#9, c_first_name#8, d_date#4] -Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#9, c_first_name#8, d_date#4] - -(51) HashAggregate [codegen id : 12] -Input [3]: [c_last_name#9, c_first_name#8, d_date#4] -Keys [3]: [c_last_name#9, c_first_name#8, d_date#4] -Functions: [] -Aggregate Attributes: [] Results: [] -(52) HashAggregate [codegen id : 12] +(48) HashAggregate [codegen id : 12] Input: [] Keys: [] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#30] Results [1]: [count#31] -(53) Exchange +(49) Exchange Input [1]: [count#31] Arguments: SinglePartition, true, [id=#32] -(54) HashAggregate [codegen id : 13] +(50) HashAggregate [codegen id : 13] Input [1]: [count#31] Keys: [] Functions [1]: [count(1)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt index a5b57a4ac945..980aeca8869f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt @@ -5,76 +5,72 @@ WholeStageCodegen (13) WholeStageCodegen (12) HashAggregate [count,count] HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #2 + WholeStageCodegen (11) HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (11) - HashAggregate [c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - Project [d_date,c_first_name,c_last_name] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_customer_sk] + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + Project [d_date,c_first_name,c_last_name] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Project [d_date_sk,d_date] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk,d_date] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Filter [c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] + Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + Filter [c_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (6) + HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) + Exchange [c_last_name,c_first_name,d_date] #6 + WholeStageCodegen (5) HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #6 - WholeStageCodegen (5) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_sold_date_sk,cs_bill_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] + InputAdapter + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (10) + HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (10) + Exchange [c_last_name,c_first_name,d_date] #8 + WholeStageCodegen (9) HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (9) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_sold_date_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + Project [ws_bill_customer_sk,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_bill_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] + InputAdapter + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt index dad6098ce468..20a24886ea24 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt @@ -1,114 +1,112 @@ == Physical Plan == -TakeOrderedAndProject (110) -+- * BroadcastHashJoin Inner BuildRight (109) - :- * Project (87) - : +- * Filter (86) - : +- * HashAggregate (85) - : +- Exchange (84) - : +- * HashAggregate (83) - : +- * Project (82) - : +- * BroadcastHashJoin Inner BuildRight (81) - : :- * Project (71) - : : +- * BroadcastHashJoin Inner BuildRight (70) - : : :- SortMergeJoin LeftSemi (64) +TakeOrderedAndProject (108) ++- * BroadcastHashJoin Inner BuildRight (107) + :- * Project (85) + : +- * Filter (84) + : +- * HashAggregate (83) + : +- Exchange (82) + : +- * HashAggregate (81) + : +- * Project (80) + : +- * BroadcastHashJoin Inner BuildRight (79) + : :- * Project (69) + : : +- * BroadcastHashJoin Inner BuildRight (68) + : : :- SortMergeJoin LeftSemi (62) : : : :- * Sort (5) : : : : +- Exchange (4) : : : : +- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.store_sales (1) - : : : +- * Sort (63) - : : : +- Exchange (62) - : : : +- * Project (61) - : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : +- * Sort (61) + : : : +- Exchange (60) + : : : +- * Project (59) + : : : +- * BroadcastHashJoin Inner BuildRight (58) : : : :- * Filter (8) : : : : +- * ColumnarToRow (7) : : : : +- Scan parquet default.item (6) - : : : +- BroadcastExchange (59) - : : : +- * HashAggregate (58) - : : : +- * HashAggregate (57) - : : : +- * HashAggregate (56) - : : : +- Exchange (55) - : : : +- * HashAggregate (54) - : : : +- SortMergeJoin LeftSemi (53) - : : : :- SortMergeJoin LeftSemi (41) - : : : : :- * Sort (26) - : : : : : +- Exchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (18) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : : :- * Filter (11) - : : : : : : : +- * ColumnarToRow (10) - : : : : : : : +- Scan parquet default.store_sales (9) - : : : : : : +- BroadcastExchange (16) - : : : : : : +- * Project (15) - : : : : : : +- * Filter (14) - : : : : : : +- * ColumnarToRow (13) - : : : : : : +- Scan parquet default.date_dim (12) - : : : : : +- BroadcastExchange (22) - : : : : : +- * Filter (21) - : : : : : +- * ColumnarToRow (20) - : : : : : +- Scan parquet default.item (19) - : : : : +- * Sort (40) - : : : : +- Exchange (39) - : : : : +- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Project (32) - : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : :- * Filter (29) - : : : : : : +- * ColumnarToRow (28) - : : : : : : +- Scan parquet default.catalog_sales (27) - : : : : : +- ReusedExchange (30) - : : : : +- BroadcastExchange (36) - : : : : +- * Filter (35) - : : : : +- * ColumnarToRow (34) - : : : : +- Scan parquet default.item (33) - : : : +- * Sort (52) - : : : +- Exchange (51) - : : : +- * Project (50) - : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * Project (47) - : : : : +- * BroadcastHashJoin Inner BuildRight (46) - : : : : :- * Filter (44) - : : : : : +- * ColumnarToRow (43) - : : : : : +- Scan parquet default.web_sales (42) - : : : : +- ReusedExchange (45) - : : : +- ReusedExchange (48) - : : +- BroadcastExchange (69) - : : +- * Project (68) - : : +- * Filter (67) - : : +- * ColumnarToRow (66) - : : +- Scan parquet default.date_dim (65) - : +- BroadcastExchange (80) - : +- SortMergeJoin LeftSemi (79) - : :- * Sort (76) - : : +- Exchange (75) - : : +- * Filter (74) - : : +- * ColumnarToRow (73) - : : +- Scan parquet default.item (72) - : +- * Sort (78) - : +- ReusedExchange (77) - +- BroadcastExchange (108) - +- * Project (107) - +- * Filter (106) - +- * HashAggregate (105) - +- Exchange (104) - +- * HashAggregate (103) - +- * Project (102) - +- * BroadcastHashJoin Inner BuildRight (101) - :- * Project (99) - : +- * BroadcastHashJoin Inner BuildRight (98) - : :- SortMergeJoin LeftSemi (92) - : : :- * Sort (89) - : : : +- ReusedExchange (88) - : : +- * Sort (91) - : : +- ReusedExchange (90) - : +- BroadcastExchange (97) - : +- * Project (96) - : +- * Filter (95) - : +- * ColumnarToRow (94) - : +- Scan parquet default.date_dim (93) - +- ReusedExchange (100) + : : : +- BroadcastExchange (57) + : : : +- * HashAggregate (56) + : : : +- Exchange (55) + : : : +- * HashAggregate (54) + : : : +- SortMergeJoin LeftSemi (53) + : : : :- SortMergeJoin LeftSemi (41) + : : : : :- * Sort (26) + : : : : : +- Exchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * Project (18) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : : :- * Filter (11) + : : : : : : : +- * ColumnarToRow (10) + : : : : : : : +- Scan parquet default.store_sales (9) + : : : : : : +- BroadcastExchange (16) + : : : : : : +- * Project (15) + : : : : : : +- * Filter (14) + : : : : : : +- * ColumnarToRow (13) + : : : : : : +- Scan parquet default.date_dim (12) + : : : : : +- BroadcastExchange (22) + : : : : : +- * Filter (21) + : : : : : +- * ColumnarToRow (20) + : : : : : +- Scan parquet default.item (19) + : : : : +- * Sort (40) + : : : : +- Exchange (39) + : : : : +- * Project (38) + : : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : : :- * Project (32) + : : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : : :- * Filter (29) + : : : : : : +- * ColumnarToRow (28) + : : : : : : +- Scan parquet default.catalog_sales (27) + : : : : : +- ReusedExchange (30) + : : : : +- BroadcastExchange (36) + : : : : +- * Filter (35) + : : : : +- * ColumnarToRow (34) + : : : : +- Scan parquet default.item (33) + : : : +- * Sort (52) + : : : +- Exchange (51) + : : : +- * Project (50) + : : : +- * BroadcastHashJoin Inner BuildRight (49) + : : : :- * Project (47) + : : : : +- * BroadcastHashJoin Inner BuildRight (46) + : : : : :- * Filter (44) + : : : : : +- * ColumnarToRow (43) + : : : : : +- Scan parquet default.web_sales (42) + : : : : +- ReusedExchange (45) + : : : +- ReusedExchange (48) + : : +- BroadcastExchange (67) + : : +- * Project (66) + : : +- * Filter (65) + : : +- * ColumnarToRow (64) + : : +- Scan parquet default.date_dim (63) + : +- BroadcastExchange (78) + : +- SortMergeJoin LeftSemi (77) + : :- * Sort (74) + : : +- Exchange (73) + : : +- * Filter (72) + : : +- * ColumnarToRow (71) + : : +- Scan parquet default.item (70) + : +- * Sort (76) + : +- ReusedExchange (75) + +- BroadcastExchange (106) + +- * Project (105) + +- * Filter (104) + +- * HashAggregate (103) + +- Exchange (102) + +- * HashAggregate (101) + +- * Project (100) + +- * BroadcastHashJoin Inner BuildRight (99) + :- * Project (97) + : +- * BroadcastHashJoin Inner BuildRight (96) + : :- SortMergeJoin LeftSemi (90) + : : :- * Sort (87) + : : : +- ReusedExchange (86) + : : +- * Sort (89) + : : +- ReusedExchange (88) + : +- BroadcastExchange (95) + : +- * Project (94) + : +- * Filter (93) + : +- * ColumnarToRow (92) + : +- Scan parquet default.date_dim (91) + +- ReusedExchange (98) (1) Scan parquet default.store_sales @@ -362,448 +360,434 @@ Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#14, class_id#15, category_id#16] -(57) HashAggregate [codegen id : 16] -Input [3]: [brand_id#14, class_id#15, category_id#16] -Keys [3]: [brand_id#14, class_id#15, category_id#16] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#14, class_id#15, category_id#16] - -(58) HashAggregate [codegen id : 16] -Input [3]: [brand_id#14, class_id#15, category_id#16] -Keys [3]: [brand_id#14, class_id#15, category_id#16] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#14, class_id#15, category_id#16] - -(59) BroadcastExchange +(57) BroadcastExchange Input [3]: [brand_id#14, class_id#15, category_id#16] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#26] -(60) BroadcastHashJoin [codegen id : 17] +(58) BroadcastHashJoin [codegen id : 17] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#14, class_id#15, category_id#16] Join condition: None -(61) Project [codegen id : 17] +(59) Project [codegen id : 17] Output [1]: [i_item_sk#6 AS ss_item_sk#27] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#14, class_id#15, category_id#16] -(62) Exchange +(60) Exchange Input [1]: [ss_item_sk#27] Arguments: hashpartitioning(ss_item_sk#27, 5), true, [id=#28] -(63) Sort [codegen id : 18] +(61) Sort [codegen id : 18] Input [1]: [ss_item_sk#27] Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 -(64) SortMergeJoin +(62) SortMergeJoin Left keys [1]: [ss_item_sk#2] Right keys [1]: [ss_item_sk#27] Join condition: None -(65) Scan parquet default.date_dim +(63) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_week_seq#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(66) ColumnarToRow [codegen id : 19] +(64) ColumnarToRow [codegen id : 19] Input [2]: [d_date_sk#10, d_week_seq#29] -(67) Filter [codegen id : 19] +(65) Filter [codegen id : 19] Input [2]: [d_date_sk#10, d_week_seq#29] Condition : ((isnotnull(d_week_seq#29) AND (d_week_seq#29 = Subquery scalar-subquery#30, [id=#31])) AND isnotnull(d_date_sk#10)) -(68) Project [codegen id : 19] +(66) Project [codegen id : 19] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_week_seq#29] -(69) BroadcastExchange +(67) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#32] -(70) BroadcastHashJoin [codegen id : 38] +(68) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(71) Project [codegen id : 38] +(69) Project [codegen id : 38] Output [3]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4] Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(72) Scan parquet default.item +(70) Scan parquet default.item Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(73) ColumnarToRow [codegen id : 20] +(71) ColumnarToRow [codegen id : 20] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(74) Filter [codegen id : 20] +(72) Filter [codegen id : 20] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) -(75) Exchange +(73) Exchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: hashpartitioning(i_item_sk#6, 5), true, [id=#33] -(76) Sort [codegen id : 21] +(74) Sort [codegen id : 21] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: [i_item_sk#6 ASC NULLS FIRST], false, 0 -(77) ReusedExchange [Reuses operator id: 62] +(75) ReusedExchange [Reuses operator id: 60] Output [1]: [ss_item_sk#27] -(78) Sort [codegen id : 37] +(76) Sort [codegen id : 37] Input [1]: [ss_item_sk#27] Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 -(79) SortMergeJoin +(77) SortMergeJoin Left keys [1]: [i_item_sk#6] Right keys [1]: [ss_item_sk#27] Join condition: None -(80) BroadcastExchange +(78) BroadcastExchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] -(81) BroadcastHashJoin [codegen id : 38] +(79) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#6] Join condition: None -(82) Project [codegen id : 38] +(80) Project [codegen id : 38] Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#7, i_class_id#8, i_category_id#9] Input [7]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(83) HashAggregate [codegen id : 38] +(81) HashAggregate [codegen id : 38] Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#7, i_class_id#8, i_category_id#9] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#35, isEmpty#36, count#37] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#38, isEmpty#39, count#40] -(84) Exchange +(82) Exchange Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#38, isEmpty#39, count#40] Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#41] -(85) HashAggregate [codegen id : 78] +(83) HashAggregate [codegen id : 78] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#38, isEmpty#39, count#40] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42, count(1)#43] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 AS sales#44, count(1)#43 AS number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#42 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46] -(86) Filter [codegen id : 78] +(84) Filter [codegen id : 78] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46 as decimal(32,6)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(32,6)))) -(87) Project [codegen id : 78] +(85) Project [codegen id : 78] Output [6]: [store AS channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#46] -(88) ReusedExchange [Reuses operator id: 4] +(86) ReusedExchange [Reuses operator id: 4] Output [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] -(89) Sort [codegen id : 40] +(87) Sort [codegen id : 40] Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] Arguments: [ss_item_sk#2 ASC NULLS FIRST], false, 0 -(90) ReusedExchange [Reuses operator id: 62] +(88) ReusedExchange [Reuses operator id: 60] Output [1]: [ss_item_sk#27] -(91) Sort [codegen id : 56] +(89) Sort [codegen id : 56] Input [1]: [ss_item_sk#27] Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 -(92) SortMergeJoin +(90) SortMergeJoin Left keys [1]: [ss_item_sk#2] Right keys [1]: [ss_item_sk#27] Join condition: None -(93) Scan parquet default.date_dim +(91) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_week_seq#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(94) ColumnarToRow [codegen id : 57] +(92) ColumnarToRow [codegen id : 57] Input [2]: [d_date_sk#10, d_week_seq#29] -(95) Filter [codegen id : 57] +(93) Filter [codegen id : 57] Input [2]: [d_date_sk#10, d_week_seq#29] Condition : ((isnotnull(d_week_seq#29) AND (d_week_seq#29 = Subquery scalar-subquery#50, [id=#51])) AND isnotnull(d_date_sk#10)) -(96) Project [codegen id : 57] +(94) Project [codegen id : 57] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_week_seq#29] -(97) BroadcastExchange +(95) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#52] -(98) BroadcastHashJoin [codegen id : 76] +(96) BroadcastHashJoin [codegen id : 76] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(99) Project [codegen id : 76] +(97) Project [codegen id : 76] Output [3]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4] Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(100) ReusedExchange [Reuses operator id: 80] +(98) ReusedExchange [Reuses operator id: 78] Output [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -(101) BroadcastHashJoin [codegen id : 76] +(99) BroadcastHashJoin [codegen id : 76] Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#53] Join condition: None -(102) Project [codegen id : 76] +(100) Project [codegen id : 76] Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#54, i_class_id#55, i_category_id#56] Input [7]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -(103) HashAggregate [codegen id : 76] +(101) HashAggregate [codegen id : 76] Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#54, i_class_id#55, i_category_id#56] Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#57, isEmpty#58, count#59] Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#60, isEmpty#61, count#62] -(104) Exchange +(102) Exchange Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#60, isEmpty#61, count#62] Arguments: hashpartitioning(i_brand_id#54, i_class_id#55, i_category_id#56, 5), true, [id=#63] -(105) HashAggregate [codegen id : 77] +(103) HashAggregate [codegen id : 77] Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#60, isEmpty#61, count#62] Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64, count(1)#65] Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sales#66, count(1)#65 AS number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68] -(106) Filter [codegen id : 77] +(104) Filter [codegen id : 77] Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#47, [id=#48] as decimal(32,6)))) -(107) Project [codegen id : 77] +(105) Project [codegen id : 77] Output [6]: [store AS channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67] Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#68] -(108) BroadcastExchange +(106) BroadcastExchange Input [6]: [channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#70] -(109) BroadcastHashJoin [codegen id : 78] +(107) BroadcastHashJoin [codegen id : 78] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] Join condition: None -(110) TakeOrderedAndProject +(108) TakeOrderedAndProject Input [12]: [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67] Arguments: 100, [i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#49, i_brand_id#7, i_class_id#8, i_category_id#9, sales#44, number_sales#45, channel#69, i_brand_id#54, i_class_id#55, i_category_id#56, sales#66, number_sales#67] ===== Subqueries ===== -Subquery:1 Hosting operator id = 86 Hosting Expression = Subquery scalar-subquery#47, [id=#48] -* HashAggregate (136) -+- Exchange (135) - +- * HashAggregate (134) - +- Union (133) - :- * Project (120) - : +- * BroadcastHashJoin Inner BuildRight (119) - : :- * Filter (113) - : : +- * ColumnarToRow (112) - : : +- Scan parquet default.store_sales (111) - : +- BroadcastExchange (118) - : +- * Project (117) - : +- * Filter (116) - : +- * ColumnarToRow (115) - : +- Scan parquet default.date_dim (114) - :- * Project (126) - : +- * BroadcastHashJoin Inner BuildRight (125) - : :- * Filter (123) - : : +- * ColumnarToRow (122) - : : +- Scan parquet default.catalog_sales (121) - : +- ReusedExchange (124) - +- * Project (132) - +- * BroadcastHashJoin Inner BuildRight (131) - :- * Filter (129) - : +- * ColumnarToRow (128) - : +- Scan parquet default.web_sales (127) - +- ReusedExchange (130) - - -(111) Scan parquet default.store_sales +Subquery:1 Hosting operator id = 84 Hosting Expression = Subquery scalar-subquery#47, [id=#48] +* HashAggregate (134) ++- Exchange (133) + +- * HashAggregate (132) + +- Union (131) + :- * Project (118) + : +- * BroadcastHashJoin Inner BuildRight (117) + : :- * Filter (111) + : : +- * ColumnarToRow (110) + : : +- Scan parquet default.store_sales (109) + : +- BroadcastExchange (116) + : +- * Project (115) + : +- * Filter (114) + : +- * ColumnarToRow (113) + : +- Scan parquet default.date_dim (112) + :- * Project (124) + : +- * BroadcastHashJoin Inner BuildRight (123) + : :- * Filter (121) + : : +- * ColumnarToRow (120) + : : +- Scan parquet default.catalog_sales (119) + : +- ReusedExchange (122) + +- * Project (130) + +- * BroadcastHashJoin Inner BuildRight (129) + :- * Filter (127) + : +- * ColumnarToRow (126) + : +- Scan parquet default.web_sales (125) + +- ReusedExchange (128) + + +(109) Scan parquet default.store_sales Output [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(112) ColumnarToRow [codegen id : 2] +(110) ColumnarToRow [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] -(113) Filter [codegen id : 2] +(111) Filter [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Condition : isnotnull(ss_sold_date_sk#1) -(114) Scan parquet default.date_dim +(112) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_year#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(115) ColumnarToRow [codegen id : 1] +(113) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] -(116) Filter [codegen id : 1] +(114) Filter [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] Condition : (((isnotnull(d_year#11) AND (d_year#11 >= 1998)) AND (d_year#11 <= 2000)) AND isnotnull(d_date_sk#10)) -(117) Project [codegen id : 1] +(115) Project [codegen id : 1] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_year#11] -(118) BroadcastExchange +(116) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#71] -(119) BroadcastHashJoin [codegen id : 2] +(117) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(120) Project [codegen id : 2] +(118) Project [codegen id : 2] Output [2]: [ss_quantity#3 AS quantity#72, ss_list_price#4 AS list_price#73] Input [4]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(121) Scan parquet default.catalog_sales +(119) Scan parquet default.catalog_sales Output [3]: [cs_sold_date_sk#18, cs_quantity#74, cs_list_price#75] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(122) ColumnarToRow [codegen id : 4] +(120) ColumnarToRow [codegen id : 4] Input [3]: [cs_sold_date_sk#18, cs_quantity#74, cs_list_price#75] -(123) Filter [codegen id : 4] +(121) Filter [codegen id : 4] Input [3]: [cs_sold_date_sk#18, cs_quantity#74, cs_list_price#75] Condition : isnotnull(cs_sold_date_sk#18) -(124) ReusedExchange [Reuses operator id: 118] +(122) ReusedExchange [Reuses operator id: 116] Output [1]: [d_date_sk#10] -(125) BroadcastHashJoin [codegen id : 4] +(123) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#18] Right keys [1]: [d_date_sk#10] Join condition: None -(126) Project [codegen id : 4] +(124) Project [codegen id : 4] Output [2]: [cs_quantity#74 AS quantity#76, cs_list_price#75 AS list_price#77] Input [4]: [cs_sold_date_sk#18, cs_quantity#74, cs_list_price#75, d_date_sk#10] -(127) Scan parquet default.web_sales +(125) Scan parquet default.web_sales Output [3]: [ws_sold_date_sk#22, ws_quantity#78, ws_list_price#79] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(128) ColumnarToRow [codegen id : 6] +(126) ColumnarToRow [codegen id : 6] Input [3]: [ws_sold_date_sk#22, ws_quantity#78, ws_list_price#79] -(129) Filter [codegen id : 6] +(127) Filter [codegen id : 6] Input [3]: [ws_sold_date_sk#22, ws_quantity#78, ws_list_price#79] Condition : isnotnull(ws_sold_date_sk#22) -(130) ReusedExchange [Reuses operator id: 118] +(128) ReusedExchange [Reuses operator id: 116] Output [1]: [d_date_sk#10] -(131) BroadcastHashJoin [codegen id : 6] +(129) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#22] Right keys [1]: [d_date_sk#10] Join condition: None -(132) Project [codegen id : 6] +(130) Project [codegen id : 6] Output [2]: [ws_quantity#78 AS quantity#80, ws_list_price#79 AS list_price#81] Input [4]: [ws_sold_date_sk#22, ws_quantity#78, ws_list_price#79, d_date_sk#10] -(133) Union +(131) Union -(134) HashAggregate [codegen id : 7] +(132) HashAggregate [codegen id : 7] Input [2]: [quantity#72, list_price#73] Keys: [] Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#73 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [2]: [sum#82, count#83] Results [2]: [sum#84, count#85] -(135) Exchange +(133) Exchange Input [2]: [sum#84, count#85] Arguments: SinglePartition, true, [id=#86] -(136) HashAggregate [codegen id : 8] +(134) HashAggregate [codegen id : 8] Input [2]: [sum#84, count#85] Keys: [] Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#73 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#73 as decimal(12,2)))), DecimalType(18,2), true))#87] Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#73 as decimal(12,2)))), DecimalType(18,2), true))#87 AS average_sales#88] -Subquery:2 Hosting operator id = 67 Hosting Expression = Subquery scalar-subquery#30, [id=#31] -* Project (140) -+- * Filter (139) - +- * ColumnarToRow (138) - +- Scan parquet default.date_dim (137) +Subquery:2 Hosting operator id = 65 Hosting Expression = Subquery scalar-subquery#30, [id=#31] +* Project (138) ++- * Filter (137) + +- * ColumnarToRow (136) + +- Scan parquet default.date_dim (135) -(137) Scan parquet default.date_dim +(135) Scan parquet default.date_dim Output [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(138) ColumnarToRow [codegen id : 1] +(136) ColumnarToRow [codegen id : 1] Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] -(139) Filter [codegen id : 1] +(137) Filter [codegen id : 1] Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#89)) AND isnotnull(d_dom#90)) AND (d_year#11 = 1999)) AND (d_moy#89 = 12)) AND (d_dom#90 = 16)) -(140) Project [codegen id : 1] +(138) Project [codegen id : 1] Output [1]: [d_week_seq#29] Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] -Subquery:3 Hosting operator id = 106 Hosting Expression = ReusedSubquery Subquery scalar-subquery#47, [id=#48] +Subquery:3 Hosting operator id = 104 Hosting Expression = ReusedSubquery Subquery scalar-subquery#47, [id=#48] -Subquery:4 Hosting operator id = 95 Hosting Expression = Subquery scalar-subquery#50, [id=#51] -* Project (144) -+- * Filter (143) - +- * ColumnarToRow (142) - +- Scan parquet default.date_dim (141) +Subquery:4 Hosting operator id = 93 Hosting Expression = Subquery scalar-subquery#50, [id=#51] +* Project (142) ++- * Filter (141) + +- * ColumnarToRow (140) + +- Scan parquet default.date_dim (139) -(141) Scan parquet default.date_dim +(139) Scan parquet default.date_dim Output [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(142) ColumnarToRow [codegen id : 1] +(140) ColumnarToRow [codegen id : 1] Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] -(143) Filter [codegen id : 1] +(141) Filter [codegen id : 1] Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#89)) AND isnotnull(d_dom#90)) AND (d_year#11 = 1998)) AND (d_moy#89 = 12)) AND (d_dom#90 = 16)) -(144) Project [codegen id : 1] +(142) Project [codegen id : 1] Output [1]: [d_week_seq#29] Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt index d6b8ba4395d2..1e0c188cd48c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt @@ -80,82 +80,80 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastExchange #4 WholeStageCodegen (16) HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (15) - HashAggregate [brand_id,class_id,category_id] - InputAdapter - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (6) - Sort [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] + InputAdapter + Exchange [brand_id,class_id,category_id] #5 + WholeStageCodegen (15) + HashAggregate [brand_id,class_id,category_id] + InputAdapter + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (6) + Sort [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #6 + WholeStageCodegen (5) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk,ss_sold_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (10) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #9 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] + BroadcastExchange #7 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (10) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #9 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk,cs_sold_date_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #7 + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (14) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #11 - WholeStageCodegen (13) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk,ws_sold_date_sk] + ReusedExchange [d_date_sk] #7 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (8) + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (14) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #11 + WholeStageCodegen (13) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk,ws_sold_date_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #7 + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + ReusedExchange [d_date_sk] #7 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 InputAdapter BroadcastExchange #12 WholeStageCodegen (19) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt index 1af2e69d5733..827fcbff5498 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt @@ -1,104 +1,102 @@ == Physical Plan == -TakeOrderedAndProject (100) -+- * BroadcastHashJoin Inner BuildRight (99) - :- * Project (77) - : +- * Filter (76) - : +- * HashAggregate (75) - : +- Exchange (74) - : +- * HashAggregate (73) - : +- * Project (72) - : +- * BroadcastHashJoin Inner BuildRight (71) - : :- * Project (65) - : : +- * BroadcastHashJoin Inner BuildRight (64) - : : :- * BroadcastHashJoin LeftSemi BuildRight (57) +TakeOrderedAndProject (98) ++- * BroadcastHashJoin Inner BuildRight (97) + :- * Project (75) + : +- * Filter (74) + : +- * HashAggregate (73) + : +- Exchange (72) + : +- * HashAggregate (71) + : +- * Project (70) + : +- * BroadcastHashJoin Inner BuildRight (69) + : :- * Project (63) + : : +- * BroadcastHashJoin Inner BuildRight (62) + : : :- * BroadcastHashJoin LeftSemi BuildRight (55) : : : :- * Filter (3) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (56) - : : : +- * Project (55) - : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : +- BroadcastExchange (54) + : : : +- * Project (53) + : : : +- * BroadcastHashJoin Inner BuildRight (52) : : : :- * Filter (6) : : : : +- * ColumnarToRow (5) : : : : +- Scan parquet default.item (4) - : : : +- BroadcastExchange (53) - : : : +- * HashAggregate (52) - : : : +- * HashAggregate (51) - : : : +- * HashAggregate (50) - : : : +- Exchange (49) - : : : +- * HashAggregate (48) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (36) - : : : : :- * Project (22) - : : : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : : : :- * Project (15) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : :- * Filter (9) - : : : : : : : +- * ColumnarToRow (8) - : : : : : : : +- Scan parquet default.store_sales (7) - : : : : : : +- BroadcastExchange (13) - : : : : : : +- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet default.item (10) - : : : : : +- BroadcastExchange (20) - : : : : : +- * Project (19) - : : : : : +- * Filter (18) - : : : : : +- * ColumnarToRow (17) - : : : : : +- Scan parquet default.date_dim (16) - : : : : +- BroadcastExchange (35) - : : : : +- * Project (34) - : : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : : :- * Project (31) - : : : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : : : :- * Filter (25) - : : : : : : +- * ColumnarToRow (24) - : : : : : : +- Scan parquet default.catalog_sales (23) - : : : : : +- BroadcastExchange (29) - : : : : : +- * Filter (28) - : : : : : +- * ColumnarToRow (27) - : : : : : +- Scan parquet default.item (26) - : : : : +- ReusedExchange (32) - : : : +- BroadcastExchange (46) - : : : +- * Project (45) - : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : :- * Project (42) - : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : :- * Filter (39) - : : : : : +- * ColumnarToRow (38) - : : : : : +- Scan parquet default.web_sales (37) - : : : : +- ReusedExchange (40) - : : : +- ReusedExchange (43) - : : +- BroadcastExchange (63) - : : +- * BroadcastHashJoin LeftSemi BuildRight (62) - : : :- * Filter (60) - : : : +- * ColumnarToRow (59) - : : : +- Scan parquet default.item (58) - : : +- ReusedExchange (61) - : +- BroadcastExchange (70) - : +- * Project (69) - : +- * Filter (68) - : +- * ColumnarToRow (67) - : +- Scan parquet default.date_dim (66) - +- BroadcastExchange (98) - +- * Project (97) - +- * Filter (96) - +- * HashAggregate (95) - +- Exchange (94) - +- * HashAggregate (93) - +- * Project (92) - +- * BroadcastHashJoin Inner BuildRight (91) - :- * Project (85) - : +- * BroadcastHashJoin Inner BuildRight (84) - : :- * BroadcastHashJoin LeftSemi BuildRight (82) - : : :- * Filter (80) - : : : +- * ColumnarToRow (79) - : : : +- Scan parquet default.store_sales (78) - : : +- ReusedExchange (81) - : +- ReusedExchange (83) - +- BroadcastExchange (90) - +- * Project (89) - +- * Filter (88) - +- * ColumnarToRow (87) - +- Scan parquet default.date_dim (86) + : : : +- BroadcastExchange (51) + : : : +- * HashAggregate (50) + : : : +- Exchange (49) + : : : +- * HashAggregate (48) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (36) + : : : : :- * Project (22) + : : : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : : : :- * Project (15) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : : :- * Filter (9) + : : : : : : : +- * ColumnarToRow (8) + : : : : : : : +- Scan parquet default.store_sales (7) + : : : : : : +- BroadcastExchange (13) + : : : : : : +- * Filter (12) + : : : : : : +- * ColumnarToRow (11) + : : : : : : +- Scan parquet default.item (10) + : : : : : +- BroadcastExchange (20) + : : : : : +- * Project (19) + : : : : : +- * Filter (18) + : : : : : +- * ColumnarToRow (17) + : : : : : +- Scan parquet default.date_dim (16) + : : : : +- BroadcastExchange (35) + : : : : +- * Project (34) + : : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : : :- * Project (31) + : : : : : +- * BroadcastHashJoin Inner BuildRight (30) + : : : : : :- * Filter (25) + : : : : : : +- * ColumnarToRow (24) + : : : : : : +- Scan parquet default.catalog_sales (23) + : : : : : +- BroadcastExchange (29) + : : : : : +- * Filter (28) + : : : : : +- * ColumnarToRow (27) + : : : : : +- Scan parquet default.item (26) + : : : : +- ReusedExchange (32) + : : : +- BroadcastExchange (46) + : : : +- * Project (45) + : : : +- * BroadcastHashJoin Inner BuildRight (44) + : : : :- * Project (42) + : : : : +- * BroadcastHashJoin Inner BuildRight (41) + : : : : :- * Filter (39) + : : : : : +- * ColumnarToRow (38) + : : : : : +- Scan parquet default.web_sales (37) + : : : : +- ReusedExchange (40) + : : : +- ReusedExchange (43) + : : +- BroadcastExchange (61) + : : +- * BroadcastHashJoin LeftSemi BuildRight (60) + : : :- * Filter (58) + : : : +- * ColumnarToRow (57) + : : : +- Scan parquet default.item (56) + : : +- ReusedExchange (59) + : +- BroadcastExchange (68) + : +- * Project (67) + : +- * Filter (66) + : +- * ColumnarToRow (65) + : +- Scan parquet default.date_dim (64) + +- BroadcastExchange (96) + +- * Project (95) + +- * Filter (94) + +- * HashAggregate (93) + +- Exchange (92) + +- * HashAggregate (91) + +- * Project (90) + +- * BroadcastHashJoin Inner BuildRight (89) + :- * Project (83) + : +- * BroadcastHashJoin Inner BuildRight (82) + : :- * BroadcastHashJoin LeftSemi BuildRight (80) + : : :- * Filter (78) + : : : +- * ColumnarToRow (77) + : : : +- Scan parquet default.store_sales (76) + : : +- ReusedExchange (79) + : +- ReusedExchange (81) + +- BroadcastExchange (88) + +- * Project (87) + +- * Filter (86) + +- * ColumnarToRow (85) + +- Scan parquet default.date_dim (84) (1) Scan parquet default.store_sales @@ -328,435 +326,421 @@ Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#13, class_id#14, category_id#15] -(51) HashAggregate [codegen id : 10] -Input [3]: [brand_id#13, class_id#14, category_id#15] -Keys [3]: [brand_id#13, class_id#14, category_id#15] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#13, class_id#14, category_id#15] - -(52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#13, class_id#14, category_id#15] -Keys [3]: [brand_id#13, class_id#14, category_id#15] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#13, class_id#14, category_id#15] - -(53) BroadcastExchange +(51) BroadcastExchange Input [3]: [brand_id#13, class_id#14, category_id#15] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#24] -(54) BroadcastHashJoin [codegen id : 11] +(52) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Right keys [3]: [brand_id#13, class_id#14, category_id#15] Join condition: None -(55) Project [codegen id : 11] +(53) Project [codegen id : 11] Output [1]: [i_item_sk#5 AS ss_item_sk#25] Input [7]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8, brand_id#13, class_id#14, category_id#15] -(56) BroadcastExchange +(54) BroadcastExchange Input [1]: [ss_item_sk#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] -(57) BroadcastHashJoin [codegen id : 25] +(55) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#2] Right keys [1]: [ss_item_sk#25] Join condition: None -(58) Scan parquet default.item +(56) Scan parquet default.item Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(59) ColumnarToRow [codegen id : 23] +(57) ColumnarToRow [codegen id : 23] Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(60) Filter [codegen id : 23] +(58) Filter [codegen id : 23] Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Condition : (((isnotnull(i_item_sk#5) AND isnotnull(i_brand_id#6)) AND isnotnull(i_class_id#7)) AND isnotnull(i_category_id#8)) -(61) ReusedExchange [Reuses operator id: 56] +(59) ReusedExchange [Reuses operator id: 54] Output [1]: [ss_item_sk#25] -(62) BroadcastHashJoin [codegen id : 23] +(60) BroadcastHashJoin [codegen id : 23] Left keys [1]: [i_item_sk#5] Right keys [1]: [ss_item_sk#25] Join condition: None -(63) BroadcastExchange +(61) BroadcastExchange Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] -(64) BroadcastHashJoin [codegen id : 25] +(62) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#5] Join condition: None -(65) Project [codegen id : 25] +(63) Project [codegen id : 25] Output [6]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8] Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(66) Scan parquet default.date_dim +(64) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_week_seq#28] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(67) ColumnarToRow [codegen id : 24] +(65) ColumnarToRow [codegen id : 24] Input [2]: [d_date_sk#10, d_week_seq#28] -(68) Filter [codegen id : 24] +(66) Filter [codegen id : 24] Input [2]: [d_date_sk#10, d_week_seq#28] Condition : ((isnotnull(d_week_seq#28) AND (d_week_seq#28 = Subquery scalar-subquery#29, [id=#30])) AND isnotnull(d_date_sk#10)) -(69) Project [codegen id : 24] +(67) Project [codegen id : 24] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_week_seq#28] -(70) BroadcastExchange +(68) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] -(71) BroadcastHashJoin [codegen id : 25] +(69) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(72) Project [codegen id : 25] +(70) Project [codegen id : 25] Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8] Input [7]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8, d_date_sk#10] -(73) HashAggregate [codegen id : 25] +(71) HashAggregate [codegen id : 25] Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#32, isEmpty#33, count#34] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#35, isEmpty#36, count#37] -(74) Exchange +(72) Exchange Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#35, isEmpty#36, count#37] Arguments: hashpartitioning(i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#38] -(75) HashAggregate [codegen id : 52] +(73) HashAggregate [codegen id : 52] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#35, isEmpty#36, count#37] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#39, count(1)#40] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sales#41, count(1)#40 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43] -(76) Filter [codegen id : 52] +(74) Filter [codegen id : 52] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43 as decimal(32,6)) > cast(Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) -(77) Project [codegen id : 52] +(75) Project [codegen id : 52] Output [6]: [store AS channel#46, i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#43] -(78) Scan parquet default.store_sales +(76) Scan parquet default.store_sales Output [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(79) ColumnarToRow [codegen id : 50] +(77) ColumnarToRow [codegen id : 50] Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] -(80) Filter [codegen id : 50] +(78) Filter [codegen id : 50] Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4] Condition : (isnotnull(ss_item_sk#2) AND isnotnull(ss_sold_date_sk#1)) -(81) ReusedExchange [Reuses operator id: 56] +(79) ReusedExchange [Reuses operator id: 54] Output [1]: [ss_item_sk#25] -(82) BroadcastHashJoin [codegen id : 50] +(80) BroadcastHashJoin [codegen id : 50] Left keys [1]: [ss_item_sk#2] Right keys [1]: [ss_item_sk#25] Join condition: None -(83) ReusedExchange [Reuses operator id: 63] +(81) ReusedExchange [Reuses operator id: 61] Output [4]: [i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] -(84) BroadcastHashJoin [codegen id : 50] +(82) BroadcastHashJoin [codegen id : 50] Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#47] Join condition: None -(85) Project [codegen id : 50] +(83) Project [codegen id : 50] Output [6]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, i_brand_id#48, i_class_id#49, i_category_id#50] Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#47, i_brand_id#48, i_class_id#49, i_category_id#50] -(86) Scan parquet default.date_dim +(84) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_week_seq#28] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(87) ColumnarToRow [codegen id : 49] +(85) ColumnarToRow [codegen id : 49] Input [2]: [d_date_sk#10, d_week_seq#28] -(88) Filter [codegen id : 49] +(86) Filter [codegen id : 49] Input [2]: [d_date_sk#10, d_week_seq#28] Condition : ((isnotnull(d_week_seq#28) AND (d_week_seq#28 = Subquery scalar-subquery#51, [id=#52])) AND isnotnull(d_date_sk#10)) -(89) Project [codegen id : 49] +(87) Project [codegen id : 49] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_week_seq#28] -(90) BroadcastExchange +(88) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] -(91) BroadcastHashJoin [codegen id : 50] +(89) BroadcastHashJoin [codegen id : 50] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(92) Project [codegen id : 50] +(90) Project [codegen id : 50] Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#48, i_class_id#49, i_category_id#50] Input [7]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, i_brand_id#48, i_class_id#49, i_category_id#50, d_date_sk#10] -(93) HashAggregate [codegen id : 50] +(91) HashAggregate [codegen id : 50] Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#48, i_class_id#49, i_category_id#50] Keys [3]: [i_brand_id#48, i_class_id#49, i_category_id#50] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#54, isEmpty#55, count#56] Results [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum#57, isEmpty#58, count#59] -(94) Exchange +(92) Exchange Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum#57, isEmpty#58, count#59] Arguments: hashpartitioning(i_brand_id#48, i_class_id#49, i_category_id#50, 5), true, [id=#60] -(95) HashAggregate [codegen id : 51] +(93) HashAggregate [codegen id : 51] Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum#57, isEmpty#58, count#59] Keys [3]: [i_brand_id#48, i_class_id#49, i_category_id#50] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#61, count(1)#62] Results [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sales#63, count(1)#62 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65] -(96) Filter [codegen id : 51] +(94) Filter [codegen id : 51] Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) -(97) Project [codegen id : 51] +(95) Project [codegen id : 51] Output [6]: [store AS channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64] Input [6]: [i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#65] -(98) BroadcastExchange +(96) BroadcastExchange Input [6]: [channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#67] -(99) BroadcastHashJoin [codegen id : 52] +(97) BroadcastHashJoin [codegen id : 52] Left keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Right keys [3]: [i_brand_id#48, i_class_id#49, i_category_id#50] Join condition: None -(100) TakeOrderedAndProject +(98) TakeOrderedAndProject Input [12]: [channel#46, i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64] Arguments: 100, [i_brand_id#6 ASC NULLS FIRST, i_class_id#7 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST], [channel#46, i_brand_id#6, i_class_id#7, i_category_id#8, sales#41, number_sales#42, channel#66, i_brand_id#48, i_class_id#49, i_category_id#50, sales#63, number_sales#64] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#44, [id=#45] -* HashAggregate (126) -+- Exchange (125) - +- * HashAggregate (124) - +- Union (123) - :- * Project (110) - : +- * BroadcastHashJoin Inner BuildRight (109) - : :- * Filter (103) - : : +- * ColumnarToRow (102) - : : +- Scan parquet default.store_sales (101) - : +- BroadcastExchange (108) - : +- * Project (107) - : +- * Filter (106) - : +- * ColumnarToRow (105) - : +- Scan parquet default.date_dim (104) - :- * Project (116) - : +- * BroadcastHashJoin Inner BuildRight (115) - : :- * Filter (113) - : : +- * ColumnarToRow (112) - : : +- Scan parquet default.catalog_sales (111) - : +- ReusedExchange (114) - +- * Project (122) - +- * BroadcastHashJoin Inner BuildRight (121) - :- * Filter (119) - : +- * ColumnarToRow (118) - : +- Scan parquet default.web_sales (117) - +- ReusedExchange (120) - - -(101) Scan parquet default.store_sales +Subquery:1 Hosting operator id = 74 Hosting Expression = Subquery scalar-subquery#44, [id=#45] +* HashAggregate (124) ++- Exchange (123) + +- * HashAggregate (122) + +- Union (121) + :- * Project (108) + : +- * BroadcastHashJoin Inner BuildRight (107) + : :- * Filter (101) + : : +- * ColumnarToRow (100) + : : +- Scan parquet default.store_sales (99) + : +- BroadcastExchange (106) + : +- * Project (105) + : +- * Filter (104) + : +- * ColumnarToRow (103) + : +- Scan parquet default.date_dim (102) + :- * Project (114) + : +- * BroadcastHashJoin Inner BuildRight (113) + : :- * Filter (111) + : : +- * ColumnarToRow (110) + : : +- Scan parquet default.catalog_sales (109) + : +- ReusedExchange (112) + +- * Project (120) + +- * BroadcastHashJoin Inner BuildRight (119) + :- * Filter (117) + : +- * ColumnarToRow (116) + : +- Scan parquet default.web_sales (115) + +- ReusedExchange (118) + + +(99) Scan parquet default.store_sales Output [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(102) ColumnarToRow [codegen id : 2] +(100) ColumnarToRow [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] -(103) Filter [codegen id : 2] +(101) Filter [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Condition : isnotnull(ss_sold_date_sk#1) -(104) Scan parquet default.date_dim +(102) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_year#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(105) ColumnarToRow [codegen id : 1] +(103) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] -(106) Filter [codegen id : 1] +(104) Filter [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] Condition : (((isnotnull(d_year#11) AND (d_year#11 >= 1998)) AND (d_year#11 <= 2000)) AND isnotnull(d_date_sk#10)) -(107) Project [codegen id : 1] +(105) Project [codegen id : 1] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_year#11] -(108) BroadcastExchange +(106) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#68] -(109) BroadcastHashJoin [codegen id : 2] +(107) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(110) Project [codegen id : 2] +(108) Project [codegen id : 2] Output [2]: [ss_quantity#3 AS quantity#69, ss_list_price#4 AS list_price#70] Input [4]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(111) Scan parquet default.catalog_sales +(109) Scan parquet default.catalog_sales Output [3]: [cs_sold_date_sk#16, cs_quantity#71, cs_list_price#72] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(112) ColumnarToRow [codegen id : 4] +(110) ColumnarToRow [codegen id : 4] Input [3]: [cs_sold_date_sk#16, cs_quantity#71, cs_list_price#72] -(113) Filter [codegen id : 4] +(111) Filter [codegen id : 4] Input [3]: [cs_sold_date_sk#16, cs_quantity#71, cs_list_price#72] Condition : isnotnull(cs_sold_date_sk#16) -(114) ReusedExchange [Reuses operator id: 108] +(112) ReusedExchange [Reuses operator id: 106] Output [1]: [d_date_sk#10] -(115) BroadcastHashJoin [codegen id : 4] +(113) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#16] Right keys [1]: [d_date_sk#10] Join condition: None -(116) Project [codegen id : 4] +(114) Project [codegen id : 4] Output [2]: [cs_quantity#71 AS quantity#73, cs_list_price#72 AS list_price#74] Input [4]: [cs_sold_date_sk#16, cs_quantity#71, cs_list_price#72, d_date_sk#10] -(117) Scan parquet default.web_sales +(115) Scan parquet default.web_sales Output [3]: [ws_sold_date_sk#20, ws_quantity#75, ws_list_price#76] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(118) ColumnarToRow [codegen id : 6] +(116) ColumnarToRow [codegen id : 6] Input [3]: [ws_sold_date_sk#20, ws_quantity#75, ws_list_price#76] -(119) Filter [codegen id : 6] +(117) Filter [codegen id : 6] Input [3]: [ws_sold_date_sk#20, ws_quantity#75, ws_list_price#76] Condition : isnotnull(ws_sold_date_sk#20) -(120) ReusedExchange [Reuses operator id: 108] +(118) ReusedExchange [Reuses operator id: 106] Output [1]: [d_date_sk#10] -(121) BroadcastHashJoin [codegen id : 6] +(119) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#20] Right keys [1]: [d_date_sk#10] Join condition: None -(122) Project [codegen id : 6] +(120) Project [codegen id : 6] Output [2]: [ws_quantity#75 AS quantity#77, ws_list_price#76 AS list_price#78] Input [4]: [ws_sold_date_sk#20, ws_quantity#75, ws_list_price#76, d_date_sk#10] -(123) Union +(121) Union -(124) HashAggregate [codegen id : 7] +(122) HashAggregate [codegen id : 7] Input [2]: [quantity#69, list_price#70] Keys: [] Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#70 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [2]: [sum#79, count#80] Results [2]: [sum#81, count#82] -(125) Exchange +(123) Exchange Input [2]: [sum#81, count#82] Arguments: SinglePartition, true, [id=#83] -(126) HashAggregate [codegen id : 8] +(124) HashAggregate [codegen id : 8] Input [2]: [sum#81, count#82] Keys: [] Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#70 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#84] Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#84 AS average_sales#85] -Subquery:2 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#29, [id=#30] -* Project (130) -+- * Filter (129) - +- * ColumnarToRow (128) - +- Scan parquet default.date_dim (127) +Subquery:2 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#29, [id=#30] +* Project (128) ++- * Filter (127) + +- * ColumnarToRow (126) + +- Scan parquet default.date_dim (125) -(127) Scan parquet default.date_dim +(125) Scan parquet default.date_dim Output [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(128) ColumnarToRow [codegen id : 1] +(126) ColumnarToRow [codegen id : 1] Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] -(129) Filter [codegen id : 1] +(127) Filter [codegen id : 1] Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#86)) AND isnotnull(d_dom#87)) AND (d_year#11 = 1999)) AND (d_moy#86 = 12)) AND (d_dom#87 = 16)) -(130) Project [codegen id : 1] +(128) Project [codegen id : 1] Output [1]: [d_week_seq#28] Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] -Subquery:3 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:3 Hosting operator id = 94 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -Subquery:4 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#51, [id=#52] -* Project (134) -+- * Filter (133) - +- * ColumnarToRow (132) - +- Scan parquet default.date_dim (131) +Subquery:4 Hosting operator id = 86 Hosting Expression = Subquery scalar-subquery#51, [id=#52] +* Project (132) ++- * Filter (131) + +- * ColumnarToRow (130) + +- Scan parquet default.date_dim (129) -(131) Scan parquet default.date_dim +(129) Scan parquet default.date_dim Output [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(132) ColumnarToRow [codegen id : 1] +(130) ColumnarToRow [codegen id : 1] Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] -(133) Filter [codegen id : 1] +(131) Filter [codegen id : 1] Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#86)) AND isnotnull(d_dom#87)) AND (d_year#11 = 1998)) AND (d_moy#86 = 12)) AND (d_dom#87 = 16)) -(134) Project [codegen id : 1] +(132) Project [codegen id : 1] Output [1]: [d_week_seq#28] Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt index 7bbf83e3de70..e58043e1de3a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt @@ -72,72 +72,70 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastExchange #3 WholeStageCodegen (10) HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #4 - WholeStageCodegen (9) - HashAggregate [brand_id,class_id,category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] + InputAdapter + Exchange [brand_id,class_id,category_id] #4 + WholeStageCodegen (9) + HashAggregate [brand_id,class_id,category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk,ss_sold_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] + BroadcastExchange #5 + WholeStageCodegen (1) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] + BroadcastExchange #6 + WholeStageCodegen (2) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (5) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk,cs_sold_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (8) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk,ws_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] + BroadcastExchange #8 + WholeStageCodegen (3) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (8) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk,ws_sold_date_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8 + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] InputAdapter - ReusedExchange [d_date_sk] #6 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8 + InputAdapter + ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #10 WholeStageCodegen (23) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt index 38292528b42f..416836e8f5d7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt @@ -1,226 +1,224 @@ == Physical Plan == -TakeOrderedAndProject (222) -+- * HashAggregate (221) - +- Exchange (220) - +- * HashAggregate (219) - +- Union (218) - :- * HashAggregate (198) - : +- Exchange (197) - : +- * HashAggregate (196) - : +- Union (195) - : :- * HashAggregate (175) - : : +- Exchange (174) - : : +- * HashAggregate (173) - : : +- Union (172) - : : :- * HashAggregate (152) - : : : +- Exchange (151) - : : : +- * HashAggregate (150) - : : : +- Union (149) - : : : :- * HashAggregate (129) - : : : : +- Exchange (128) - : : : : +- * HashAggregate (127) - : : : : +- Union (126) - : : : : :- * Project (87) - : : : : : +- * Filter (86) - : : : : : +- * HashAggregate (85) - : : : : : +- Exchange (84) - : : : : : +- * HashAggregate (83) - : : : : : +- * Project (82) - : : : : : +- * BroadcastHashJoin Inner BuildRight (81) - : : : : : :- * Project (71) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (70) - : : : : : : :- SortMergeJoin LeftSemi (64) +TakeOrderedAndProject (220) ++- * HashAggregate (219) + +- Exchange (218) + +- * HashAggregate (217) + +- Union (216) + :- * HashAggregate (196) + : +- Exchange (195) + : +- * HashAggregate (194) + : +- Union (193) + : :- * HashAggregate (173) + : : +- Exchange (172) + : : +- * HashAggregate (171) + : : +- Union (170) + : : :- * HashAggregate (150) + : : : +- Exchange (149) + : : : +- * HashAggregate (148) + : : : +- Union (147) + : : : :- * HashAggregate (127) + : : : : +- Exchange (126) + : : : : +- * HashAggregate (125) + : : : : +- Union (124) + : : : : :- * Project (85) + : : : : : +- * Filter (84) + : : : : : +- * HashAggregate (83) + : : : : : +- Exchange (82) + : : : : : +- * HashAggregate (81) + : : : : : +- * Project (80) + : : : : : +- * BroadcastHashJoin Inner BuildRight (79) + : : : : : :- * Project (69) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (68) + : : : : : : :- SortMergeJoin LeftSemi (62) : : : : : : : :- * Sort (5) : : : : : : : : +- Exchange (4) : : : : : : : : +- * Filter (3) : : : : : : : : +- * ColumnarToRow (2) : : : : : : : : +- Scan parquet default.store_sales (1) - : : : : : : : +- * Sort (63) - : : : : : : : +- Exchange (62) - : : : : : : : +- * Project (61) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : : : : : +- * Sort (61) + : : : : : : : +- Exchange (60) + : : : : : : : +- * Project (59) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (58) : : : : : : : :- * Filter (8) : : : : : : : : +- * ColumnarToRow (7) : : : : : : : : +- Scan parquet default.item (6) - : : : : : : : +- BroadcastExchange (59) - : : : : : : : +- * HashAggregate (58) - : : : : : : : +- * HashAggregate (57) - : : : : : : : +- * HashAggregate (56) - : : : : : : : +- Exchange (55) - : : : : : : : +- * HashAggregate (54) - : : : : : : : +- SortMergeJoin LeftSemi (53) - : : : : : : : :- SortMergeJoin LeftSemi (41) - : : : : : : : : :- * Sort (26) - : : : : : : : : : +- Exchange (25) - : : : : : : : : : +- * Project (24) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : : : : : :- * Project (18) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : : : : : : :- * Filter (11) - : : : : : : : : : : : +- * ColumnarToRow (10) - : : : : : : : : : : : +- Scan parquet default.store_sales (9) - : : : : : : : : : : +- BroadcastExchange (16) - : : : : : : : : : : +- * Project (15) - : : : : : : : : : : +- * Filter (14) - : : : : : : : : : : +- * ColumnarToRow (13) - : : : : : : : : : : +- Scan parquet default.date_dim (12) - : : : : : : : : : +- BroadcastExchange (22) - : : : : : : : : : +- * Filter (21) - : : : : : : : : : +- * ColumnarToRow (20) - : : : : : : : : : +- Scan parquet default.item (19) - : : : : : : : : +- * Sort (40) - : : : : : : : : +- Exchange (39) - : : : : : : : : +- * Project (38) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : : : : : :- * Project (32) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : : : : : :- * Filter (29) - : : : : : : : : : : +- * ColumnarToRow (28) - : : : : : : : : : : +- Scan parquet default.catalog_sales (27) - : : : : : : : : : +- ReusedExchange (30) - : : : : : : : : +- BroadcastExchange (36) - : : : : : : : : +- * Filter (35) - : : : : : : : : +- * ColumnarToRow (34) - : : : : : : : : +- Scan parquet default.item (33) - : : : : : : : +- * Sort (52) - : : : : : : : +- Exchange (51) - : : : : : : : +- * Project (50) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : : : : : :- * Project (47) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (46) - : : : : : : : : :- * Filter (44) - : : : : : : : : : +- * ColumnarToRow (43) - : : : : : : : : : +- Scan parquet default.web_sales (42) - : : : : : : : : +- ReusedExchange (45) - : : : : : : : +- ReusedExchange (48) - : : : : : : +- BroadcastExchange (69) - : : : : : : +- * Project (68) - : : : : : : +- * Filter (67) - : : : : : : +- * ColumnarToRow (66) - : : : : : : +- Scan parquet default.date_dim (65) - : : : : : +- BroadcastExchange (80) - : : : : : +- SortMergeJoin LeftSemi (79) - : : : : : :- * Sort (76) - : : : : : : +- Exchange (75) - : : : : : : +- * Filter (74) - : : : : : : +- * ColumnarToRow (73) - : : : : : : +- Scan parquet default.item (72) - : : : : : +- * Sort (78) - : : : : : +- ReusedExchange (77) - : : : : :- * Project (106) - : : : : : +- * Filter (105) - : : : : : +- * HashAggregate (104) - : : : : : +- Exchange (103) - : : : : : +- * HashAggregate (102) - : : : : : +- * Project (101) - : : : : : +- * BroadcastHashJoin Inner BuildRight (100) - : : : : : :- * Project (98) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (97) - : : : : : : :- SortMergeJoin LeftSemi (95) - : : : : : : : :- * Sort (92) - : : : : : : : : +- Exchange (91) - : : : : : : : : +- * Filter (90) - : : : : : : : : +- * ColumnarToRow (89) - : : : : : : : : +- Scan parquet default.catalog_sales (88) - : : : : : : : +- * Sort (94) - : : : : : : : +- ReusedExchange (93) - : : : : : : +- ReusedExchange (96) - : : : : : +- ReusedExchange (99) - : : : : +- * Project (125) - : : : : +- * Filter (124) - : : : : +- * HashAggregate (123) - : : : : +- Exchange (122) - : : : : +- * HashAggregate (121) - : : : : +- * Project (120) - : : : : +- * BroadcastHashJoin Inner BuildRight (119) - : : : : :- * Project (117) - : : : : : +- * BroadcastHashJoin Inner BuildRight (116) - : : : : : :- SortMergeJoin LeftSemi (114) - : : : : : : :- * Sort (111) - : : : : : : : +- Exchange (110) - : : : : : : : +- * Filter (109) - : : : : : : : +- * ColumnarToRow (108) - : : : : : : : +- Scan parquet default.web_sales (107) - : : : : : : +- * Sort (113) - : : : : : : +- ReusedExchange (112) - : : : : : +- ReusedExchange (115) - : : : : +- ReusedExchange (118) - : : : +- * HashAggregate (148) - : : : +- Exchange (147) - : : : +- * HashAggregate (146) - : : : +- * HashAggregate (145) - : : : +- Exchange (144) - : : : +- * HashAggregate (143) - : : : +- Union (142) - : : : :- * Project (133) - : : : : +- * Filter (132) - : : : : +- * HashAggregate (131) - : : : : +- ReusedExchange (130) - : : : :- * Project (137) - : : : : +- * Filter (136) - : : : : +- * HashAggregate (135) - : : : : +- ReusedExchange (134) - : : : +- * Project (141) - : : : +- * Filter (140) - : : : +- * HashAggregate (139) - : : : +- ReusedExchange (138) - : : +- * HashAggregate (171) - : : +- Exchange (170) - : : +- * HashAggregate (169) - : : +- * HashAggregate (168) - : : +- Exchange (167) - : : +- * HashAggregate (166) - : : +- Union (165) - : : :- * Project (156) - : : : +- * Filter (155) - : : : +- * HashAggregate (154) - : : : +- ReusedExchange (153) - : : :- * Project (160) - : : : +- * Filter (159) - : : : +- * HashAggregate (158) - : : : +- ReusedExchange (157) - : : +- * Project (164) - : : +- * Filter (163) - : : +- * HashAggregate (162) - : : +- ReusedExchange (161) - : +- * HashAggregate (194) - : +- Exchange (193) - : +- * HashAggregate (192) - : +- * HashAggregate (191) - : +- Exchange (190) - : +- * HashAggregate (189) - : +- Union (188) - : :- * Project (179) - : : +- * Filter (178) - : : +- * HashAggregate (177) - : : +- ReusedExchange (176) - : :- * Project (183) - : : +- * Filter (182) - : : +- * HashAggregate (181) - : : +- ReusedExchange (180) - : +- * Project (187) - : +- * Filter (186) - : +- * HashAggregate (185) - : +- ReusedExchange (184) - +- * HashAggregate (217) - +- Exchange (216) - +- * HashAggregate (215) - +- * HashAggregate (214) - +- Exchange (213) - +- * HashAggregate (212) - +- Union (211) - :- * Project (202) - : +- * Filter (201) - : +- * HashAggregate (200) - : +- ReusedExchange (199) - :- * Project (206) - : +- * Filter (205) - : +- * HashAggregate (204) - : +- ReusedExchange (203) - +- * Project (210) - +- * Filter (209) - +- * HashAggregate (208) - +- ReusedExchange (207) + : : : : : : : +- BroadcastExchange (57) + : : : : : : : +- * HashAggregate (56) + : : : : : : : +- Exchange (55) + : : : : : : : +- * HashAggregate (54) + : : : : : : : +- SortMergeJoin LeftSemi (53) + : : : : : : : :- SortMergeJoin LeftSemi (41) + : : : : : : : : :- * Sort (26) + : : : : : : : : : +- Exchange (25) + : : : : : : : : : +- * Project (24) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : : : : : :- * Project (18) + : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : : : : : : :- * Filter (11) + : : : : : : : : : : : +- * ColumnarToRow (10) + : : : : : : : : : : : +- Scan parquet default.store_sales (9) + : : : : : : : : : : +- BroadcastExchange (16) + : : : : : : : : : : +- * Project (15) + : : : : : : : : : : +- * Filter (14) + : : : : : : : : : : +- * ColumnarToRow (13) + : : : : : : : : : : +- Scan parquet default.date_dim (12) + : : : : : : : : : +- BroadcastExchange (22) + : : : : : : : : : +- * Filter (21) + : : : : : : : : : +- * ColumnarToRow (20) + : : : : : : : : : +- Scan parquet default.item (19) + : : : : : : : : +- * Sort (40) + : : : : : : : : +- Exchange (39) + : : : : : : : : +- * Project (38) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : : : : : : :- * Project (32) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : : : : : : :- * Filter (29) + : : : : : : : : : : +- * ColumnarToRow (28) + : : : : : : : : : : +- Scan parquet default.catalog_sales (27) + : : : : : : : : : +- ReusedExchange (30) + : : : : : : : : +- BroadcastExchange (36) + : : : : : : : : +- * Filter (35) + : : : : : : : : +- * ColumnarToRow (34) + : : : : : : : : +- Scan parquet default.item (33) + : : : : : : : +- * Sort (52) + : : : : : : : +- Exchange (51) + : : : : : : : +- * Project (50) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (49) + : : : : : : : :- * Project (47) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (46) + : : : : : : : : :- * Filter (44) + : : : : : : : : : +- * ColumnarToRow (43) + : : : : : : : : : +- Scan parquet default.web_sales (42) + : : : : : : : : +- ReusedExchange (45) + : : : : : : : +- ReusedExchange (48) + : : : : : : +- BroadcastExchange (67) + : : : : : : +- * Project (66) + : : : : : : +- * Filter (65) + : : : : : : +- * ColumnarToRow (64) + : : : : : : +- Scan parquet default.date_dim (63) + : : : : : +- BroadcastExchange (78) + : : : : : +- SortMergeJoin LeftSemi (77) + : : : : : :- * Sort (74) + : : : : : : +- Exchange (73) + : : : : : : +- * Filter (72) + : : : : : : +- * ColumnarToRow (71) + : : : : : : +- Scan parquet default.item (70) + : : : : : +- * Sort (76) + : : : : : +- ReusedExchange (75) + : : : : :- * Project (104) + : : : : : +- * Filter (103) + : : : : : +- * HashAggregate (102) + : : : : : +- Exchange (101) + : : : : : +- * HashAggregate (100) + : : : : : +- * Project (99) + : : : : : +- * BroadcastHashJoin Inner BuildRight (98) + : : : : : :- * Project (96) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (95) + : : : : : : :- SortMergeJoin LeftSemi (93) + : : : : : : : :- * Sort (90) + : : : : : : : : +- Exchange (89) + : : : : : : : : +- * Filter (88) + : : : : : : : : +- * ColumnarToRow (87) + : : : : : : : : +- Scan parquet default.catalog_sales (86) + : : : : : : : +- * Sort (92) + : : : : : : : +- ReusedExchange (91) + : : : : : : +- ReusedExchange (94) + : : : : : +- ReusedExchange (97) + : : : : +- * Project (123) + : : : : +- * Filter (122) + : : : : +- * HashAggregate (121) + : : : : +- Exchange (120) + : : : : +- * HashAggregate (119) + : : : : +- * Project (118) + : : : : +- * BroadcastHashJoin Inner BuildRight (117) + : : : : :- * Project (115) + : : : : : +- * BroadcastHashJoin Inner BuildRight (114) + : : : : : :- SortMergeJoin LeftSemi (112) + : : : : : : :- * Sort (109) + : : : : : : : +- Exchange (108) + : : : : : : : +- * Filter (107) + : : : : : : : +- * ColumnarToRow (106) + : : : : : : : +- Scan parquet default.web_sales (105) + : : : : : : +- * Sort (111) + : : : : : : +- ReusedExchange (110) + : : : : : +- ReusedExchange (113) + : : : : +- ReusedExchange (116) + : : : +- * HashAggregate (146) + : : : +- Exchange (145) + : : : +- * HashAggregate (144) + : : : +- * HashAggregate (143) + : : : +- Exchange (142) + : : : +- * HashAggregate (141) + : : : +- Union (140) + : : : :- * Project (131) + : : : : +- * Filter (130) + : : : : +- * HashAggregate (129) + : : : : +- ReusedExchange (128) + : : : :- * Project (135) + : : : : +- * Filter (134) + : : : : +- * HashAggregate (133) + : : : : +- ReusedExchange (132) + : : : +- * Project (139) + : : : +- * Filter (138) + : : : +- * HashAggregate (137) + : : : +- ReusedExchange (136) + : : +- * HashAggregate (169) + : : +- Exchange (168) + : : +- * HashAggregate (167) + : : +- * HashAggregate (166) + : : +- Exchange (165) + : : +- * HashAggregate (164) + : : +- Union (163) + : : :- * Project (154) + : : : +- * Filter (153) + : : : +- * HashAggregate (152) + : : : +- ReusedExchange (151) + : : :- * Project (158) + : : : +- * Filter (157) + : : : +- * HashAggregate (156) + : : : +- ReusedExchange (155) + : : +- * Project (162) + : : +- * Filter (161) + : : +- * HashAggregate (160) + : : +- ReusedExchange (159) + : +- * HashAggregate (192) + : +- Exchange (191) + : +- * HashAggregate (190) + : +- * HashAggregate (189) + : +- Exchange (188) + : +- * HashAggregate (187) + : +- Union (186) + : :- * Project (177) + : : +- * Filter (176) + : : +- * HashAggregate (175) + : : +- ReusedExchange (174) + : :- * Project (181) + : : +- * Filter (180) + : : +- * HashAggregate (179) + : : +- ReusedExchange (178) + : +- * Project (185) + : +- * Filter (184) + : +- * HashAggregate (183) + : +- ReusedExchange (182) + +- * HashAggregate (215) + +- Exchange (214) + +- * HashAggregate (213) + +- * HashAggregate (212) + +- Exchange (211) + +- * HashAggregate (210) + +- Union (209) + :- * Project (200) + : +- * Filter (199) + : +- * HashAggregate (198) + : +- ReusedExchange (197) + :- * Project (204) + : +- * Filter (203) + : +- * HashAggregate (202) + : +- ReusedExchange (201) + +- * Project (208) + +- * Filter (207) + +- * HashAggregate (206) + +- ReusedExchange (205) (1) Scan parquet default.store_sales @@ -474,987 +472,973 @@ Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#14, class_id#15, category_id#16] -(57) HashAggregate [codegen id : 16] -Input [3]: [brand_id#14, class_id#15, category_id#16] -Keys [3]: [brand_id#14, class_id#15, category_id#16] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#14, class_id#15, category_id#16] - -(58) HashAggregate [codegen id : 16] -Input [3]: [brand_id#14, class_id#15, category_id#16] -Keys [3]: [brand_id#14, class_id#15, category_id#16] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#14, class_id#15, category_id#16] - -(59) BroadcastExchange +(57) BroadcastExchange Input [3]: [brand_id#14, class_id#15, category_id#16] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#26] -(60) BroadcastHashJoin [codegen id : 17] +(58) BroadcastHashJoin [codegen id : 17] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#14, class_id#15, category_id#16] Join condition: None -(61) Project [codegen id : 17] +(59) Project [codegen id : 17] Output [1]: [i_item_sk#6 AS ss_item_sk#27] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#14, class_id#15, category_id#16] -(62) Exchange +(60) Exchange Input [1]: [ss_item_sk#27] Arguments: hashpartitioning(ss_item_sk#27, 5), true, [id=#28] -(63) Sort [codegen id : 18] +(61) Sort [codegen id : 18] Input [1]: [ss_item_sk#27] Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 -(64) SortMergeJoin +(62) SortMergeJoin Left keys [1]: [ss_item_sk#2] Right keys [1]: [ss_item_sk#27] Join condition: None -(65) Scan parquet default.date_dim +(63) Scan parquet default.date_dim Output [3]: [d_date_sk#10, d_year#11, d_moy#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(66) ColumnarToRow [codegen id : 19] +(64) ColumnarToRow [codegen id : 19] Input [3]: [d_date_sk#10, d_year#11, d_moy#29] -(67) Filter [codegen id : 19] +(65) Filter [codegen id : 19] Input [3]: [d_date_sk#10, d_year#11, d_moy#29] Condition : ((((isnotnull(d_year#11) AND isnotnull(d_moy#29)) AND (d_year#11 = 2000)) AND (d_moy#29 = 11)) AND isnotnull(d_date_sk#10)) -(68) Project [codegen id : 19] +(66) Project [codegen id : 19] Output [1]: [d_date_sk#10] Input [3]: [d_date_sk#10, d_year#11, d_moy#29] -(69) BroadcastExchange +(67) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] -(70) BroadcastHashJoin [codegen id : 38] +(68) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(71) Project [codegen id : 38] +(69) Project [codegen id : 38] Output [3]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4] Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(72) Scan parquet default.item +(70) Scan parquet default.item Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(73) ColumnarToRow [codegen id : 20] +(71) ColumnarToRow [codegen id : 20] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(74) Filter [codegen id : 20] +(72) Filter [codegen id : 20] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : isnotnull(i_item_sk#6) -(75) Exchange +(73) Exchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: hashpartitioning(i_item_sk#6, 5), true, [id=#31] -(76) Sort [codegen id : 21] +(74) Sort [codegen id : 21] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: [i_item_sk#6 ASC NULLS FIRST], false, 0 -(77) ReusedExchange [Reuses operator id: 62] +(75) ReusedExchange [Reuses operator id: 60] Output [1]: [ss_item_sk#27] -(78) Sort [codegen id : 37] +(76) Sort [codegen id : 37] Input [1]: [ss_item_sk#27] Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 -(79) SortMergeJoin +(77) SortMergeJoin Left keys [1]: [i_item_sk#6] Right keys [1]: [ss_item_sk#27] Join condition: None -(80) BroadcastExchange +(78) BroadcastExchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#32] -(81) BroadcastHashJoin [codegen id : 38] +(79) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#6] Join condition: None -(82) Project [codegen id : 38] +(80) Project [codegen id : 38] Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#7, i_class_id#8, i_category_id#9] Input [7]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(83) HashAggregate [codegen id : 38] +(81) HashAggregate [codegen id : 38] Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#7, i_class_id#8, i_category_id#9] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#33, isEmpty#34, count#35] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#36, isEmpty#37, count#38] -(84) Exchange +(82) Exchange Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#36, isEmpty#37, count#38] Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#39] -(85) HashAggregate [codegen id : 39] +(83) HashAggregate [codegen id : 39] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#36, isEmpty#37, count#38] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40, count(1)#41] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40 AS sales#42, count(1)#41 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44] -(86) Filter [codegen id : 39] +(84) Filter [codegen id : 39] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44 as decimal(32,6)) > cast(Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(87) Project [codegen id : 39] +(85) Project [codegen id : 39] Output [6]: [store AS channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44] -(88) Scan parquet default.catalog_sales +(86) Scan parquet default.catalog_sales Output [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(89) ColumnarToRow [codegen id : 40] +(87) ColumnarToRow [codegen id : 40] Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] -(90) Filter [codegen id : 40] +(88) Filter [codegen id : 40] Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] Condition : (isnotnull(cs_item_sk#19) AND isnotnull(cs_sold_date_sk#18)) -(91) Exchange +(89) Exchange Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] Arguments: hashpartitioning(cs_item_sk#19, 5), true, [id=#50] -(92) Sort [codegen id : 41] +(90) Sort [codegen id : 41] Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] Arguments: [cs_item_sk#19 ASC NULLS FIRST], false, 0 -(93) ReusedExchange [Reuses operator id: 62] +(91) ReusedExchange [Reuses operator id: 60] Output [1]: [ss_item_sk#27] -(94) Sort [codegen id : 57] +(92) Sort [codegen id : 57] Input [1]: [ss_item_sk#27] Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 -(95) SortMergeJoin +(93) SortMergeJoin Left keys [1]: [cs_item_sk#19] Right keys [1]: [ss_item_sk#27] Join condition: None -(96) ReusedExchange [Reuses operator id: 69] +(94) ReusedExchange [Reuses operator id: 67] Output [1]: [d_date_sk#10] -(97) BroadcastHashJoin [codegen id : 77] +(95) BroadcastHashJoin [codegen id : 77] Left keys [1]: [cs_sold_date_sk#18] Right keys [1]: [d_date_sk#10] Join condition: None -(98) Project [codegen id : 77] +(96) Project [codegen id : 77] Output [3]: [cs_item_sk#19, cs_quantity#48, cs_list_price#49] Input [5]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49, d_date_sk#10] -(99) ReusedExchange [Reuses operator id: 80] +(97) ReusedExchange [Reuses operator id: 78] Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(100) BroadcastHashJoin [codegen id : 77] +(98) BroadcastHashJoin [codegen id : 77] Left keys [1]: [cs_item_sk#19] Right keys [1]: [i_item_sk#6] Join condition: None -(101) Project [codegen id : 77] +(99) Project [codegen id : 77] Output [5]: [cs_quantity#48, cs_list_price#49, i_brand_id#7, i_class_id#8, i_category_id#9] Input [7]: [cs_item_sk#19, cs_quantity#48, cs_list_price#49, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(102) HashAggregate [codegen id : 77] +(100) HashAggregate [codegen id : 77] Input [5]: [cs_quantity#48, cs_list_price#49, i_brand_id#7, i_class_id#8, i_category_id#9] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#51, isEmpty#52, count#53] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#54, isEmpty#55, count#56] -(103) Exchange +(101) Exchange Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#54, isEmpty#55, count#56] Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#57] -(104) HashAggregate [codegen id : 78] +(102) HashAggregate [codegen id : 78] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#54, isEmpty#55, count#56] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58, count(1)#59] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sales#60, count(1)#59 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62] -(105) Filter [codegen id : 78] +(103) Filter [codegen id : 78] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(106) Project [codegen id : 78] +(104) Project [codegen id : 78] Output [6]: [catalog AS channel#63, i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62] -(107) Scan parquet default.web_sales +(105) Scan parquet default.web_sales Output [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(108) ColumnarToRow [codegen id : 79] +(106) ColumnarToRow [codegen id : 79] Input [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] -(109) Filter [codegen id : 79] +(107) Filter [codegen id : 79] Input [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] Condition : (isnotnull(ws_item_sk#23) AND isnotnull(ws_sold_date_sk#22)) -(110) Exchange +(108) Exchange Input [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] Arguments: hashpartitioning(ws_item_sk#23, 5), true, [id=#66] -(111) Sort [codegen id : 80] +(109) Sort [codegen id : 80] Input [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] Arguments: [ws_item_sk#23 ASC NULLS FIRST], false, 0 -(112) ReusedExchange [Reuses operator id: 62] +(110) ReusedExchange [Reuses operator id: 60] Output [1]: [ss_item_sk#27] -(113) Sort [codegen id : 96] +(111) Sort [codegen id : 96] Input [1]: [ss_item_sk#27] Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 -(114) SortMergeJoin +(112) SortMergeJoin Left keys [1]: [ws_item_sk#23] Right keys [1]: [ss_item_sk#27] Join condition: None -(115) ReusedExchange [Reuses operator id: 69] +(113) ReusedExchange [Reuses operator id: 67] Output [1]: [d_date_sk#10] -(116) BroadcastHashJoin [codegen id : 116] +(114) BroadcastHashJoin [codegen id : 116] Left keys [1]: [ws_sold_date_sk#22] Right keys [1]: [d_date_sk#10] Join condition: None -(117) Project [codegen id : 116] +(115) Project [codegen id : 116] Output [3]: [ws_item_sk#23, ws_quantity#64, ws_list_price#65] Input [5]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65, d_date_sk#10] -(118) ReusedExchange [Reuses operator id: 80] +(116) ReusedExchange [Reuses operator id: 78] Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(119) BroadcastHashJoin [codegen id : 116] +(117) BroadcastHashJoin [codegen id : 116] Left keys [1]: [ws_item_sk#23] Right keys [1]: [i_item_sk#6] Join condition: None -(120) Project [codegen id : 116] +(118) Project [codegen id : 116] Output [5]: [ws_quantity#64, ws_list_price#65, i_brand_id#7, i_class_id#8, i_category_id#9] Input [7]: [ws_item_sk#23, ws_quantity#64, ws_list_price#65, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(121) HashAggregate [codegen id : 116] +(119) HashAggregate [codegen id : 116] Input [5]: [ws_quantity#64, ws_list_price#65, i_brand_id#7, i_class_id#8, i_category_id#9] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#67, isEmpty#68, count#69] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#70, isEmpty#71, count#72] -(122) Exchange +(120) Exchange Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#70, isEmpty#71, count#72] Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#73] -(123) HashAggregate [codegen id : 117] +(121) HashAggregate [codegen id : 117] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#70, isEmpty#71, count#72] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74, count(1)#75] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74 AS sales#76, count(1)#75 AS number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78] -(124) Filter [codegen id : 117] +(122) Filter [codegen id : 117] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(125) Project [codegen id : 117] +(123) Project [codegen id : 117] Output [6]: [web AS channel#79, i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78] -(126) Union +(124) Union -(127) HashAggregate [codegen id : 118] +(125) HashAggregate [codegen id : 118] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(sales#42), partial_sum(number_sales#43)] Aggregate Attributes [3]: [sum#80, isEmpty#81, sum#82] Results [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#83, isEmpty#84, sum#85] -(128) Exchange +(126) Exchange Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#83, isEmpty#84, sum#85] Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#86] -(129) HashAggregate [codegen id : 119] +(127) HashAggregate [codegen id : 119] Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#83, isEmpty#84, sum#85] Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(sales#42), sum(number_sales#43)] Aggregate Attributes [2]: [sum(sales#42)#87, sum(number_sales#43)#88] Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum(sales#42)#87 AS sum_sales#89, sum(number_sales#43)#88 AS number_sales#90] -(130) ReusedExchange [Reuses operator id: 84] +(128) ReusedExchange [Reuses operator id: 82] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#91, isEmpty#92, count#93] -(131) HashAggregate [codegen id : 158] +(129) HashAggregate [codegen id : 158] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#91, isEmpty#92, count#93] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#94, count(1)#95] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#94 AS sales#42, count(1)#95 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#94 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96] -(132) Filter [codegen id : 158] +(130) Filter [codegen id : 158] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(133) Project [codegen id : 158] +(131) Project [codegen id : 158] Output [6]: [store AS channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96] -(134) ReusedExchange [Reuses operator id: 103] +(132) ReusedExchange [Reuses operator id: 101] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#97, isEmpty#98, count#99] -(135) HashAggregate [codegen id : 197] +(133) HashAggregate [codegen id : 197] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#97, isEmpty#98, count#99] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#100, count(1)#101] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#100 AS sales#60, count(1)#101 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#100 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102] -(136) Filter [codegen id : 197] +(134) Filter [codegen id : 197] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(137) Project [codegen id : 197] +(135) Project [codegen id : 197] Output [6]: [catalog AS channel#103, i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102] -(138) ReusedExchange [Reuses operator id: 122] +(136) ReusedExchange [Reuses operator id: 120] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#104, isEmpty#105, count#106] -(139) HashAggregate [codegen id : 236] +(137) HashAggregate [codegen id : 236] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#104, isEmpty#105, count#106] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#107, count(1)#108] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sales#76, count(1)#108 AS number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#109] -(140) Filter [codegen id : 236] +(138) Filter [codegen id : 236] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#109] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#109) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#109 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(141) Project [codegen id : 236] +(139) Project [codegen id : 236] Output [6]: [web AS channel#110, i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#109] -(142) Union +(140) Union -(143) HashAggregate [codegen id : 237] +(141) HashAggregate [codegen id : 237] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(sales#42), partial_sum(number_sales#43)] Aggregate Attributes [3]: [sum#111, isEmpty#112, sum#113] Results [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#114, isEmpty#115, sum#116] -(144) Exchange +(142) Exchange Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#114, isEmpty#115, sum#116] Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#117] -(145) HashAggregate [codegen id : 238] +(143) HashAggregate [codegen id : 238] Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#114, isEmpty#115, sum#116] Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(sales#42), sum(number_sales#43)] Aggregate Attributes [2]: [sum(sales#42)#118, sum(number_sales#43)#119] Results [5]: [channel#47, i_brand_id#7, i_class_id#8, sum(sales#42)#118 AS sum_sales#89, sum(number_sales#43)#119 AS number_sales#90] -(146) HashAggregate [codegen id : 238] +(144) HashAggregate [codegen id : 238] Input [5]: [channel#47, i_brand_id#7, i_class_id#8, sum_sales#89, number_sales#90] Keys [3]: [channel#47, i_brand_id#7, i_class_id#8] Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)] Aggregate Attributes [3]: [sum#120, isEmpty#121, sum#122] Results [6]: [channel#47, i_brand_id#7, i_class_id#8, sum#123, isEmpty#124, sum#125] -(147) Exchange +(145) Exchange Input [6]: [channel#47, i_brand_id#7, i_class_id#8, sum#123, isEmpty#124, sum#125] Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, 5), true, [id=#126] -(148) HashAggregate [codegen id : 239] +(146) HashAggregate [codegen id : 239] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, sum#123, isEmpty#124, sum#125] Keys [3]: [channel#47, i_brand_id#7, i_class_id#8] Functions [2]: [sum(sum_sales#89), sum(number_sales#90)] Aggregate Attributes [2]: [sum(sum_sales#89)#127, sum(number_sales#90)#128] Results [6]: [channel#47, i_brand_id#7, i_class_id#8, null AS i_category_id#129, sum(sum_sales#89)#127 AS sum(sum_sales)#130, sum(number_sales#90)#128 AS sum(number_sales)#131] -(149) Union +(147) Union -(150) HashAggregate [codegen id : 240] +(148) HashAggregate [codegen id : 240] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Functions: [] Aggregate Attributes: [] Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] -(151) Exchange +(149) Exchange Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90, 5), true, [id=#132] -(152) HashAggregate [codegen id : 241] +(150) HashAggregate [codegen id : 241] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Functions: [] Aggregate Attributes: [] Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] -(153) ReusedExchange [Reuses operator id: 84] +(151) ReusedExchange [Reuses operator id: 82] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#133, isEmpty#134, count#135] -(154) HashAggregate [codegen id : 280] +(152) HashAggregate [codegen id : 280] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#133, isEmpty#134, count#135] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#136, count(1)#137] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#136 AS sales#42, count(1)#137 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#136 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#138] -(155) Filter [codegen id : 280] +(153) Filter [codegen id : 280] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#138] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#138) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#138 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(156) Project [codegen id : 280] +(154) Project [codegen id : 280] Output [6]: [store AS channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#138] -(157) ReusedExchange [Reuses operator id: 103] +(155) ReusedExchange [Reuses operator id: 101] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#139, isEmpty#140, count#141] -(158) HashAggregate [codegen id : 319] +(156) HashAggregate [codegen id : 319] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#139, isEmpty#140, count#141] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#142, count(1)#143] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#142 AS sales#60, count(1)#143 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#142 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#144] -(159) Filter [codegen id : 319] +(157) Filter [codegen id : 319] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#144] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#144) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#144 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(160) Project [codegen id : 319] +(158) Project [codegen id : 319] Output [6]: [catalog AS channel#145, i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#144] -(161) ReusedExchange [Reuses operator id: 122] +(159) ReusedExchange [Reuses operator id: 120] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#146, isEmpty#147, count#148] -(162) HashAggregate [codegen id : 358] +(160) HashAggregate [codegen id : 358] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#146, isEmpty#147, count#148] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#149, count(1)#150] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#149 AS sales#76, count(1)#150 AS number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#149 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#151] -(163) Filter [codegen id : 358] +(161) Filter [codegen id : 358] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#151] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#151) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#151 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(164) Project [codegen id : 358] +(162) Project [codegen id : 358] Output [6]: [web AS channel#152, i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#151] -(165) Union +(163) Union -(166) HashAggregate [codegen id : 359] +(164) HashAggregate [codegen id : 359] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(sales#42), partial_sum(number_sales#43)] Aggregate Attributes [3]: [sum#153, isEmpty#154, sum#155] Results [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#156, isEmpty#157, sum#158] -(167) Exchange +(165) Exchange Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#156, isEmpty#157, sum#158] Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#159] -(168) HashAggregate [codegen id : 360] +(166) HashAggregate [codegen id : 360] Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#156, isEmpty#157, sum#158] Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(sales#42), sum(number_sales#43)] Aggregate Attributes [2]: [sum(sales#42)#160, sum(number_sales#43)#161] Results [4]: [channel#47, i_brand_id#7, sum(sales#42)#160 AS sum_sales#89, sum(number_sales#43)#161 AS number_sales#90] -(169) HashAggregate [codegen id : 360] +(167) HashAggregate [codegen id : 360] Input [4]: [channel#47, i_brand_id#7, sum_sales#89, number_sales#90] Keys [2]: [channel#47, i_brand_id#7] Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)] Aggregate Attributes [3]: [sum#162, isEmpty#163, sum#164] Results [5]: [channel#47, i_brand_id#7, sum#165, isEmpty#166, sum#167] -(170) Exchange +(168) Exchange Input [5]: [channel#47, i_brand_id#7, sum#165, isEmpty#166, sum#167] Arguments: hashpartitioning(channel#47, i_brand_id#7, 5), true, [id=#168] -(171) HashAggregate [codegen id : 361] +(169) HashAggregate [codegen id : 361] Input [5]: [channel#47, i_brand_id#7, sum#165, isEmpty#166, sum#167] Keys [2]: [channel#47, i_brand_id#7] Functions [2]: [sum(sum_sales#89), sum(number_sales#90)] Aggregate Attributes [2]: [sum(sum_sales#89)#169, sum(number_sales#90)#170] Results [6]: [channel#47, i_brand_id#7, null AS i_class_id#171, null AS i_category_id#172, sum(sum_sales#89)#169 AS sum(sum_sales)#173, sum(number_sales#90)#170 AS sum(number_sales)#174] -(172) Union +(170) Union -(173) HashAggregate [codegen id : 362] +(171) HashAggregate [codegen id : 362] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Functions: [] Aggregate Attributes: [] Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] -(174) Exchange +(172) Exchange Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90, 5), true, [id=#175] -(175) HashAggregate [codegen id : 363] +(173) HashAggregate [codegen id : 363] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Functions: [] Aggregate Attributes: [] Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] -(176) ReusedExchange [Reuses operator id: 84] +(174) ReusedExchange [Reuses operator id: 82] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#176, isEmpty#177, count#178] -(177) HashAggregate [codegen id : 402] +(175) HashAggregate [codegen id : 402] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#176, isEmpty#177, count#178] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#179, count(1)#180] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#179 AS sales#42, count(1)#180 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#179 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#181] -(178) Filter [codegen id : 402] +(176) Filter [codegen id : 402] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#181] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#181) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#181 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(179) Project [codegen id : 402] +(177) Project [codegen id : 402] Output [6]: [store AS channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#181] -(180) ReusedExchange [Reuses operator id: 103] +(178) ReusedExchange [Reuses operator id: 101] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#182, isEmpty#183, count#184] -(181) HashAggregate [codegen id : 441] +(179) HashAggregate [codegen id : 441] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#182, isEmpty#183, count#184] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#185, count(1)#186] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#185 AS sales#60, count(1)#186 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#185 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#187] -(182) Filter [codegen id : 441] +(180) Filter [codegen id : 441] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#187] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#187) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#187 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(183) Project [codegen id : 441] +(181) Project [codegen id : 441] Output [6]: [catalog AS channel#188, i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#187] -(184) ReusedExchange [Reuses operator id: 122] +(182) ReusedExchange [Reuses operator id: 120] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#189, isEmpty#190, count#191] -(185) HashAggregate [codegen id : 480] +(183) HashAggregate [codegen id : 480] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#189, isEmpty#190, count#191] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#192, count(1)#193] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#192 AS sales#76, count(1)#193 AS number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#192 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#194] -(186) Filter [codegen id : 480] +(184) Filter [codegen id : 480] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#194] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#194) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#194 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(187) Project [codegen id : 480] +(185) Project [codegen id : 480] Output [6]: [web AS channel#195, i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#194] -(188) Union +(186) Union -(189) HashAggregate [codegen id : 481] +(187) HashAggregate [codegen id : 481] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(sales#42), partial_sum(number_sales#43)] Aggregate Attributes [3]: [sum#196, isEmpty#197, sum#198] Results [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#199, isEmpty#200, sum#201] -(190) Exchange +(188) Exchange Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#199, isEmpty#200, sum#201] Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#202] -(191) HashAggregate [codegen id : 482] +(189) HashAggregate [codegen id : 482] Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#199, isEmpty#200, sum#201] Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(sales#42), sum(number_sales#43)] Aggregate Attributes [2]: [sum(sales#42)#203, sum(number_sales#43)#204] Results [3]: [channel#47, sum(sales#42)#203 AS sum_sales#89, sum(number_sales#43)#204 AS number_sales#90] -(192) HashAggregate [codegen id : 482] +(190) HashAggregate [codegen id : 482] Input [3]: [channel#47, sum_sales#89, number_sales#90] Keys [1]: [channel#47] Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)] Aggregate Attributes [3]: [sum#205, isEmpty#206, sum#207] Results [4]: [channel#47, sum#208, isEmpty#209, sum#210] -(193) Exchange +(191) Exchange Input [4]: [channel#47, sum#208, isEmpty#209, sum#210] Arguments: hashpartitioning(channel#47, 5), true, [id=#211] -(194) HashAggregate [codegen id : 483] +(192) HashAggregate [codegen id : 483] Input [4]: [channel#47, sum#208, isEmpty#209, sum#210] Keys [1]: [channel#47] Functions [2]: [sum(sum_sales#89), sum(number_sales#90)] Aggregate Attributes [2]: [sum(sum_sales#89)#212, sum(number_sales#90)#213] Results [6]: [channel#47, null AS i_brand_id#214, null AS i_class_id#215, null AS i_category_id#216, sum(sum_sales#89)#212 AS sum(sum_sales)#217, sum(number_sales#90)#213 AS sum(number_sales)#218] -(195) Union +(193) Union -(196) HashAggregate [codegen id : 484] +(194) HashAggregate [codegen id : 484] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Functions: [] Aggregate Attributes: [] Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] -(197) Exchange +(195) Exchange Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90, 5), true, [id=#219] -(198) HashAggregate [codegen id : 485] +(196) HashAggregate [codegen id : 485] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Functions: [] Aggregate Attributes: [] Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] -(199) ReusedExchange [Reuses operator id: 84] +(197) ReusedExchange [Reuses operator id: 82] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#220, isEmpty#221, count#222] -(200) HashAggregate [codegen id : 524] +(198) HashAggregate [codegen id : 524] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#220, isEmpty#221, count#222] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#223, count(1)#224] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#223 AS sales#42, count(1)#224 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#223 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#225] -(201) Filter [codegen id : 524] +(199) Filter [codegen id : 524] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#225] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#225) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#225 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(202) Project [codegen id : 524] +(200) Project [codegen id : 524] Output [6]: [store AS channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#225] -(203) ReusedExchange [Reuses operator id: 103] +(201) ReusedExchange [Reuses operator id: 101] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#226, isEmpty#227, count#228] -(204) HashAggregate [codegen id : 563] +(202) HashAggregate [codegen id : 563] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#226, isEmpty#227, count#228] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#229, count(1)#230] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#229 AS sales#60, count(1)#230 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#229 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#231] -(205) Filter [codegen id : 563] +(203) Filter [codegen id : 563] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#231] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#231) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#231 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(206) Project [codegen id : 563] +(204) Project [codegen id : 563] Output [6]: [catalog AS channel#232, i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#231] -(207) ReusedExchange [Reuses operator id: 122] +(205) ReusedExchange [Reuses operator id: 120] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#233, isEmpty#234, count#235] -(208) HashAggregate [codegen id : 602] +(206) HashAggregate [codegen id : 602] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#233, isEmpty#234, count#235] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#236, count(1)#237] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#236 AS sales#76, count(1)#237 AS number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#236 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#238] -(209) Filter [codegen id : 602] +(207) Filter [codegen id : 602] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#238] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#238) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#238 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(210) Project [codegen id : 602] +(208) Project [codegen id : 602] Output [6]: [web AS channel#239, i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#238] -(211) Union +(209) Union -(212) HashAggregate [codegen id : 603] +(210) HashAggregate [codegen id : 603] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(sales#42), partial_sum(number_sales#43)] Aggregate Attributes [3]: [sum#240, isEmpty#241, sum#242] Results [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#243, isEmpty#244, sum#245] -(213) Exchange +(211) Exchange Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#243, isEmpty#244, sum#245] Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, 5), true, [id=#246] -(214) HashAggregate [codegen id : 604] +(212) HashAggregate [codegen id : 604] Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#243, isEmpty#244, sum#245] Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(sales#42), sum(number_sales#43)] Aggregate Attributes [2]: [sum(sales#42)#247, sum(number_sales#43)#248] Results [2]: [sum(sales#42)#247 AS sum_sales#89, sum(number_sales#43)#248 AS number_sales#90] -(215) HashAggregate [codegen id : 604] +(213) HashAggregate [codegen id : 604] Input [2]: [sum_sales#89, number_sales#90] Keys: [] Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)] Aggregate Attributes [3]: [sum#249, isEmpty#250, sum#251] Results [3]: [sum#252, isEmpty#253, sum#254] -(216) Exchange +(214) Exchange Input [3]: [sum#252, isEmpty#253, sum#254] Arguments: SinglePartition, true, [id=#255] -(217) HashAggregate [codegen id : 605] +(215) HashAggregate [codegen id : 605] Input [3]: [sum#252, isEmpty#253, sum#254] Keys: [] Functions [2]: [sum(sum_sales#89), sum(number_sales#90)] Aggregate Attributes [2]: [sum(sum_sales#89)#256, sum(number_sales#90)#257] Results [6]: [null AS channel#258, null AS i_brand_id#259, null AS i_class_id#260, null AS i_category_id#261, sum(sum_sales#89)#256 AS sum(sum_sales)#262, sum(number_sales#90)#257 AS sum(number_sales)#263] -(218) Union +(216) Union -(219) HashAggregate [codegen id : 606] +(217) HashAggregate [codegen id : 606] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Functions: [] Aggregate Attributes: [] Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] -(220) Exchange +(218) Exchange Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90, 5), true, [id=#264] -(221) HashAggregate [codegen id : 607] +(219) HashAggregate [codegen id : 607] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Functions: [] Aggregate Attributes: [] Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] -(222) TakeOrderedAndProject +(220) TakeOrderedAndProject Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Arguments: 100, [channel#47 ASC NULLS FIRST, i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] ===== Subqueries ===== -Subquery:1 Hosting operator id = 86 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* HashAggregate (252) -+- Exchange (251) - +- * HashAggregate (250) - +- Union (249) - :- * Project (232) - : +- * BroadcastHashJoin Inner BuildRight (231) - : :- * Filter (225) - : : +- * ColumnarToRow (224) - : : +- Scan parquet default.store_sales (223) - : +- BroadcastExchange (230) - : +- * Project (229) - : +- * Filter (228) - : +- * ColumnarToRow (227) - : +- Scan parquet default.date_dim (226) - :- * Project (242) - : +- * BroadcastHashJoin Inner BuildRight (241) - : :- * Filter (235) - : : +- * ColumnarToRow (234) - : : +- Scan parquet default.catalog_sales (233) - : +- BroadcastExchange (240) - : +- * Project (239) - : +- * Filter (238) - : +- * ColumnarToRow (237) - : +- Scan parquet default.date_dim (236) - +- * Project (248) - +- * BroadcastHashJoin Inner BuildRight (247) - :- * Filter (245) - : +- * ColumnarToRow (244) - : +- Scan parquet default.web_sales (243) - +- ReusedExchange (246) - - -(223) Scan parquet default.store_sales +Subquery:1 Hosting operator id = 84 Hosting Expression = Subquery scalar-subquery#45, [id=#46] +* HashAggregate (250) ++- Exchange (249) + +- * HashAggregate (248) + +- Union (247) + :- * Project (230) + : +- * BroadcastHashJoin Inner BuildRight (229) + : :- * Filter (223) + : : +- * ColumnarToRow (222) + : : +- Scan parquet default.store_sales (221) + : +- BroadcastExchange (228) + : +- * Project (227) + : +- * Filter (226) + : +- * ColumnarToRow (225) + : +- Scan parquet default.date_dim (224) + :- * Project (240) + : +- * BroadcastHashJoin Inner BuildRight (239) + : :- * Filter (233) + : : +- * ColumnarToRow (232) + : : +- Scan parquet default.catalog_sales (231) + : +- BroadcastExchange (238) + : +- * Project (237) + : +- * Filter (236) + : +- * ColumnarToRow (235) + : +- Scan parquet default.date_dim (234) + +- * Project (246) + +- * BroadcastHashJoin Inner BuildRight (245) + :- * Filter (243) + : +- * ColumnarToRow (242) + : +- Scan parquet default.web_sales (241) + +- ReusedExchange (244) + + +(221) Scan parquet default.store_sales Output [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(224) ColumnarToRow [codegen id : 2] +(222) ColumnarToRow [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] -(225) Filter [codegen id : 2] +(223) Filter [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Condition : isnotnull(ss_sold_date_sk#1) -(226) Scan parquet default.date_dim +(224) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_year#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(227) ColumnarToRow [codegen id : 1] +(225) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] -(228) Filter [codegen id : 1] +(226) Filter [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] Condition : (((isnotnull(d_year#11) AND (d_year#11 >= 1999)) AND (d_year#11 <= 2001)) AND isnotnull(d_date_sk#10)) -(229) Project [codegen id : 1] +(227) Project [codegen id : 1] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_year#11] -(230) BroadcastExchange +(228) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#265] -(231) BroadcastHashJoin [codegen id : 2] +(229) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(232) Project [codegen id : 2] +(230) Project [codegen id : 2] Output [2]: [ss_quantity#3 AS quantity#266, ss_list_price#4 AS list_price#267] Input [4]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(233) Scan parquet default.catalog_sales +(231) Scan parquet default.catalog_sales Output [3]: [cs_sold_date_sk#18, cs_quantity#48, cs_list_price#49] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(234) ColumnarToRow [codegen id : 4] +(232) ColumnarToRow [codegen id : 4] Input [3]: [cs_sold_date_sk#18, cs_quantity#48, cs_list_price#49] -(235) Filter [codegen id : 4] +(233) Filter [codegen id : 4] Input [3]: [cs_sold_date_sk#18, cs_quantity#48, cs_list_price#49] Condition : isnotnull(cs_sold_date_sk#18) -(236) Scan parquet default.date_dim +(234) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_year#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(237) ColumnarToRow [codegen id : 3] +(235) ColumnarToRow [codegen id : 3] Input [2]: [d_date_sk#10, d_year#11] -(238) Filter [codegen id : 3] +(236) Filter [codegen id : 3] Input [2]: [d_date_sk#10, d_year#11] Condition : (((isnotnull(d_year#11) AND (d_year#11 >= 1998)) AND (d_year#11 <= 2000)) AND isnotnull(d_date_sk#10)) -(239) Project [codegen id : 3] +(237) Project [codegen id : 3] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_year#11] -(240) BroadcastExchange +(238) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#268] -(241) BroadcastHashJoin [codegen id : 4] +(239) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#18] Right keys [1]: [d_date_sk#10] Join condition: None -(242) Project [codegen id : 4] +(240) Project [codegen id : 4] Output [2]: [cs_quantity#48 AS quantity#269, cs_list_price#49 AS list_price#270] Input [4]: [cs_sold_date_sk#18, cs_quantity#48, cs_list_price#49, d_date_sk#10] -(243) Scan parquet default.web_sales +(241) Scan parquet default.web_sales Output [3]: [ws_sold_date_sk#22, ws_quantity#64, ws_list_price#65] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(244) ColumnarToRow [codegen id : 6] +(242) ColumnarToRow [codegen id : 6] Input [3]: [ws_sold_date_sk#22, ws_quantity#64, ws_list_price#65] -(245) Filter [codegen id : 6] +(243) Filter [codegen id : 6] Input [3]: [ws_sold_date_sk#22, ws_quantity#64, ws_list_price#65] Condition : isnotnull(ws_sold_date_sk#22) -(246) ReusedExchange [Reuses operator id: 240] +(244) ReusedExchange [Reuses operator id: 238] Output [1]: [d_date_sk#10] -(247) BroadcastHashJoin [codegen id : 6] +(245) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#22] Right keys [1]: [d_date_sk#10] Join condition: None -(248) Project [codegen id : 6] +(246) Project [codegen id : 6] Output [2]: [ws_quantity#64 AS quantity#271, ws_list_price#65 AS list_price#272] Input [4]: [ws_sold_date_sk#22, ws_quantity#64, ws_list_price#65, d_date_sk#10] -(249) Union +(247) Union -(250) HashAggregate [codegen id : 7] +(248) HashAggregate [codegen id : 7] Input [2]: [quantity#266, list_price#267] Keys: [] Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#266 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#267 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [2]: [sum#273, count#274] Results [2]: [sum#275, count#276] -(251) Exchange +(249) Exchange Input [2]: [sum#275, count#276] Arguments: SinglePartition, true, [id=#277] -(252) HashAggregate [codegen id : 8] +(250) HashAggregate [codegen id : 8] Input [2]: [sum#275, count#276] Keys: [] Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#266 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#267 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#266 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#267 as decimal(12,2)))), DecimalType(18,2), true))#278] Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#266 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#267 as decimal(12,2)))), DecimalType(18,2), true))#278 AS average_sales#279] -Subquery:2 Hosting operator id = 105 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:2 Hosting operator id = 103 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:3 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:3 Hosting operator id = 122 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:4 Hosting operator id = 132 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:4 Hosting operator id = 130 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:5 Hosting operator id = 136 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:5 Hosting operator id = 134 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:6 Hosting operator id = 140 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:6 Hosting operator id = 138 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:7 Hosting operator id = 155 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:7 Hosting operator id = 153 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:8 Hosting operator id = 159 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:8 Hosting operator id = 157 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:9 Hosting operator id = 163 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:9 Hosting operator id = 161 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:10 Hosting operator id = 178 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:10 Hosting operator id = 176 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:11 Hosting operator id = 182 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:11 Hosting operator id = 180 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:12 Hosting operator id = 186 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:12 Hosting operator id = 184 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:13 Hosting operator id = 201 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:13 Hosting operator id = 199 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:14 Hosting operator id = 205 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:14 Hosting operator id = 203 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:15 Hosting operator id = 209 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:15 Hosting operator id = 207 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt index 30856e02f2b6..b7b6edd57e2b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt @@ -125,82 +125,80 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastExchange #9 WholeStageCodegen (16) HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #10 - WholeStageCodegen (15) - HashAggregate [brand_id,class_id,category_id] - InputAdapter - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (6) - Sort [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #11 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] + InputAdapter + Exchange [brand_id,class_id,category_id] #10 + WholeStageCodegen (15) + HashAggregate [brand_id,class_id,category_id] + InputAdapter + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (6) + Sort [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #11 + WholeStageCodegen (5) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk,ss_sold_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #12 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] InputAdapter - BroadcastExchange #13 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (10) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] + BroadcastExchange #12 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (10) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #14 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk,cs_sold_date_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #12 + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] InputAdapter - BroadcastExchange #15 - WholeStageCodegen (8) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (14) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #16 - WholeStageCodegen (13) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk,ws_sold_date_sk] + ReusedExchange [d_date_sk] #12 + InputAdapter + BroadcastExchange #15 + WholeStageCodegen (8) + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (14) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #16 + WholeStageCodegen (13) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk,ws_sold_date_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #12 + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15 + ReusedExchange [d_date_sk] #12 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15 InputAdapter BroadcastExchange #17 WholeStageCodegen (19) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt index 238053a3428e..55536ec92731 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt @@ -1,210 +1,208 @@ == Physical Plan == -TakeOrderedAndProject (206) -+- * HashAggregate (205) - +- Exchange (204) - +- * HashAggregate (203) - +- Union (202) - :- * HashAggregate (182) - : +- Exchange (181) - : +- * HashAggregate (180) - : +- Union (179) - : :- * HashAggregate (159) - : : +- Exchange (158) - : : +- * HashAggregate (157) - : : +- Union (156) - : : :- * HashAggregate (136) - : : : +- Exchange (135) - : : : +- * HashAggregate (134) - : : : +- Union (133) - : : : :- * HashAggregate (113) - : : : : +- Exchange (112) - : : : : +- * HashAggregate (111) - : : : : +- Union (110) - : : : : :- * Project (77) - : : : : : +- * Filter (76) - : : : : : +- * HashAggregate (75) - : : : : : +- Exchange (74) - : : : : : +- * HashAggregate (73) - : : : : : +- * Project (72) - : : : : : +- * BroadcastHashJoin Inner BuildRight (71) - : : : : : :- * Project (65) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (64) - : : : : : : :- * BroadcastHashJoin LeftSemi BuildRight (57) +TakeOrderedAndProject (204) ++- * HashAggregate (203) + +- Exchange (202) + +- * HashAggregate (201) + +- Union (200) + :- * HashAggregate (180) + : +- Exchange (179) + : +- * HashAggregate (178) + : +- Union (177) + : :- * HashAggregate (157) + : : +- Exchange (156) + : : +- * HashAggregate (155) + : : +- Union (154) + : : :- * HashAggregate (134) + : : : +- Exchange (133) + : : : +- * HashAggregate (132) + : : : +- Union (131) + : : : :- * HashAggregate (111) + : : : : +- Exchange (110) + : : : : +- * HashAggregate (109) + : : : : +- Union (108) + : : : : :- * Project (75) + : : : : : +- * Filter (74) + : : : : : +- * HashAggregate (73) + : : : : : +- Exchange (72) + : : : : : +- * HashAggregate (71) + : : : : : +- * Project (70) + : : : : : +- * BroadcastHashJoin Inner BuildRight (69) + : : : : : :- * Project (63) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (62) + : : : : : : :- * BroadcastHashJoin LeftSemi BuildRight (55) : : : : : : : :- * Filter (3) : : : : : : : : +- * ColumnarToRow (2) : : : : : : : : +- Scan parquet default.store_sales (1) - : : : : : : : +- BroadcastExchange (56) - : : : : : : : +- * Project (55) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : : : : : +- BroadcastExchange (54) + : : : : : : : +- * Project (53) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (52) : : : : : : : :- * Filter (6) : : : : : : : : +- * ColumnarToRow (5) : : : : : : : : +- Scan parquet default.item (4) - : : : : : : : +- BroadcastExchange (53) - : : : : : : : +- * HashAggregate (52) - : : : : : : : +- * HashAggregate (51) - : : : : : : : +- * HashAggregate (50) - : : : : : : : +- Exchange (49) - : : : : : : : +- * HashAggregate (48) - : : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : : : : : :- * BroadcastHashJoin LeftSemi BuildRight (36) - : : : : : : : : :- * Project (22) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : : : : : : : :- * Project (15) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : : : : : :- * Filter (9) - : : : : : : : : : : : +- * ColumnarToRow (8) - : : : : : : : : : : : +- Scan parquet default.store_sales (7) - : : : : : : : : : : +- BroadcastExchange (13) - : : : : : : : : : : +- * Filter (12) - : : : : : : : : : : +- * ColumnarToRow (11) - : : : : : : : : : : +- Scan parquet default.item (10) - : : : : : : : : : +- BroadcastExchange (20) - : : : : : : : : : +- * Project (19) - : : : : : : : : : +- * Filter (18) - : : : : : : : : : +- * ColumnarToRow (17) - : : : : : : : : : +- Scan parquet default.date_dim (16) - : : : : : : : : +- BroadcastExchange (35) - : : : : : : : : +- * Project (34) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : : : : : : :- * Project (31) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : : : : : : : :- * Filter (25) - : : : : : : : : : : +- * ColumnarToRow (24) - : : : : : : : : : : +- Scan parquet default.catalog_sales (23) - : : : : : : : : : +- BroadcastExchange (29) - : : : : : : : : : +- * Filter (28) - : : : : : : : : : +- * ColumnarToRow (27) - : : : : : : : : : +- Scan parquet default.item (26) - : : : : : : : : +- ReusedExchange (32) - : : : : : : : +- BroadcastExchange (46) - : : : : : : : +- * Project (45) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : : : : : :- * Project (42) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : : : : : :- * Filter (39) - : : : : : : : : : +- * ColumnarToRow (38) - : : : : : : : : : +- Scan parquet default.web_sales (37) - : : : : : : : : +- ReusedExchange (40) - : : : : : : : +- ReusedExchange (43) - : : : : : : +- BroadcastExchange (63) - : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (62) - : : : : : : :- * Filter (60) - : : : : : : : +- * ColumnarToRow (59) - : : : : : : : +- Scan parquet default.item (58) - : : : : : : +- ReusedExchange (61) - : : : : : +- BroadcastExchange (70) - : : : : : +- * Project (69) - : : : : : +- * Filter (68) - : : : : : +- * ColumnarToRow (67) - : : : : : +- Scan parquet default.date_dim (66) - : : : : :- * Project (93) - : : : : : +- * Filter (92) - : : : : : +- * HashAggregate (91) - : : : : : +- Exchange (90) - : : : : : +- * HashAggregate (89) - : : : : : +- * Project (88) - : : : : : +- * BroadcastHashJoin Inner BuildRight (87) - : : : : : :- * Project (85) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (84) - : : : : : : :- * BroadcastHashJoin LeftSemi BuildRight (82) - : : : : : : : :- * Filter (80) - : : : : : : : : +- * ColumnarToRow (79) - : : : : : : : : +- Scan parquet default.catalog_sales (78) - : : : : : : : +- ReusedExchange (81) - : : : : : : +- ReusedExchange (83) - : : : : : +- ReusedExchange (86) - : : : : +- * Project (109) - : : : : +- * Filter (108) - : : : : +- * HashAggregate (107) - : : : : +- Exchange (106) - : : : : +- * HashAggregate (105) - : : : : +- * Project (104) - : : : : +- * BroadcastHashJoin Inner BuildRight (103) - : : : : :- * Project (101) - : : : : : +- * BroadcastHashJoin Inner BuildRight (100) - : : : : : :- * BroadcastHashJoin LeftSemi BuildRight (98) - : : : : : : :- * Filter (96) - : : : : : : : +- * ColumnarToRow (95) - : : : : : : : +- Scan parquet default.web_sales (94) - : : : : : : +- ReusedExchange (97) - : : : : : +- ReusedExchange (99) - : : : : +- ReusedExchange (102) - : : : +- * HashAggregate (132) - : : : +- Exchange (131) - : : : +- * HashAggregate (130) - : : : +- * HashAggregate (129) - : : : +- Exchange (128) - : : : +- * HashAggregate (127) - : : : +- Union (126) - : : : :- * Project (117) - : : : : +- * Filter (116) - : : : : +- * HashAggregate (115) - : : : : +- ReusedExchange (114) - : : : :- * Project (121) - : : : : +- * Filter (120) - : : : : +- * HashAggregate (119) - : : : : +- ReusedExchange (118) - : : : +- * Project (125) - : : : +- * Filter (124) - : : : +- * HashAggregate (123) - : : : +- ReusedExchange (122) - : : +- * HashAggregate (155) - : : +- Exchange (154) - : : +- * HashAggregate (153) - : : +- * HashAggregate (152) - : : +- Exchange (151) - : : +- * HashAggregate (150) - : : +- Union (149) - : : :- * Project (140) - : : : +- * Filter (139) - : : : +- * HashAggregate (138) - : : : +- ReusedExchange (137) - : : :- * Project (144) - : : : +- * Filter (143) - : : : +- * HashAggregate (142) - : : : +- ReusedExchange (141) - : : +- * Project (148) - : : +- * Filter (147) - : : +- * HashAggregate (146) - : : +- ReusedExchange (145) - : +- * HashAggregate (178) - : +- Exchange (177) - : +- * HashAggregate (176) - : +- * HashAggregate (175) - : +- Exchange (174) - : +- * HashAggregate (173) - : +- Union (172) - : :- * Project (163) - : : +- * Filter (162) - : : +- * HashAggregate (161) - : : +- ReusedExchange (160) - : :- * Project (167) - : : +- * Filter (166) - : : +- * HashAggregate (165) - : : +- ReusedExchange (164) - : +- * Project (171) - : +- * Filter (170) - : +- * HashAggregate (169) - : +- ReusedExchange (168) - +- * HashAggregate (201) - +- Exchange (200) - +- * HashAggregate (199) - +- * HashAggregate (198) - +- Exchange (197) - +- * HashAggregate (196) - +- Union (195) - :- * Project (186) - : +- * Filter (185) - : +- * HashAggregate (184) - : +- ReusedExchange (183) - :- * Project (190) - : +- * Filter (189) - : +- * HashAggregate (188) - : +- ReusedExchange (187) - +- * Project (194) - +- * Filter (193) - +- * HashAggregate (192) - +- ReusedExchange (191) + : : : : : : : +- BroadcastExchange (51) + : : : : : : : +- * HashAggregate (50) + : : : : : : : +- Exchange (49) + : : : : : : : +- * HashAggregate (48) + : : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) + : : : : : : : :- * BroadcastHashJoin LeftSemi BuildRight (36) + : : : : : : : : :- * Project (22) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : : : : : : : :- * Project (15) + : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : : : : : : :- * Filter (9) + : : : : : : : : : : : +- * ColumnarToRow (8) + : : : : : : : : : : : +- Scan parquet default.store_sales (7) + : : : : : : : : : : +- BroadcastExchange (13) + : : : : : : : : : : +- * Filter (12) + : : : : : : : : : : +- * ColumnarToRow (11) + : : : : : : : : : : +- Scan parquet default.item (10) + : : : : : : : : : +- BroadcastExchange (20) + : : : : : : : : : +- * Project (19) + : : : : : : : : : +- * Filter (18) + : : : : : : : : : +- * ColumnarToRow (17) + : : : : : : : : : +- Scan parquet default.date_dim (16) + : : : : : : : : +- BroadcastExchange (35) + : : : : : : : : +- * Project (34) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : : : : : : :- * Project (31) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (30) + : : : : : : : : : :- * Filter (25) + : : : : : : : : : : +- * ColumnarToRow (24) + : : : : : : : : : : +- Scan parquet default.catalog_sales (23) + : : : : : : : : : +- BroadcastExchange (29) + : : : : : : : : : +- * Filter (28) + : : : : : : : : : +- * ColumnarToRow (27) + : : : : : : : : : +- Scan parquet default.item (26) + : : : : : : : : +- ReusedExchange (32) + : : : : : : : +- BroadcastExchange (46) + : : : : : : : +- * Project (45) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (44) + : : : : : : : :- * Project (42) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (41) + : : : : : : : : :- * Filter (39) + : : : : : : : : : +- * ColumnarToRow (38) + : : : : : : : : : +- Scan parquet default.web_sales (37) + : : : : : : : : +- ReusedExchange (40) + : : : : : : : +- ReusedExchange (43) + : : : : : : +- BroadcastExchange (61) + : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (60) + : : : : : : :- * Filter (58) + : : : : : : : +- * ColumnarToRow (57) + : : : : : : : +- Scan parquet default.item (56) + : : : : : : +- ReusedExchange (59) + : : : : : +- BroadcastExchange (68) + : : : : : +- * Project (67) + : : : : : +- * Filter (66) + : : : : : +- * ColumnarToRow (65) + : : : : : +- Scan parquet default.date_dim (64) + : : : : :- * Project (91) + : : : : : +- * Filter (90) + : : : : : +- * HashAggregate (89) + : : : : : +- Exchange (88) + : : : : : +- * HashAggregate (87) + : : : : : +- * Project (86) + : : : : : +- * BroadcastHashJoin Inner BuildRight (85) + : : : : : :- * Project (83) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (82) + : : : : : : :- * BroadcastHashJoin LeftSemi BuildRight (80) + : : : : : : : :- * Filter (78) + : : : : : : : : +- * ColumnarToRow (77) + : : : : : : : : +- Scan parquet default.catalog_sales (76) + : : : : : : : +- ReusedExchange (79) + : : : : : : +- ReusedExchange (81) + : : : : : +- ReusedExchange (84) + : : : : +- * Project (107) + : : : : +- * Filter (106) + : : : : +- * HashAggregate (105) + : : : : +- Exchange (104) + : : : : +- * HashAggregate (103) + : : : : +- * Project (102) + : : : : +- * BroadcastHashJoin Inner BuildRight (101) + : : : : :- * Project (99) + : : : : : +- * BroadcastHashJoin Inner BuildRight (98) + : : : : : :- * BroadcastHashJoin LeftSemi BuildRight (96) + : : : : : : :- * Filter (94) + : : : : : : : +- * ColumnarToRow (93) + : : : : : : : +- Scan parquet default.web_sales (92) + : : : : : : +- ReusedExchange (95) + : : : : : +- ReusedExchange (97) + : : : : +- ReusedExchange (100) + : : : +- * HashAggregate (130) + : : : +- Exchange (129) + : : : +- * HashAggregate (128) + : : : +- * HashAggregate (127) + : : : +- Exchange (126) + : : : +- * HashAggregate (125) + : : : +- Union (124) + : : : :- * Project (115) + : : : : +- * Filter (114) + : : : : +- * HashAggregate (113) + : : : : +- ReusedExchange (112) + : : : :- * Project (119) + : : : : +- * Filter (118) + : : : : +- * HashAggregate (117) + : : : : +- ReusedExchange (116) + : : : +- * Project (123) + : : : +- * Filter (122) + : : : +- * HashAggregate (121) + : : : +- ReusedExchange (120) + : : +- * HashAggregate (153) + : : +- Exchange (152) + : : +- * HashAggregate (151) + : : +- * HashAggregate (150) + : : +- Exchange (149) + : : +- * HashAggregate (148) + : : +- Union (147) + : : :- * Project (138) + : : : +- * Filter (137) + : : : +- * HashAggregate (136) + : : : +- ReusedExchange (135) + : : :- * Project (142) + : : : +- * Filter (141) + : : : +- * HashAggregate (140) + : : : +- ReusedExchange (139) + : : +- * Project (146) + : : +- * Filter (145) + : : +- * HashAggregate (144) + : : +- ReusedExchange (143) + : +- * HashAggregate (176) + : +- Exchange (175) + : +- * HashAggregate (174) + : +- * HashAggregate (173) + : +- Exchange (172) + : +- * HashAggregate (171) + : +- Union (170) + : :- * Project (161) + : : +- * Filter (160) + : : +- * HashAggregate (159) + : : +- ReusedExchange (158) + : :- * Project (165) + : : +- * Filter (164) + : : +- * HashAggregate (163) + : : +- ReusedExchange (162) + : +- * Project (169) + : +- * Filter (168) + : +- * HashAggregate (167) + : +- ReusedExchange (166) + +- * HashAggregate (199) + +- Exchange (198) + +- * HashAggregate (197) + +- * HashAggregate (196) + +- Exchange (195) + +- * HashAggregate (194) + +- Union (193) + :- * Project (184) + : +- * Filter (183) + : +- * HashAggregate (182) + : +- ReusedExchange (181) + :- * Project (188) + : +- * Filter (187) + : +- * HashAggregate (186) + : +- ReusedExchange (185) + +- * Project (192) + +- * Filter (191) + +- * HashAggregate (190) + +- ReusedExchange (189) (1) Scan parquet default.store_sales @@ -434,947 +432,933 @@ Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#13, class_id#14, category_id#15] -(51) HashAggregate [codegen id : 10] -Input [3]: [brand_id#13, class_id#14, category_id#15] -Keys [3]: [brand_id#13, class_id#14, category_id#15] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#13, class_id#14, category_id#15] - -(52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#13, class_id#14, category_id#15] -Keys [3]: [brand_id#13, class_id#14, category_id#15] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#13, class_id#14, category_id#15] - -(53) BroadcastExchange +(51) BroadcastExchange Input [3]: [brand_id#13, class_id#14, category_id#15] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#24] -(54) BroadcastHashJoin [codegen id : 11] +(52) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Right keys [3]: [brand_id#13, class_id#14, category_id#15] Join condition: None -(55) Project [codegen id : 11] +(53) Project [codegen id : 11] Output [1]: [i_item_sk#5 AS ss_item_sk#25] Input [7]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8, brand_id#13, class_id#14, category_id#15] -(56) BroadcastExchange +(54) BroadcastExchange Input [1]: [ss_item_sk#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] -(57) BroadcastHashJoin [codegen id : 25] +(55) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#2] Right keys [1]: [ss_item_sk#25] Join condition: None -(58) Scan parquet default.item +(56) Scan parquet default.item Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(59) ColumnarToRow [codegen id : 23] +(57) ColumnarToRow [codegen id : 23] Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(60) Filter [codegen id : 23] +(58) Filter [codegen id : 23] Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Condition : isnotnull(i_item_sk#5) -(61) ReusedExchange [Reuses operator id: 56] +(59) ReusedExchange [Reuses operator id: 54] Output [1]: [ss_item_sk#25] -(62) BroadcastHashJoin [codegen id : 23] +(60) BroadcastHashJoin [codegen id : 23] Left keys [1]: [i_item_sk#5] Right keys [1]: [ss_item_sk#25] Join condition: None -(63) BroadcastExchange +(61) BroadcastExchange Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] -(64) BroadcastHashJoin [codegen id : 25] +(62) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#5] Join condition: None -(65) Project [codegen id : 25] +(63) Project [codegen id : 25] Output [6]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8] Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(66) Scan parquet default.date_dim +(64) Scan parquet default.date_dim Output [3]: [d_date_sk#10, d_year#11, d_moy#28] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(67) ColumnarToRow [codegen id : 24] +(65) ColumnarToRow [codegen id : 24] Input [3]: [d_date_sk#10, d_year#11, d_moy#28] -(68) Filter [codegen id : 24] +(66) Filter [codegen id : 24] Input [3]: [d_date_sk#10, d_year#11, d_moy#28] Condition : ((((isnotnull(d_year#11) AND isnotnull(d_moy#28)) AND (d_year#11 = 2000)) AND (d_moy#28 = 11)) AND isnotnull(d_date_sk#10)) -(69) Project [codegen id : 24] +(67) Project [codegen id : 24] Output [1]: [d_date_sk#10] Input [3]: [d_date_sk#10, d_year#11, d_moy#28] -(70) BroadcastExchange +(68) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] -(71) BroadcastHashJoin [codegen id : 25] +(69) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(72) Project [codegen id : 25] +(70) Project [codegen id : 25] Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8] Input [7]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8, d_date_sk#10] -(73) HashAggregate [codegen id : 25] +(71) HashAggregate [codegen id : 25] Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#30, isEmpty#31, count#32] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#33, isEmpty#34, count#35] -(74) Exchange +(72) Exchange Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#33, isEmpty#34, count#35] Arguments: hashpartitioning(i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#36] -(75) HashAggregate [codegen id : 26] +(73) HashAggregate [codegen id : 26] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#33, isEmpty#34, count#35] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37, count(1)#38] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sales#39, count(1)#38 AS number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41] -(76) Filter [codegen id : 26] +(74) Filter [codegen id : 26] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41 as decimal(32,6)) > cast(Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(77) Project [codegen id : 26] +(75) Project [codegen id : 26] Output [6]: [store AS channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41] -(78) Scan parquet default.catalog_sales +(76) Scan parquet default.catalog_sales Output [4]: [cs_sold_date_sk#16, cs_item_sk#17, cs_quantity#45, cs_list_price#46] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(79) ColumnarToRow [codegen id : 51] +(77) ColumnarToRow [codegen id : 51] Input [4]: [cs_sold_date_sk#16, cs_item_sk#17, cs_quantity#45, cs_list_price#46] -(80) Filter [codegen id : 51] +(78) Filter [codegen id : 51] Input [4]: [cs_sold_date_sk#16, cs_item_sk#17, cs_quantity#45, cs_list_price#46] Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_sold_date_sk#16)) -(81) ReusedExchange [Reuses operator id: 56] +(79) ReusedExchange [Reuses operator id: 54] Output [1]: [ss_item_sk#25] -(82) BroadcastHashJoin [codegen id : 51] +(80) BroadcastHashJoin [codegen id : 51] Left keys [1]: [cs_item_sk#17] Right keys [1]: [ss_item_sk#25] Join condition: None -(83) ReusedExchange [Reuses operator id: 63] +(81) ReusedExchange [Reuses operator id: 61] Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(84) BroadcastHashJoin [codegen id : 51] +(82) BroadcastHashJoin [codegen id : 51] Left keys [1]: [cs_item_sk#17] Right keys [1]: [i_item_sk#5] Join condition: None -(85) Project [codegen id : 51] +(83) Project [codegen id : 51] Output [6]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46, i_brand_id#6, i_class_id#7, i_category_id#8] Input [8]: [cs_sold_date_sk#16, cs_item_sk#17, cs_quantity#45, cs_list_price#46, i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(86) ReusedExchange [Reuses operator id: 70] +(84) ReusedExchange [Reuses operator id: 68] Output [1]: [d_date_sk#10] -(87) BroadcastHashJoin [codegen id : 51] +(85) BroadcastHashJoin [codegen id : 51] Left keys [1]: [cs_sold_date_sk#16] Right keys [1]: [d_date_sk#10] Join condition: None -(88) Project [codegen id : 51] +(86) Project [codegen id : 51] Output [5]: [cs_quantity#45, cs_list_price#46, i_brand_id#6, i_class_id#7, i_category_id#8] Input [7]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46, i_brand_id#6, i_class_id#7, i_category_id#8, d_date_sk#10] -(89) HashAggregate [codegen id : 51] +(87) HashAggregate [codegen id : 51] Input [5]: [cs_quantity#45, cs_list_price#46, i_brand_id#6, i_class_id#7, i_category_id#8] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#47, isEmpty#48, count#49] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#50, isEmpty#51, count#52] -(90) Exchange +(88) Exchange Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#50, isEmpty#51, count#52] Arguments: hashpartitioning(i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#53] -(91) HashAggregate [codegen id : 52] +(89) HashAggregate [codegen id : 52] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#50, isEmpty#51, count#52] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54, count(1)#55] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sales#56, count(1)#55 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58] -(92) Filter [codegen id : 52] +(90) Filter [codegen id : 52] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(93) Project [codegen id : 52] +(91) Project [codegen id : 52] Output [6]: [catalog AS channel#59, i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58] -(94) Scan parquet default.web_sales +(92) Scan parquet default.web_sales Output [4]: [ws_sold_date_sk#20, ws_item_sk#21, ws_quantity#60, ws_list_price#61] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(95) ColumnarToRow [codegen id : 77] +(93) ColumnarToRow [codegen id : 77] Input [4]: [ws_sold_date_sk#20, ws_item_sk#21, ws_quantity#60, ws_list_price#61] -(96) Filter [codegen id : 77] +(94) Filter [codegen id : 77] Input [4]: [ws_sold_date_sk#20, ws_item_sk#21, ws_quantity#60, ws_list_price#61] Condition : (isnotnull(ws_item_sk#21) AND isnotnull(ws_sold_date_sk#20)) -(97) ReusedExchange [Reuses operator id: 56] +(95) ReusedExchange [Reuses operator id: 54] Output [1]: [ss_item_sk#25] -(98) BroadcastHashJoin [codegen id : 77] +(96) BroadcastHashJoin [codegen id : 77] Left keys [1]: [ws_item_sk#21] Right keys [1]: [ss_item_sk#25] Join condition: None -(99) ReusedExchange [Reuses operator id: 63] +(97) ReusedExchange [Reuses operator id: 61] Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(100) BroadcastHashJoin [codegen id : 77] +(98) BroadcastHashJoin [codegen id : 77] Left keys [1]: [ws_item_sk#21] Right keys [1]: [i_item_sk#5] Join condition: None -(101) Project [codegen id : 77] +(99) Project [codegen id : 77] Output [6]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61, i_brand_id#6, i_class_id#7, i_category_id#8] Input [8]: [ws_sold_date_sk#20, ws_item_sk#21, ws_quantity#60, ws_list_price#61, i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(102) ReusedExchange [Reuses operator id: 70] +(100) ReusedExchange [Reuses operator id: 68] Output [1]: [d_date_sk#10] -(103) BroadcastHashJoin [codegen id : 77] +(101) BroadcastHashJoin [codegen id : 77] Left keys [1]: [ws_sold_date_sk#20] Right keys [1]: [d_date_sk#10] Join condition: None -(104) Project [codegen id : 77] +(102) Project [codegen id : 77] Output [5]: [ws_quantity#60, ws_list_price#61, i_brand_id#6, i_class_id#7, i_category_id#8] Input [7]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61, i_brand_id#6, i_class_id#7, i_category_id#8, d_date_sk#10] -(105) HashAggregate [codegen id : 77] +(103) HashAggregate [codegen id : 77] Input [5]: [ws_quantity#60, ws_list_price#61, i_brand_id#6, i_class_id#7, i_category_id#8] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#62, isEmpty#63, count#64] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#65, isEmpty#66, count#67] -(106) Exchange +(104) Exchange Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#65, isEmpty#66, count#67] Arguments: hashpartitioning(i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#68] -(107) HashAggregate [codegen id : 78] +(105) HashAggregate [codegen id : 78] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#65, isEmpty#66, count#67] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69, count(1)#70] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sales#71, count(1)#70 AS number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73] -(108) Filter [codegen id : 78] +(106) Filter [codegen id : 78] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(109) Project [codegen id : 78] +(107) Project [codegen id : 78] Output [6]: [web AS channel#74, i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73] -(110) Union +(108) Union -(111) HashAggregate [codegen id : 79] +(109) HashAggregate [codegen id : 79] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [partial_sum(sales#39), partial_sum(number_sales#40)] Aggregate Attributes [3]: [sum#75, isEmpty#76, sum#77] Results [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#78, isEmpty#79, sum#80] -(112) Exchange +(110) Exchange Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#78, isEmpty#79, sum#80] Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#81] -(113) HashAggregate [codegen id : 80] +(111) HashAggregate [codegen id : 80] Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#78, isEmpty#79, sum#80] Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(sales#39), sum(number_sales#40)] Aggregate Attributes [2]: [sum(sales#39)#82, sum(number_sales#40)#83] Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum(sales#39)#82 AS sum_sales#84, sum(number_sales#40)#83 AS number_sales#85] -(114) ReusedExchange [Reuses operator id: 74] +(112) ReusedExchange [Reuses operator id: 72] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#86, isEmpty#87, count#88] -(115) HashAggregate [codegen id : 106] +(113) HashAggregate [codegen id : 106] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#86, isEmpty#87, count#88] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#89, count(1)#90] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#89 AS sales#39, count(1)#90 AS number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#89 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#91] -(116) Filter [codegen id : 106] +(114) Filter [codegen id : 106] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#91] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#91) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#91 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(117) Project [codegen id : 106] +(115) Project [codegen id : 106] Output [6]: [store AS channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#91] -(118) ReusedExchange [Reuses operator id: 90] +(116) ReusedExchange [Reuses operator id: 88] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#92, isEmpty#93, count#94] -(119) HashAggregate [codegen id : 132] +(117) HashAggregate [codegen id : 132] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#92, isEmpty#93, count#94] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#95, count(1)#96] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#95 AS sales#56, count(1)#96 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#95 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#97] -(120) Filter [codegen id : 132] +(118) Filter [codegen id : 132] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#97] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#97) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#97 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(121) Project [codegen id : 132] +(119) Project [codegen id : 132] Output [6]: [catalog AS channel#98, i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#97] -(122) ReusedExchange [Reuses operator id: 106] +(120) ReusedExchange [Reuses operator id: 104] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#99, isEmpty#100, count#101] -(123) HashAggregate [codegen id : 158] +(121) HashAggregate [codegen id : 158] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#99, isEmpty#100, count#101] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#102, count(1)#103] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#102 AS sales#71, count(1)#103 AS number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#102 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#104] -(124) Filter [codegen id : 158] +(122) Filter [codegen id : 158] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#104] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#104) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#104 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(125) Project [codegen id : 158] +(123) Project [codegen id : 158] Output [6]: [web AS channel#105, i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#104] -(126) Union +(124) Union -(127) HashAggregate [codegen id : 159] +(125) HashAggregate [codegen id : 159] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [partial_sum(sales#39), partial_sum(number_sales#40)] Aggregate Attributes [3]: [sum#106, isEmpty#107, sum#108] Results [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#109, isEmpty#110, sum#111] -(128) Exchange +(126) Exchange Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#109, isEmpty#110, sum#111] Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#112] -(129) HashAggregate [codegen id : 160] +(127) HashAggregate [codegen id : 160] Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#109, isEmpty#110, sum#111] Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(sales#39), sum(number_sales#40)] Aggregate Attributes [2]: [sum(sales#39)#113, sum(number_sales#40)#114] Results [5]: [channel#44, i_brand_id#6, i_class_id#7, sum(sales#39)#113 AS sum_sales#84, sum(number_sales#40)#114 AS number_sales#85] -(130) HashAggregate [codegen id : 160] +(128) HashAggregate [codegen id : 160] Input [5]: [channel#44, i_brand_id#6, i_class_id#7, sum_sales#84, number_sales#85] Keys [3]: [channel#44, i_brand_id#6, i_class_id#7] Functions [2]: [partial_sum(sum_sales#84), partial_sum(number_sales#85)] Aggregate Attributes [3]: [sum#115, isEmpty#116, sum#117] Results [6]: [channel#44, i_brand_id#6, i_class_id#7, sum#118, isEmpty#119, sum#120] -(131) Exchange +(129) Exchange Input [6]: [channel#44, i_brand_id#6, i_class_id#7, sum#118, isEmpty#119, sum#120] Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, 5), true, [id=#121] -(132) HashAggregate [codegen id : 161] +(130) HashAggregate [codegen id : 161] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, sum#118, isEmpty#119, sum#120] Keys [3]: [channel#44, i_brand_id#6, i_class_id#7] Functions [2]: [sum(sum_sales#84), sum(number_sales#85)] Aggregate Attributes [2]: [sum(sum_sales#84)#122, sum(number_sales#85)#123] Results [6]: [channel#44, i_brand_id#6, i_class_id#7, null AS i_category_id#124, sum(sum_sales#84)#122 AS sum(sum_sales)#125, sum(number_sales#85)#123 AS sum(number_sales)#126] -(133) Union +(131) Union -(134) HashAggregate [codegen id : 162] +(132) HashAggregate [codegen id : 162] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Keys [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Functions: [] Aggregate Attributes: [] Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] -(135) Exchange +(133) Exchange Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85, 5), true, [id=#127] -(136) HashAggregate [codegen id : 163] +(134) HashAggregate [codegen id : 163] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Keys [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Functions: [] Aggregate Attributes: [] Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] -(137) ReusedExchange [Reuses operator id: 74] +(135) ReusedExchange [Reuses operator id: 72] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#128, isEmpty#129, count#130] -(138) HashAggregate [codegen id : 189] +(136) HashAggregate [codegen id : 189] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#128, isEmpty#129, count#130] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#131, count(1)#132] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#131 AS sales#39, count(1)#132 AS number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#131 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#133] -(139) Filter [codegen id : 189] +(137) Filter [codegen id : 189] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#133] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#133) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#133 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(140) Project [codegen id : 189] +(138) Project [codegen id : 189] Output [6]: [store AS channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#133] -(141) ReusedExchange [Reuses operator id: 90] +(139) ReusedExchange [Reuses operator id: 88] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#134, isEmpty#135, count#136] -(142) HashAggregate [codegen id : 215] +(140) HashAggregate [codegen id : 215] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#134, isEmpty#135, count#136] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#137, count(1)#138] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#137 AS sales#56, count(1)#138 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#137 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#139] -(143) Filter [codegen id : 215] +(141) Filter [codegen id : 215] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#139] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#139) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#139 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(144) Project [codegen id : 215] +(142) Project [codegen id : 215] Output [6]: [catalog AS channel#140, i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#139] -(145) ReusedExchange [Reuses operator id: 106] +(143) ReusedExchange [Reuses operator id: 104] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#141, isEmpty#142, count#143] -(146) HashAggregate [codegen id : 241] +(144) HashAggregate [codegen id : 241] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#141, isEmpty#142, count#143] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#144, count(1)#145] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#144 AS sales#71, count(1)#145 AS number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#144 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#146] -(147) Filter [codegen id : 241] +(145) Filter [codegen id : 241] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#146] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#146) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#146 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(148) Project [codegen id : 241] +(146) Project [codegen id : 241] Output [6]: [web AS channel#147, i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#146] -(149) Union +(147) Union -(150) HashAggregate [codegen id : 242] +(148) HashAggregate [codegen id : 242] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [partial_sum(sales#39), partial_sum(number_sales#40)] Aggregate Attributes [3]: [sum#148, isEmpty#149, sum#150] Results [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#151, isEmpty#152, sum#153] -(151) Exchange +(149) Exchange Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#151, isEmpty#152, sum#153] Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#154] -(152) HashAggregate [codegen id : 243] +(150) HashAggregate [codegen id : 243] Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#151, isEmpty#152, sum#153] Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(sales#39), sum(number_sales#40)] Aggregate Attributes [2]: [sum(sales#39)#155, sum(number_sales#40)#156] Results [4]: [channel#44, i_brand_id#6, sum(sales#39)#155 AS sum_sales#84, sum(number_sales#40)#156 AS number_sales#85] -(153) HashAggregate [codegen id : 243] +(151) HashAggregate [codegen id : 243] Input [4]: [channel#44, i_brand_id#6, sum_sales#84, number_sales#85] Keys [2]: [channel#44, i_brand_id#6] Functions [2]: [partial_sum(sum_sales#84), partial_sum(number_sales#85)] Aggregate Attributes [3]: [sum#157, isEmpty#158, sum#159] Results [5]: [channel#44, i_brand_id#6, sum#160, isEmpty#161, sum#162] -(154) Exchange +(152) Exchange Input [5]: [channel#44, i_brand_id#6, sum#160, isEmpty#161, sum#162] Arguments: hashpartitioning(channel#44, i_brand_id#6, 5), true, [id=#163] -(155) HashAggregate [codegen id : 244] +(153) HashAggregate [codegen id : 244] Input [5]: [channel#44, i_brand_id#6, sum#160, isEmpty#161, sum#162] Keys [2]: [channel#44, i_brand_id#6] Functions [2]: [sum(sum_sales#84), sum(number_sales#85)] Aggregate Attributes [2]: [sum(sum_sales#84)#164, sum(number_sales#85)#165] Results [6]: [channel#44, i_brand_id#6, null AS i_class_id#166, null AS i_category_id#167, sum(sum_sales#84)#164 AS sum(sum_sales)#168, sum(number_sales#85)#165 AS sum(number_sales)#169] -(156) Union +(154) Union -(157) HashAggregate [codegen id : 245] +(155) HashAggregate [codegen id : 245] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Keys [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Functions: [] Aggregate Attributes: [] Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] -(158) Exchange +(156) Exchange Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85, 5), true, [id=#170] -(159) HashAggregate [codegen id : 246] +(157) HashAggregate [codegen id : 246] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Keys [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Functions: [] Aggregate Attributes: [] Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] -(160) ReusedExchange [Reuses operator id: 74] +(158) ReusedExchange [Reuses operator id: 72] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#171, isEmpty#172, count#173] -(161) HashAggregate [codegen id : 272] +(159) HashAggregate [codegen id : 272] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#171, isEmpty#172, count#173] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#174, count(1)#175] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#174 AS sales#39, count(1)#175 AS number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#174 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#176] -(162) Filter [codegen id : 272] +(160) Filter [codegen id : 272] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#176] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#176) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#176 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(163) Project [codegen id : 272] +(161) Project [codegen id : 272] Output [6]: [store AS channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#176] -(164) ReusedExchange [Reuses operator id: 90] +(162) ReusedExchange [Reuses operator id: 88] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#177, isEmpty#178, count#179] -(165) HashAggregate [codegen id : 298] +(163) HashAggregate [codegen id : 298] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#177, isEmpty#178, count#179] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#180, count(1)#181] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#180 AS sales#56, count(1)#181 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#180 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#182] -(166) Filter [codegen id : 298] +(164) Filter [codegen id : 298] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#182] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#182) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#182 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(167) Project [codegen id : 298] +(165) Project [codegen id : 298] Output [6]: [catalog AS channel#183, i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#182] -(168) ReusedExchange [Reuses operator id: 106] +(166) ReusedExchange [Reuses operator id: 104] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#184, isEmpty#185, count#186] -(169) HashAggregate [codegen id : 324] +(167) HashAggregate [codegen id : 324] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#184, isEmpty#185, count#186] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#187, count(1)#188] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#187 AS sales#71, count(1)#188 AS number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#187 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#189] -(170) Filter [codegen id : 324] +(168) Filter [codegen id : 324] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#189] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#189) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#189 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(171) Project [codegen id : 324] +(169) Project [codegen id : 324] Output [6]: [web AS channel#190, i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#189] -(172) Union +(170) Union -(173) HashAggregate [codegen id : 325] +(171) HashAggregate [codegen id : 325] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [partial_sum(sales#39), partial_sum(number_sales#40)] Aggregate Attributes [3]: [sum#191, isEmpty#192, sum#193] Results [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#194, isEmpty#195, sum#196] -(174) Exchange +(172) Exchange Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#194, isEmpty#195, sum#196] Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#197] -(175) HashAggregate [codegen id : 326] +(173) HashAggregate [codegen id : 326] Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#194, isEmpty#195, sum#196] Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(sales#39), sum(number_sales#40)] Aggregate Attributes [2]: [sum(sales#39)#198, sum(number_sales#40)#199] Results [3]: [channel#44, sum(sales#39)#198 AS sum_sales#84, sum(number_sales#40)#199 AS number_sales#85] -(176) HashAggregate [codegen id : 326] +(174) HashAggregate [codegen id : 326] Input [3]: [channel#44, sum_sales#84, number_sales#85] Keys [1]: [channel#44] Functions [2]: [partial_sum(sum_sales#84), partial_sum(number_sales#85)] Aggregate Attributes [3]: [sum#200, isEmpty#201, sum#202] Results [4]: [channel#44, sum#203, isEmpty#204, sum#205] -(177) Exchange +(175) Exchange Input [4]: [channel#44, sum#203, isEmpty#204, sum#205] Arguments: hashpartitioning(channel#44, 5), true, [id=#206] -(178) HashAggregate [codegen id : 327] +(176) HashAggregate [codegen id : 327] Input [4]: [channel#44, sum#203, isEmpty#204, sum#205] Keys [1]: [channel#44] Functions [2]: [sum(sum_sales#84), sum(number_sales#85)] Aggregate Attributes [2]: [sum(sum_sales#84)#207, sum(number_sales#85)#208] Results [6]: [channel#44, null AS i_brand_id#209, null AS i_class_id#210, null AS i_category_id#211, sum(sum_sales#84)#207 AS sum(sum_sales)#212, sum(number_sales#85)#208 AS sum(number_sales)#213] -(179) Union +(177) Union -(180) HashAggregate [codegen id : 328] +(178) HashAggregate [codegen id : 328] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Keys [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Functions: [] Aggregate Attributes: [] Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] -(181) Exchange +(179) Exchange Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85, 5), true, [id=#214] -(182) HashAggregate [codegen id : 329] +(180) HashAggregate [codegen id : 329] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Keys [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Functions: [] Aggregate Attributes: [] Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] -(183) ReusedExchange [Reuses operator id: 74] +(181) ReusedExchange [Reuses operator id: 72] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#215, isEmpty#216, count#217] -(184) HashAggregate [codegen id : 355] +(182) HashAggregate [codegen id : 355] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#215, isEmpty#216, count#217] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#218, count(1)#219] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#218 AS sales#39, count(1)#219 AS number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#218 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#220] -(185) Filter [codegen id : 355] +(183) Filter [codegen id : 355] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#220] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#220) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#220 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(186) Project [codegen id : 355] +(184) Project [codegen id : 355] Output [6]: [store AS channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#220] -(187) ReusedExchange [Reuses operator id: 90] +(185) ReusedExchange [Reuses operator id: 88] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#221, isEmpty#222, count#223] -(188) HashAggregate [codegen id : 381] +(186) HashAggregate [codegen id : 381] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#221, isEmpty#222, count#223] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#224, count(1)#225] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#224 AS sales#56, count(1)#225 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#224 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#226] -(189) Filter [codegen id : 381] +(187) Filter [codegen id : 381] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#226] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#226) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#226 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(190) Project [codegen id : 381] +(188) Project [codegen id : 381] Output [6]: [catalog AS channel#227, i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#226] -(191) ReusedExchange [Reuses operator id: 106] +(189) ReusedExchange [Reuses operator id: 104] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#228, isEmpty#229, count#230] -(192) HashAggregate [codegen id : 407] +(190) HashAggregate [codegen id : 407] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#228, isEmpty#229, count#230] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#231, count(1)#232] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#231 AS sales#71, count(1)#232 AS number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#231 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#233] -(193) Filter [codegen id : 407] +(191) Filter [codegen id : 407] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#233] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#233) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#233 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(194) Project [codegen id : 407] +(192) Project [codegen id : 407] Output [6]: [web AS channel#234, i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#233] -(195) Union +(193) Union -(196) HashAggregate [codegen id : 408] +(194) HashAggregate [codegen id : 408] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [partial_sum(sales#39), partial_sum(number_sales#40)] Aggregate Attributes [3]: [sum#235, isEmpty#236, sum#237] Results [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#238, isEmpty#239, sum#240] -(197) Exchange +(195) Exchange Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#238, isEmpty#239, sum#240] Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, 5), true, [id=#241] -(198) HashAggregate [codegen id : 409] +(196) HashAggregate [codegen id : 409] Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#238, isEmpty#239, sum#240] Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(sales#39), sum(number_sales#40)] Aggregate Attributes [2]: [sum(sales#39)#242, sum(number_sales#40)#243] Results [2]: [sum(sales#39)#242 AS sum_sales#84, sum(number_sales#40)#243 AS number_sales#85] -(199) HashAggregate [codegen id : 409] +(197) HashAggregate [codegen id : 409] Input [2]: [sum_sales#84, number_sales#85] Keys: [] Functions [2]: [partial_sum(sum_sales#84), partial_sum(number_sales#85)] Aggregate Attributes [3]: [sum#244, isEmpty#245, sum#246] Results [3]: [sum#247, isEmpty#248, sum#249] -(200) Exchange +(198) Exchange Input [3]: [sum#247, isEmpty#248, sum#249] Arguments: SinglePartition, true, [id=#250] -(201) HashAggregate [codegen id : 410] +(199) HashAggregate [codegen id : 410] Input [3]: [sum#247, isEmpty#248, sum#249] Keys: [] Functions [2]: [sum(sum_sales#84), sum(number_sales#85)] Aggregate Attributes [2]: [sum(sum_sales#84)#251, sum(number_sales#85)#252] Results [6]: [null AS channel#253, null AS i_brand_id#254, null AS i_class_id#255, null AS i_category_id#256, sum(sum_sales#84)#251 AS sum(sum_sales)#257, sum(number_sales#85)#252 AS sum(number_sales)#258] -(202) Union +(200) Union -(203) HashAggregate [codegen id : 411] +(201) HashAggregate [codegen id : 411] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Keys [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Functions: [] Aggregate Attributes: [] Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] -(204) Exchange +(202) Exchange Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85, 5), true, [id=#259] -(205) HashAggregate [codegen id : 412] +(203) HashAggregate [codegen id : 412] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Keys [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Functions: [] Aggregate Attributes: [] Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] -(206) TakeOrderedAndProject +(204) TakeOrderedAndProject Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Arguments: 100, [channel#44 ASC NULLS FIRST, i_brand_id#6 ASC NULLS FIRST, i_class_id#7 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST], [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#42, [id=#43] -* HashAggregate (236) -+- Exchange (235) - +- * HashAggregate (234) - +- Union (233) - :- * Project (216) - : +- * BroadcastHashJoin Inner BuildRight (215) - : :- * Filter (209) - : : +- * ColumnarToRow (208) - : : +- Scan parquet default.store_sales (207) - : +- BroadcastExchange (214) - : +- * Project (213) - : +- * Filter (212) - : +- * ColumnarToRow (211) - : +- Scan parquet default.date_dim (210) - :- * Project (226) - : +- * BroadcastHashJoin Inner BuildRight (225) - : :- * Filter (219) - : : +- * ColumnarToRow (218) - : : +- Scan parquet default.catalog_sales (217) - : +- BroadcastExchange (224) - : +- * Project (223) - : +- * Filter (222) - : +- * ColumnarToRow (221) - : +- Scan parquet default.date_dim (220) - +- * Project (232) - +- * BroadcastHashJoin Inner BuildRight (231) - :- * Filter (229) - : +- * ColumnarToRow (228) - : +- Scan parquet default.web_sales (227) - +- ReusedExchange (230) - - -(207) Scan parquet default.store_sales +Subquery:1 Hosting operator id = 74 Hosting Expression = Subquery scalar-subquery#42, [id=#43] +* HashAggregate (234) ++- Exchange (233) + +- * HashAggregate (232) + +- Union (231) + :- * Project (214) + : +- * BroadcastHashJoin Inner BuildRight (213) + : :- * Filter (207) + : : +- * ColumnarToRow (206) + : : +- Scan parquet default.store_sales (205) + : +- BroadcastExchange (212) + : +- * Project (211) + : +- * Filter (210) + : +- * ColumnarToRow (209) + : +- Scan parquet default.date_dim (208) + :- * Project (224) + : +- * BroadcastHashJoin Inner BuildRight (223) + : :- * Filter (217) + : : +- * ColumnarToRow (216) + : : +- Scan parquet default.catalog_sales (215) + : +- BroadcastExchange (222) + : +- * Project (221) + : +- * Filter (220) + : +- * ColumnarToRow (219) + : +- Scan parquet default.date_dim (218) + +- * Project (230) + +- * BroadcastHashJoin Inner BuildRight (229) + :- * Filter (227) + : +- * ColumnarToRow (226) + : +- Scan parquet default.web_sales (225) + +- ReusedExchange (228) + + +(205) Scan parquet default.store_sales Output [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(208) ColumnarToRow [codegen id : 2] +(206) ColumnarToRow [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] -(209) Filter [codegen id : 2] +(207) Filter [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Condition : isnotnull(ss_sold_date_sk#1) -(210) Scan parquet default.date_dim +(208) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_year#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(211) ColumnarToRow [codegen id : 1] +(209) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] -(212) Filter [codegen id : 1] +(210) Filter [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] Condition : (((isnotnull(d_year#11) AND (d_year#11 >= 1999)) AND (d_year#11 <= 2001)) AND isnotnull(d_date_sk#10)) -(213) Project [codegen id : 1] +(211) Project [codegen id : 1] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_year#11] -(214) BroadcastExchange +(212) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#260] -(215) BroadcastHashJoin [codegen id : 2] +(213) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(216) Project [codegen id : 2] +(214) Project [codegen id : 2] Output [2]: [ss_quantity#3 AS quantity#261, ss_list_price#4 AS list_price#262] Input [4]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(217) Scan parquet default.catalog_sales +(215) Scan parquet default.catalog_sales Output [3]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(218) ColumnarToRow [codegen id : 4] +(216) ColumnarToRow [codegen id : 4] Input [3]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46] -(219) Filter [codegen id : 4] +(217) Filter [codegen id : 4] Input [3]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46] Condition : isnotnull(cs_sold_date_sk#16) -(220) Scan parquet default.date_dim +(218) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_year#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(221) ColumnarToRow [codegen id : 3] +(219) ColumnarToRow [codegen id : 3] Input [2]: [d_date_sk#10, d_year#11] -(222) Filter [codegen id : 3] +(220) Filter [codegen id : 3] Input [2]: [d_date_sk#10, d_year#11] Condition : (((isnotnull(d_year#11) AND (d_year#11 >= 1998)) AND (d_year#11 <= 2000)) AND isnotnull(d_date_sk#10)) -(223) Project [codegen id : 3] +(221) Project [codegen id : 3] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_year#11] -(224) BroadcastExchange +(222) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#263] -(225) BroadcastHashJoin [codegen id : 4] +(223) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#16] Right keys [1]: [d_date_sk#10] Join condition: None -(226) Project [codegen id : 4] +(224) Project [codegen id : 4] Output [2]: [cs_quantity#45 AS quantity#264, cs_list_price#46 AS list_price#265] Input [4]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46, d_date_sk#10] -(227) Scan parquet default.web_sales +(225) Scan parquet default.web_sales Output [3]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(228) ColumnarToRow [codegen id : 6] +(226) ColumnarToRow [codegen id : 6] Input [3]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61] -(229) Filter [codegen id : 6] +(227) Filter [codegen id : 6] Input [3]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61] Condition : isnotnull(ws_sold_date_sk#20) -(230) ReusedExchange [Reuses operator id: 224] +(228) ReusedExchange [Reuses operator id: 222] Output [1]: [d_date_sk#10] -(231) BroadcastHashJoin [codegen id : 6] +(229) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#20] Right keys [1]: [d_date_sk#10] Join condition: None -(232) Project [codegen id : 6] +(230) Project [codegen id : 6] Output [2]: [ws_quantity#60 AS quantity#266, ws_list_price#61 AS list_price#267] Input [4]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61, d_date_sk#10] -(233) Union +(231) Union -(234) HashAggregate [codegen id : 7] +(232) HashAggregate [codegen id : 7] Input [2]: [quantity#261, list_price#262] Keys: [] Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [2]: [sum#268, count#269] Results [2]: [sum#270, count#271] -(235) Exchange +(233) Exchange Input [2]: [sum#270, count#271] Arguments: SinglePartition, true, [id=#272] -(236) HashAggregate [codegen id : 8] +(234) HashAggregate [codegen id : 8] Input [2]: [sum#270, count#271] Keys: [] Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#273] Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#273 AS average_sales#274] -Subquery:2 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:2 Hosting operator id = 90 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:3 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:3 Hosting operator id = 106 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:4 Hosting operator id = 116 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:4 Hosting operator id = 114 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:5 Hosting operator id = 120 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:5 Hosting operator id = 118 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:6 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:6 Hosting operator id = 122 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:7 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:7 Hosting operator id = 137 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:8 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:8 Hosting operator id = 141 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:9 Hosting operator id = 147 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:9 Hosting operator id = 145 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:10 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:10 Hosting operator id = 160 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:11 Hosting operator id = 166 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:11 Hosting operator id = 164 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:12 Hosting operator id = 170 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:12 Hosting operator id = 168 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:13 Hosting operator id = 185 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:13 Hosting operator id = 183 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:14 Hosting operator id = 189 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:14 Hosting operator id = 187 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:15 Hosting operator id = 193 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:15 Hosting operator id = 191 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt index e96f1d6fed14..2491c579e1d2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt @@ -117,72 +117,70 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastExchange #8 WholeStageCodegen (10) HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #9 - WholeStageCodegen (9) - HashAggregate [brand_id,class_id,category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] + InputAdapter + Exchange [brand_id,class_id,category_id] #9 + WholeStageCodegen (9) + HashAggregate [brand_id,class_id,category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk,ss_sold_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #10 - WholeStageCodegen (1) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] InputAdapter - BroadcastExchange #11 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] + BroadcastExchange #10 + WholeStageCodegen (1) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter - BroadcastExchange #12 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] + BroadcastExchange #11 + WholeStageCodegen (2) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (5) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk,cs_sold_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #13 - WholeStageCodegen (3) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] InputAdapter - ReusedExchange [d_date_sk] #11 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (8) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk,ws_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] + BroadcastExchange #13 + WholeStageCodegen (3) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [d_date_sk] #11 + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (8) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk,ws_sold_date_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] InputAdapter - ReusedExchange [d_date_sk] #11 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + InputAdapter + ReusedExchange [d_date_sk] #11 InputAdapter BroadcastExchange #15 WholeStageCodegen (23) From 29701dc197fc552c43fe60476b6fa5ed2fef0a95 Mon Sep 17 00:00:00 2001 From: "tanel.kiis@gmail.com" Date: Wed, 14 Oct 2020 21:27:35 +0300 Subject: [PATCH 02/22] Clearer naming --- .../spark/sql/catalyst/optimizer/Optimizer.scala | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) 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 1b22a57ecc16..433c4b308fa8 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 @@ -484,15 +484,16 @@ object RemoveRedundantAliases extends Rule[LogicalPlan] { */ object RemoveRedundantAggregates extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { - case upper @ Aggregate(_, _, lower: Aggregate) if isRedundant(upper, lower) => + case upper @ Aggregate(_, _, lower: Aggregate) if lowerIsRedundant(upper, lower) => upper.copy(child = lower.child) } - private def isRedundant(upper: Aggregate, lower: Aggregate): Boolean = { - val referencesOnlyGrouping = upper.references.subsetOf(AttributeSet(lower.groupingExpressions)) - val hasAggregateExpressions = upper.aggregateExpressions + private def lowerIsRedundant(upper: Aggregate, lower: Aggregate): Boolean = { + val upperReferencesOnlyGrouping = upper.references + .subsetOf(AttributeSet(lower.groupingExpressions)) + val upperHasAggregateExpressions = upper.aggregateExpressions .exists(_.find(_.isInstanceOf[AggregateExpression]).nonEmpty) - referencesOnlyGrouping && !hasAggregateExpressions + upperReferencesOnlyGrouping && !upperHasAggregateExpressions } } From 4ce064480be88119f3cfe880b965f1665af67bae Mon Sep 17 00:00:00 2001 From: "tanel.kiis@gmail.com" Date: Wed, 14 Oct 2020 22:01:19 +0300 Subject: [PATCH 03/22] Clearer naming --- .../org/apache/spark/sql/catalyst/optimizer/Optimizer.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 433c4b308fa8..3b9d85f7542f 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 @@ -491,9 +491,9 @@ object RemoveRedundantAggregates extends Rule[LogicalPlan] { private def lowerIsRedundant(upper: Aggregate, lower: Aggregate): Boolean = { val upperReferencesOnlyGrouping = upper.references .subsetOf(AttributeSet(lower.groupingExpressions)) - val upperHasAggregateExpressions = upper.aggregateExpressions - .exists(_.find(_.isInstanceOf[AggregateExpression]).nonEmpty) - upperReferencesOnlyGrouping && !upperHasAggregateExpressions + val upperHasNoAggregateExpressions = upper.aggregateExpressions + .forall(_.find(_.isInstanceOf[AggregateExpression]).isEmpty) + upperReferencesOnlyGrouping && upperHasNoAggregateExpressions } } From ef64abfdd8916504f25e9ff5f2c7f13bcb50752d Mon Sep 17 00:00:00 2001 From: Tanel Kiis Date: Fri, 16 Oct 2020 15:00:11 +0300 Subject: [PATCH 04/22] Handle aliases --- .../sql/catalyst/expressions/predicates.scala | 49 +++++++++++---- .../sql/catalyst/optimizer/Optimizer.scala | 62 ++++++++++++------- .../optimizer/PushDownLeftSemiAntiJoin.scala | 4 +- .../RemoveRedundantAggregatesSuite.scala | 27 +++++++- 4 files changed, 102 insertions(+), 40 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 1f55045dbca7..46f61e700c45 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -97,7 +97,7 @@ object Predicate extends CodeGeneratorWithInterpretedFallback[Expression, BasePr } } -trait PredicateHelper extends Logging { +trait PredicateHelper extends Logging with AliasHelper { protected def splitConjunctivePredicates(condition: Expression): Seq[Expression] = { condition match { case And(cond1, cond2) => @@ -150,18 +150,6 @@ trait PredicateHelper extends Logging { } } - // Substitute any known alias from a map. - protected def replaceAlias( - condition: Expression, - aliases: AttributeMap[Expression]): Expression = { - // Use transformUp to prevent infinite recursion when the replacement expression - // redefines the same ExprId, - condition.transformUp { - case a: Attribute => - aliases.getOrElse(a, a) - } - } - /** * Returns true if `expr` can be evaluated using only the output of `plan`. This method * can be used to determine when it is acceptable to move expression evaluation within a query @@ -249,6 +237,41 @@ trait PredicateHelper extends Logging { } } +/** + * Helper methods for collecting and replacing aliases. + */ +trait AliasHelper { + + protected def getAliasMap(plan: Project): AttributeMap[Expression] = { + // Create a map of Aliases to their values from the child projection. + // e.g., 'SELECT a + b AS c, d ...' produces Map(c -> a + b). + AttributeMap(plan.projectList.collect { case a: Alias => (a.toAttribute, a.child) }) + } + + protected def getAliasMap(plan: Aggregate): AttributeMap[Expression] = { + // Find all the aliased expressions in the aggregate list that don't include any actual + // AggregateExpression or PythonUDF, and create a map from the alias to the expression + val aliasMap = plan.aggregateExpressions.collect { + case a: Alias if a.child.find(e => e.isInstanceOf[AggregateExpression] || + PythonUDF.isGroupedAggPandasUDF(e)).isEmpty => + (a.toAttribute, a.child) + } + AttributeMap(aliasMap) + } + + // Substitute any known alias from a map. + protected def replaceAlias( + condition: Expression, + aliases: AttributeMap[Expression]): Expression = { + // Use transformUp to prevent infinite recursion when the replacement expression + // redefines the same ExprId, + condition.transformUp { + case a: Attribute => + aliases.getOrElse(a, a) + } + } +} + @ExpressionDescription( usage = "_FUNC_ expr - Logical not.", examples = """ 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 3b9d85f7542f..72f602c8ede7 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 @@ -482,18 +482,51 @@ object RemoveRedundantAliases extends Rule[LogicalPlan] { * Remove redundant aggregates from a query plan. A redundant aggregate is an aggregate whose * only goal is to keep distinct values, while its parent aggregate would ignore duplicate values. */ -object RemoveRedundantAggregates extends Rule[LogicalPlan] { +object RemoveRedundantAggregates extends Rule[LogicalPlan] with AliasHelper { def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { case upper @ Aggregate(_, _, lower: Aggregate) if lowerIsRedundant(upper, lower) => - upper.copy(child = lower.child) + val aliasMap = getAliasMap(lower) + upper.copy( + child = lower.child, + groupingExpressions = upper.groupingExpressions.map(replaceAlias(_, aliasMap)), + aggregateExpressions = upper.aggregateExpressions.map( + replaceAliasButKeepOuter(_, aliasMap)) + ) } private def lowerIsRedundant(upper: Aggregate, lower: Aggregate): Boolean = { - val upperReferencesOnlyGrouping = upper.references - .subsetOf(AttributeSet(lower.groupingExpressions)) + val isDeterministic = upper.aggregateExpressions.forall(_.deterministic) && + lower.aggregateExpressions.forall(_.deterministic) + + val upperReferencesOnlyGrouping = upper.references.subsetOf(AttributeSet( + lower.aggregateExpressions.filter(!isAggregate(_)).map(_.toAttribute))) + val upperHasNoAggregateExpressions = upper.aggregateExpressions - .forall(_.find(_.isInstanceOf[AggregateExpression]).isEmpty) - upperReferencesOnlyGrouping && upperHasNoAggregateExpressions + .forall(_.find(isAggregate).isEmpty) + + isDeterministic && upperReferencesOnlyGrouping && upperHasNoAggregateExpressions + } + + private def isAggregate(expr: Expression): Boolean = { + expr.find(e => e.isInstanceOf[AggregateExpression] || + PythonUDF.isGroupedAggPandasUDF(e)).isDefined + } + + /** + * Replace all attributes, that reference an alias, with the aliased expression, + * but keep the name of the name of the outmost attribute. + */ + private def replaceAliasButKeepOuter( + expr: NamedExpression, + aliasMap: AttributeMap[Expression]): NamedExpression = { + + val replaced = expr match { + case a: Attribute if aliasMap.contains(a) => + Alias(replaceAlias(a, aliasMap), a.name)(a.exprId, a.qualifier) + case _ => replaceAlias(expr, aliasMap) + } + + replaced.asInstanceOf[NamedExpression] } } @@ -1258,23 +1291,6 @@ object PushPredicateThroughNonJoin extends Rule[LogicalPlan] with PredicateHelpe } } - def getAliasMap(plan: Project): AttributeMap[Expression] = { - // Create a map of Aliases to their values from the child projection. - // e.g., 'SELECT a + b AS c, d ...' produces Map(c -> a + b). - AttributeMap(plan.projectList.collect { case a: Alias => (a.toAttribute, a.child) }) - } - - def getAliasMap(plan: Aggregate): AttributeMap[Expression] = { - // Find all the aliased expressions in the aggregate list that don't include any actual - // AggregateExpression or PythonUDF, and create a map from the alias to the expression - val aliasMap = plan.aggregateExpressions.collect { - case a: Alias if a.child.find(e => e.isInstanceOf[AggregateExpression] || - PythonUDF.isGroupedAggPandasUDF(e)).isEmpty => - (a.toAttribute, a.child) - } - AttributeMap(aliasMap) - } - def canPushThrough(p: UnaryNode): Boolean = p match { // Note that some operators (e.g. project, aggregate, union) are being handled separately // (earlier in this rule). diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushDownLeftSemiAntiJoin.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushDownLeftSemiAntiJoin.scala index 606db85fcdea..50fe0192d6f2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushDownLeftSemiAntiJoin.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushDownLeftSemiAntiJoin.scala @@ -42,7 +42,7 @@ object PushDownLeftSemiAntiJoin extends Rule[LogicalPlan] with PredicateHelper { // No join condition, just push down the Join below Project p.copy(child = Join(gChild, rightOp, joinType, joinCond, hint)) } else { - val aliasMap = PushPredicateThroughNonJoin.getAliasMap(p) + val aliasMap = getAliasMap(p) val newJoinCond = if (aliasMap.nonEmpty) { Option(replaceAlias(joinCond.get, aliasMap)) } else { @@ -55,7 +55,7 @@ object PushDownLeftSemiAntiJoin extends Rule[LogicalPlan] with PredicateHelper { case join @ Join(agg: Aggregate, rightOp, LeftSemiOrAnti(_), _, _) if agg.aggregateExpressions.forall(_.deterministic) && agg.groupingExpressions.nonEmpty && !agg.aggregateExpressions.exists(ScalarSubquery.hasCorrelatedScalarSubquery) => - val aliasMap = PushPredicateThroughNonJoin.getAliasMap(agg) + val aliasMap = getAliasMap(agg) val canPushDownPredicate = (predicate: Expression) => { val replaced = replaceAlias(predicate, aliasMap) predicate.references.nonEmpty && diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala index 50b0451e9ad9..050c5230c01a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala @@ -70,14 +70,37 @@ class RemoveRedundantAggregatesSuite extends PlanTest { comparePlans(optimized, expected) } - test("Keep non-redundant aggregate") { + test("Remove redundant aggregate with aliases") { val relation = LocalRelation('a.int, 'b.int) val query = relation - .groupBy('a)('a, first('b) as 'b) + .groupBy('a + 'b)(('a + 'b) as 'c, count('b)) + .groupBy('c)('c) + .analyze + val expected = relation + .groupBy('a + 'b)(('a + 'b) as 'c) + .analyze + val optimized = Optimize.execute(query) + comparePlans(optimized, expected) + } + + test("Keep non-redundant aggregate - upper has agg expression") { + val relation = LocalRelation('a.int, 'b.int) + val query = relation + .groupBy('a, 'b)('a, 'b) // The count would change if we remove the first aggregate .groupBy('a)('a, count('b)) .analyze val optimized = Optimize.execute(query) comparePlans(optimized, query) } + + test("Keep non-redundant aggregate - upper references non-grouping") { + val relation = LocalRelation('a.int, 'b.int) + val query = relation + .groupBy('a)('a, count('b) as 'c) + .groupBy('c)('c) + .analyze + val optimized = Optimize.execute(query) + comparePlans(optimized, query) + } } From 4bf08bbaba3dc8c7431c405d705629a580da673e Mon Sep 17 00:00:00 2001 From: "tanel.kiis@gmail.com" Date: Fri, 16 Oct 2020 17:14:39 +0300 Subject: [PATCH 05/22] UTs for non-deterministic cases --- .../RemoveRedundantAggregatesSuite.scala | 20 +++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala index 050c5230c01a..f97ac01b9c85 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala @@ -103,4 +103,24 @@ class RemoveRedundantAggregatesSuite extends PlanTest { val optimized = Optimize.execute(query) comparePlans(optimized, query) } + + test("Keep non-redundant aggregate - lower is non-deterministic") { + val relation = LocalRelation('a.int, 'b.int) + val query = relation + .groupBy('a, 'b)('a, 'b + rand(0)) + .groupBy('a)('a) + .analyze + val optimized = Optimize.execute(query) + comparePlans(optimized, query) + } + + test("Keep non-redundant aggregate - upper is non-deterministic") { + val relation = LocalRelation('a.int, 'b.int) + val query = relation + .groupBy('a, 'b)('a, 'b) + .groupBy('a)('a + rand(0)) + .analyze + val optimized = Optimize.execute(query) + comparePlans(optimized, query) + } } From 832ff021d8ddd2c9de7edd019c022cea53876dfc Mon Sep 17 00:00:00 2001 From: "tanel.kiis@gmail.com" Date: Wed, 21 Oct 2020 06:23:56 +0300 Subject: [PATCH 06/22] Extract alias helper --- .../catalyst/expressions/AliasHelper.scala | 56 +++++++++++++++++++ .../sql/catalyst/expressions/predicates.scala | 35 ------------ 2 files changed, 56 insertions(+), 35 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AliasHelper.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AliasHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AliasHelper.scala new file mode 100644 index 000000000000..aa5dccf04a66 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AliasHelper.scala @@ -0,0 +1,56 @@ +/* + * 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.expressions + +import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression +import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Project} + +/** + * Helper methods for collecting and replacing aliases. + */ +trait AliasHelper { + + protected def getAliasMap(plan: Project): AttributeMap[Expression] = { + // Create a map of Aliases to their values from the child projection. + // e.g., 'SELECT a + b AS c, d ...' produces Map(c -> a + b). + AttributeMap(plan.projectList.collect { case a: Alias => (a.toAttribute, a.child) }) + } + + protected def getAliasMap(plan: Aggregate): AttributeMap[Expression] = { + // Find all the aliased expressions in the aggregate list that don't include any actual + // AggregateExpression or PythonUDF, and create a map from the alias to the expression + val aliasMap = plan.aggregateExpressions.collect { + case a: Alias if a.child.find(e => e.isInstanceOf[AggregateExpression] || + PythonUDF.isGroupedAggPandasUDF(e)).isEmpty => + (a.toAttribute, a.child) + } + AttributeMap(aliasMap) + } + + // Substitute any known alias from a map. + protected def replaceAlias( + condition: Expression, + aliases: AttributeMap[Expression]): Expression = { + // Use transformUp to prevent infinite recursion when the replacement expression + // redefines the same ExprId, + condition.transformUp { + case a: Attribute => + aliases.getOrElse(a, a) + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 46f61e700c45..9d50038e56b0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -237,41 +237,6 @@ trait PredicateHelper extends Logging with AliasHelper { } } -/** - * Helper methods for collecting and replacing aliases. - */ -trait AliasHelper { - - protected def getAliasMap(plan: Project): AttributeMap[Expression] = { - // Create a map of Aliases to their values from the child projection. - // e.g., 'SELECT a + b AS c, d ...' produces Map(c -> a + b). - AttributeMap(plan.projectList.collect { case a: Alias => (a.toAttribute, a.child) }) - } - - protected def getAliasMap(plan: Aggregate): AttributeMap[Expression] = { - // Find all the aliased expressions in the aggregate list that don't include any actual - // AggregateExpression or PythonUDF, and create a map from the alias to the expression - val aliasMap = plan.aggregateExpressions.collect { - case a: Alias if a.child.find(e => e.isInstanceOf[AggregateExpression] || - PythonUDF.isGroupedAggPandasUDF(e)).isEmpty => - (a.toAttribute, a.child) - } - AttributeMap(aliasMap) - } - - // Substitute any known alias from a map. - protected def replaceAlias( - condition: Expression, - aliases: AttributeMap[Expression]): Expression = { - // Use transformUp to prevent infinite recursion when the replacement expression - // redefines the same ExprId, - condition.transformUp { - case a: Attribute => - aliases.getOrElse(a, a) - } - } -} - @ExpressionDescription( usage = "_FUNC_ expr - Logical not.", examples = """ From 6cdc43a5c4ed6de961ed40cd0784178465d13232 Mon Sep 17 00:00:00 2001 From: Tanel Kiis Date: Wed, 21 Oct 2020 08:59:24 +0300 Subject: [PATCH 07/22] cleanup --- .../org/apache/spark/sql/catalyst/optimizer/Optimizer.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 b3246b123d8a..f707064380c0 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 @@ -488,7 +488,7 @@ object RemoveRedundantAggregates extends Rule[LogicalPlan] with AliasHelper { child = lower.child, groupingExpressions = upper.groupingExpressions.map(replaceAlias(_, aliasMap)), aggregateExpressions = upper.aggregateExpressions.map( - replaceAliasButKeepOuter(_, aliasMap)) + replaceAliasButKeepName(_, aliasMap)) ) } @@ -512,9 +512,9 @@ object RemoveRedundantAggregates extends Rule[LogicalPlan] with AliasHelper { /** * Replace all attributes, that reference an alias, with the aliased expression, - * but keep the name of the name of the outmost attribute. + * but keep the name of the outmost attribute. */ - private def replaceAliasButKeepOuter( + private def replaceAliasButKeepName( expr: NamedExpression, aliasMap: AttributeMap[Expression]): NamedExpression = { From 2d63bb48a45abcb0efd9d9fa1d6a322de96b896f Mon Sep 17 00:00:00 2001 From: "tanel.kiis@gmail.com" Date: Thu, 22 Oct 2020 03:54:29 +0300 Subject: [PATCH 08/22] Extract AliasHelper --- .../sql/catalyst/analysis/Analyzer.scala | 43 +++--------- .../catalyst/expressions/AliasHelper.scala | 69 ++++++++++++++++--- .../sql/catalyst/expressions/predicates.scala | 19 ++--- .../sql/catalyst/optimizer/Optimizer.scala | 51 ++------------ .../sql/catalyst/optimizer/subquery.scala | 4 +- 5 files changed, 83 insertions(+), 103 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 0ba150ec1efb..ff4dfa9b3411 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -2193,7 +2193,7 @@ class Analyzer( * those in a HAVING clause or ORDER BY clause. These expressions are pushed down to the * underlying aggregate operator and then projected away after the original operator. */ - object ResolveAggregateFunctions extends Rule[LogicalPlan] { + object ResolveAggregateFunctions extends Rule[LogicalPlan] with AliasHelper { def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { // Resolve aggregate with having clause to Filter(..., Aggregate()). Note, to avoid wrongly // resolve the having condition expression, here we skip resolving it in ResolveReferences @@ -2226,8 +2226,7 @@ class Analyzer( // Aggregate. checkAnalysis(resolvedAggregate) - val originalAggExprs = aggregate.aggregateExpressions.map( - CleanupAliases.trimNonTopLevelAliases(_).asInstanceOf[NamedExpression]) + val originalAggExprs = aggregate.aggregateExpressions.map(trimNonTopLevelAliases) // If the ordering expression is same with original aggregate expression, we don't need // to push down this ordering expression and can reference the original aggregate @@ -2370,7 +2369,7 @@ class Analyzer( case _ => false }.isDefined } } - CleanupAliases.trimNonTopLevelAliases(expr) match { + trimNonTopLevelAliases(expr) match { case UnresolvedAlias(g: Generator, _) => hasInnerGenerator(g) case Alias(g: Generator, _) => hasInnerGenerator(g) case MultiAlias(g: Generator, _) => hasInnerGenerator(g) @@ -2440,7 +2439,7 @@ class Analyzer( val projectExprs = Array.ofDim[NamedExpression](aggList.length) val newAggList = aggList - .map(CleanupAliases.trimNonTopLevelAliases(_).asInstanceOf[NamedExpression]) + .map(trimNonTopLevelAliases) .zipWithIndex .flatMap { case (AliasedGenerator(generator, names, outer), idx) => @@ -2483,7 +2482,7 @@ class Analyzer( var resolvedGenerator: Generate = null val newProjectList = projectList - .map(CleanupAliases.trimNonTopLevelAliases(_).asInstanceOf[NamedExpression]) + .map(trimNonTopLevelAliases) .flatMap { case AliasedGenerator(generator, names, outer) if generator.childrenResolved => // It's a sanity check, this should not happen as the previous case will throw @@ -3495,45 +3494,23 @@ object EliminateUnions extends Rule[LogicalPlan] { * are not in its `children`, e.g. `RuntimeReplaceable`, the transformation for Aliases in this * rule can't work for those parameters. */ -object CleanupAliases extends Rule[LogicalPlan] { - def trimAliases(e: Expression): Expression = { - e.transformDown { - case Alias(child, _) => child - case MultiAlias(child, _) => child - } - } - - def trimNonTopLevelAliases(e: Expression): Expression = e match { - case a: Alias => - a.copy(child = trimAliases(a.child))( - exprId = a.exprId, - qualifier = a.qualifier, - explicitMetadata = Some(a.metadata)) - case a: MultiAlias => - a.copy(child = trimAliases(a.child)) - case other => trimAliases(other) - } - +object CleanupAliases extends Rule[LogicalPlan] with AliasHelper { override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { case Project(projectList, child) => - val cleanedProjectList = - projectList.map(trimNonTopLevelAliases(_).asInstanceOf[NamedExpression]) + val cleanedProjectList = projectList.map(trimNonTopLevelAliases) Project(cleanedProjectList, child) case Aggregate(grouping, aggs, child) => - val cleanedAggs = aggs.map(trimNonTopLevelAliases(_).asInstanceOf[NamedExpression]) + val cleanedAggs = aggs.map(trimNonTopLevelAliases) Aggregate(grouping.map(trimAliases), cleanedAggs, child) case Window(windowExprs, partitionSpec, orderSpec, child) => - val cleanedWindowExprs = - windowExprs.map(e => trimNonTopLevelAliases(e).asInstanceOf[NamedExpression]) + val cleanedWindowExprs = windowExprs.map(trimNonTopLevelAliases) Window(cleanedWindowExprs, partitionSpec.map(trimAliases), orderSpec.map(trimAliases(_).asInstanceOf[SortOrder]), child) case CollectMetrics(name, metrics, child) => - val cleanedMetrics = metrics.map { - e => trimNonTopLevelAliases(e).asInstanceOf[NamedExpression] - } + val cleanedMetrics = metrics.map(trimNonTopLevelAliases) CollectMetrics(name, cleanedMetrics, child) // Operators that operate on objects should only have expressions from encoders, which should diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AliasHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AliasHelper.scala index aa5dccf04a66..8eb0a4227d9f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AliasHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AliasHelper.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.expressions +import org.apache.spark.sql.catalyst.analysis.MultiAlias import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Project} @@ -25,32 +26,78 @@ import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Project} */ trait AliasHelper { - protected def getAliasMap(plan: Project): AttributeMap[Expression] = { + protected def getAliasMap(plan: Project): AttributeMap[Alias] = { // Create a map of Aliases to their values from the child projection. - // e.g., 'SELECT a + b AS c, d ...' produces Map(c -> a + b). - AttributeMap(plan.projectList.collect { case a: Alias => (a.toAttribute, a.child) }) + // e.g., 'SELECT a + b AS c, d ...' produces Map(c -> Alias(a + b, c)). + getAliasMap(plan.projectList) } - protected def getAliasMap(plan: Aggregate): AttributeMap[Expression] = { + protected def getAliasMap(plan: Aggregate): AttributeMap[Alias] = { // Find all the aliased expressions in the aggregate list that don't include any actual // AggregateExpression or PythonUDF, and create a map from the alias to the expression val aliasMap = plan.aggregateExpressions.collect { case a: Alias if a.child.find(e => e.isInstanceOf[AggregateExpression] || PythonUDF.isGroupedAggPandasUDF(e)).isEmpty => - (a.toAttribute, a.child) + (a.toAttribute, a) } AttributeMap(aliasMap) } - // Substitute any known alias from a map. + protected def getAliasMap(exprs: Seq[NamedExpression]): AttributeMap[Alias] = { + // Create a map of Aliases to their values from the child projection. + // e.g., 'SELECT a + b AS c, d ...' produces Map(c -> Alias(a + b, c)). + AttributeMap(exprs.collect { case a: Alias => (a.toAttribute, a) }) + } + + /** + * Replace all attributes, that reference an alias, with the aliased expression + */ protected def replaceAlias( - condition: Expression, - aliases: AttributeMap[Expression]): Expression = { + expr: Expression, + aliases: AttributeMap[Alias]): Expression = { // Use transformUp to prevent infinite recursion when the replacement expression // redefines the same ExprId, - condition.transformUp { - case a: Attribute => - aliases.getOrElse(a, a) + expr.transformUp { + case a: Attribute => aliases.get(a).map(_.child).getOrElse(a) + } + } + + /** + * Replace all attributes, that reference an alias, with the aliased expression, + * but keep the name of the outmost attribute. + */ + protected def replaceAliasButKeepName( + expr: NamedExpression, + aliasMap: AttributeMap[Alias]): NamedExpression = { + + val replaced = expr match { + case a: Attribute if aliasMap.contains(a) => + Alias(replaceAlias(a, aliasMap), a.name)(a.exprId, a.qualifier) + case _ => replaceAlias(expr, aliasMap) } + + replaced.asInstanceOf[NamedExpression] + } + + protected def trimAliases(e: Expression): Expression = { + e.transformDown { + case Alias(child, _) => child + case MultiAlias(child, _) => child + } + } + + protected def trimNonTopLevelAliases(e: Expression): Expression = e match { + case a: Alias => + a.copy(child = trimAliases(a.child))( + exprId = a.exprId, + qualifier = a.qualifier, + explicitMetadata = Some(a.metadata)) + case a: MultiAlias => + a.copy(child = trimAliases(a.child)) + case other => trimAliases(other) + } + + protected def trimNonTopLevelAliases(e: NamedExpression): NamedExpression = { + trimNonTopLevelAliases(e.asInstanceOf[Expression]).asInstanceOf[NamedExpression] } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 9d50038e56b0..f440534745ba 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -97,7 +97,7 @@ object Predicate extends CodeGeneratorWithInterpretedFallback[Expression, BasePr } } -trait PredicateHelper extends Logging with AliasHelper { +trait PredicateHelper extends AliasHelper with Logging { protected def splitConjunctivePredicates(condition: Expression): Seq[Expression] = { condition match { case And(cond1, cond2) => @@ -117,18 +117,13 @@ trait PredicateHelper extends Logging with AliasHelper { plan: LogicalPlan): Option[(Expression, LogicalPlan)] = { plan match { - case Project(projectList, child) => - val aliases = AttributeMap(projectList.collect { - case a @ Alias(child, _) => (a.toAttribute, child) - }) - findExpressionAndTrackLineageDown(replaceAlias(exp, aliases), child) + case p: Project => + val aliases = getAliasMap(p) + findExpressionAndTrackLineageDown(replaceAlias(exp, aliases), p.child) // we can unwrap only if there are row projections, and no aggregation operation - case Aggregate(_, aggregateExpressions, child) => - val aliasMap = AttributeMap(aggregateExpressions.collect { - case a: Alias if a.child.find(_.isInstanceOf[AggregateExpression]).isEmpty => - (a.toAttribute, a.child) - }) - findExpressionAndTrackLineageDown(replaceAlias(exp, aliasMap), child) + case a: Aggregate => + val aliasMap = getAliasMap(a) + findExpressionAndTrackLineageDown(replaceAlias(exp, aliasMap), a.child) case l: LeafNode if exp.references.subsetOf(l.outputSet) => Some((exp, l)) case other => 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 f707064380c0..4177c418904c 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 @@ -484,6 +484,7 @@ object RemoveRedundantAggregates extends Rule[LogicalPlan] with AliasHelper { def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { case upper @ Aggregate(_, _, lower: Aggregate) if lowerIsRedundant(upper, lower) => val aliasMap = getAliasMap(lower) + upper.copy( child = lower.child, groupingExpressions = upper.groupingExpressions.map(replaceAlias(_, aliasMap)), @@ -509,23 +510,6 @@ object RemoveRedundantAggregates extends Rule[LogicalPlan] with AliasHelper { expr.find(e => e.isInstanceOf[AggregateExpression] || PythonUDF.isGroupedAggPandasUDF(e)).isDefined } - - /** - * Replace all attributes, that reference an alias, with the aliased expression, - * but keep the name of the outmost attribute. - */ - private def replaceAliasButKeepName( - expr: NamedExpression, - aliasMap: AttributeMap[Expression]): NamedExpression = { - - val replaced = expr match { - case a: Attribute if aliasMap.contains(a) => - Alias(replaceAlias(a, aliasMap), a.name)(a.exprId, a.qualifier) - case _ => replaceAlias(expr, aliasMap) - } - - replaced.asInstanceOf[NamedExpression] - } } /** @@ -782,7 +766,7 @@ object ColumnPruning extends Rule[LogicalPlan] { * and the upper project consists of the same number of columns which is equal or aliasing. * `GlobalLimit(LocalLimit)` pattern is also considered. */ -object CollapseProject extends Rule[LogicalPlan] { +object CollapseProject extends Rule[LogicalPlan] with AliasHelper { def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { case p1 @ Project(_, p2: Project) => @@ -811,43 +795,20 @@ object CollapseProject extends Rule[LogicalPlan] { s.copy(child = p2.copy(projectList = buildCleanedProjectList(l1, p2.projectList))) } - private def collectAliases(projectList: Seq[NamedExpression]): AttributeMap[Alias] = { - AttributeMap(projectList.collect { - case a: Alias => a.toAttribute -> a - }) - } - private def haveCommonNonDeterministicOutput( upper: Seq[NamedExpression], lower: Seq[NamedExpression]): Boolean = { - // Create a map of Aliases to their values from the lower projection. - // e.g., 'SELECT ... FROM (SELECT a + b AS c, d ...)' produces Map(c -> Alias(a + b, c)). - val aliases = collectAliases(lower) + val aliases = getAliasMap(lower) // Collapse upper and lower Projects if and only if their overlapped expressions are all // deterministic. - upper.exists(_.collect { - case a: Attribute if aliases.contains(a) => aliases(a).child - }.exists(!_.deterministic)) + upper.map(replaceAlias(_, aliases)).exists(!_.deterministic) } private def buildCleanedProjectList( upper: Seq[NamedExpression], lower: Seq[NamedExpression]): Seq[NamedExpression] = { - // Create a map of Aliases to their values from the lower projection. - // e.g., 'SELECT ... FROM (SELECT a + b AS c, d ...)' produces Map(c -> Alias(a + b, c)). - val aliases = collectAliases(lower) - - // Substitute any attributes that are produced by the lower projection, so that we safely - // eliminate it. - // e.g., 'SELECT c + 1 FROM (SELECT a + b AS C ...' produces 'SELECT a + b + 1 ...' - // Use transformUp to prevent infinite recursion. - val rewrittenUpper = upper.map(_.transformUp { - case a: Attribute => aliases.getOrElse(a, a) - }) - // collapse upper and lower Projects may introduce unnecessary Aliases, trim them here. - rewrittenUpper.map { p => - CleanupAliases.trimNonTopLevelAliases(p).asInstanceOf[NamedExpression] - } + val aliases = getAliasMap(lower) + upper.map(replaceAliasButKeepName(_, aliases)) } private def isRenaming(list1: Seq[NamedExpression], list2: Seq[NamedExpression]): Boolean = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala index f184253ef059..cb076f6e3518 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala @@ -335,7 +335,7 @@ object PullupCorrelatedPredicates extends Rule[LogicalPlan] with PredicateHelper /** * This rule rewrites correlated [[ScalarSubquery]] expressions into LEFT OUTER joins. */ -object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { +object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] with AliasHelper { /** * Extract all correlated scalar subqueries from an expression. The subqueries are collected using * the given collector. The expression is rewritten and returned. @@ -357,7 +357,7 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { */ private def tryEvalExpr(expr: Expression): Expression = { // Removes Alias over given expression, because Alias is not foldable. - if (!CleanupAliases.trimAliases(expr).foldable) { + if (!trimAliases(expr).foldable) { // SPARK-28441: Some expressions, like PythonUDF, can't be statically evaluated. // Needs to evaluate them on query runtime. expr From a82699eedcf1f72d89ba9fbdade6d1a411a14c4d Mon Sep 17 00:00:00 2001 From: "tanel.kiis@gmail.com" Date: Thu, 22 Oct 2020 04:18:12 +0300 Subject: [PATCH 09/22] Loosen the determinicity check --- .../sql/catalyst/optimizer/Optimizer.scala | 10 ++++--- .../RemoveRedundantAggregatesSuite.scala | 30 +++++++++++-------- 2 files changed, 23 insertions(+), 17 deletions(-) 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 4177c418904c..8626dd58df0d 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 @@ -494,8 +494,8 @@ object RemoveRedundantAggregates extends Rule[LogicalPlan] with AliasHelper { } private def lowerIsRedundant(upper: Aggregate, lower: Aggregate): Boolean = { - val isDeterministic = upper.aggregateExpressions.forall(_.deterministic) && - lower.aggregateExpressions.forall(_.deterministic) + val isDeterministic = !CollapseProject.haveCommonNonDeterministicOutput( + upper.aggregateExpressions, lower.aggregateExpressions) val upperReferencesOnlyGrouping = upper.references.subsetOf(AttributeSet( lower.aggregateExpressions.filter(!isAggregate(_)).map(_.toAttribute))) @@ -795,13 +795,15 @@ object CollapseProject extends Rule[LogicalPlan] with AliasHelper { s.copy(child = p2.copy(projectList = buildCleanedProjectList(l1, p2.projectList))) } - private def haveCommonNonDeterministicOutput( + def haveCommonNonDeterministicOutput( upper: Seq[NamedExpression], lower: Seq[NamedExpression]): Boolean = { val aliases = getAliasMap(lower) // Collapse upper and lower Projects if and only if their overlapped expressions are all // deterministic. - upper.map(replaceAlias(_, aliases)).exists(!_.deterministic) + upper.exists(_.collect { + case a: Attribute if aliases.contains(a) => aliases(a).child + }.exists(!_.deterministic)) } private def buildCleanedProjectList( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala index f97ac01b9c85..c51efa02b4ac 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala @@ -83,6 +83,19 @@ class RemoveRedundantAggregatesSuite extends PlanTest { comparePlans(optimized, expected) } + test("Remove redundant aggregate with non-deterministic upper") { + val relation = LocalRelation('a.int, 'b.int) + val query = relation + .groupBy('a)('a) + .groupBy('a)('a, rand(0) as 'c) + .analyze + val expected = relation + .groupBy('a)('a, rand(0) as 'c) + .analyze + val optimized = Optimize.execute(query) + comparePlans(optimized, expected) + } + test("Keep non-redundant aggregate - upper has agg expression") { val relation = LocalRelation('a.int, 'b.int) val query = relation @@ -104,21 +117,12 @@ class RemoveRedundantAggregatesSuite extends PlanTest { comparePlans(optimized, query) } - test("Keep non-redundant aggregate - lower is non-deterministic") { + test("Keep redundant aggregate with non-deterministic lower") { + // This case could be simplified val relation = LocalRelation('a.int, 'b.int) val query = relation - .groupBy('a, 'b)('a, 'b + rand(0)) - .groupBy('a)('a) - .analyze - val optimized = Optimize.execute(query) - comparePlans(optimized, query) - } - - test("Keep non-redundant aggregate - upper is non-deterministic") { - val relation = LocalRelation('a.int, 'b.int) - val query = relation - .groupBy('a, 'b)('a, 'b) - .groupBy('a)('a + rand(0)) + .groupBy('a)('a, rand(0) as 'c) + .groupBy('a, 'c)('a, 'c) .analyze val optimized = Optimize.execute(query) comparePlans(optimized, query) From 38d700751e6689d272129ee2922f357025289910 Mon Sep 17 00:00:00 2001 From: "tanel.kiis@gmail.com" Date: Thu, 22 Oct 2020 04:31:49 +0300 Subject: [PATCH 10/22] Typo --- .../apache/spark/sql/catalyst/expressions/AliasHelper.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AliasHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AliasHelper.scala index 8eb0a4227d9f..7e0f52e980ec 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AliasHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AliasHelper.scala @@ -54,11 +54,11 @@ trait AliasHelper { */ protected def replaceAlias( expr: Expression, - aliases: AttributeMap[Alias]): Expression = { + aliasMap: AttributeMap[Alias]): Expression = { // Use transformUp to prevent infinite recursion when the replacement expression // redefines the same ExprId, expr.transformUp { - case a: Attribute => aliases.get(a).map(_.child).getOrElse(a) + case a: Attribute => aliasMap.get(a).map(_.child).getOrElse(a) } } From fab04270ff669c7956788522bdaaced4bac25bd3 Mon Sep 17 00:00:00 2001 From: "tanel.kiis@gmail.com" Date: Thu, 22 Oct 2020 07:19:07 +0300 Subject: [PATCH 11/22] Keep old behaviour with attribute naming --- .../catalyst/expressions/AliasHelper.scala | 19 ++++++++----------- 1 file changed, 8 insertions(+), 11 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AliasHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AliasHelper.scala index 7e0f52e980ec..2b6b4af7f480 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AliasHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AliasHelper.scala @@ -57,9 +57,9 @@ trait AliasHelper { aliasMap: AttributeMap[Alias]): Expression = { // Use transformUp to prevent infinite recursion when the replacement expression // redefines the same ExprId, - expr.transformUp { - case a: Attribute => aliasMap.get(a).map(_.child).getOrElse(a) - } + trimAliases(expr.transformUp { + case a: Attribute => aliasMap.getOrElse(a, a) + }) } /** @@ -69,14 +69,11 @@ trait AliasHelper { protected def replaceAliasButKeepName( expr: NamedExpression, aliasMap: AttributeMap[Alias]): NamedExpression = { - - val replaced = expr match { - case a: Attribute if aliasMap.contains(a) => - Alias(replaceAlias(a, aliasMap), a.name)(a.exprId, a.qualifier) - case _ => replaceAlias(expr, aliasMap) - } - - replaced.asInstanceOf[NamedExpression] + // Use transformUp to prevent infinite recursion when the replacement expression + // redefines the same ExprId, + trimNonTopLevelAliases(expr.transformUp { + case a: Attribute => aliasMap.getOrElse(a, a) + }).asInstanceOf[NamedExpression] } protected def trimAliases(e: Expression): Expression = { From 12d1bf4a7e0a333e8d5eb19e9d12d9c6f489f5fb Mon Sep 17 00:00:00 2001 From: "tanel.kiis@gmail.com" Date: Tue, 27 Oct 2020 21:02:23 +0200 Subject: [PATCH 12/22] Fix merge conflict --- .../org/apache/spark/sql/catalyst/optimizer/Optimizer.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 6d9e3bba1b9e..73efc4ad1eaa 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 @@ -795,7 +795,7 @@ object CollapseProject extends Rule[LogicalPlan] with AliasHelper { s.copy(child = p2.copy(projectList = buildCleanedProjectList(l1, p2.projectList))) } - private def haveCommonNonDeterministicOutput( + def haveCommonNonDeterministicOutput( upper: Seq[NamedExpression], lower: Seq[NamedExpression]): Boolean = { val aliases = getAliasMap(lower) From 6d687181170acdc03cf46ed8182a81a88245d9c5 Mon Sep 17 00:00:00 2001 From: "tanel.kiis@gmail.com" Date: Fri, 18 Dec 2020 18:43:56 +0200 Subject: [PATCH 13/22] Fix merge conflict --- .../approved-plans-v1_4/q38/explain.txt | 173 ++++++----------- .../approved-plans-v1_4/q38/simplified.txt | 180 +++++------------- 2 files changed, 106 insertions(+), 247 deletions(-) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt index 8ec8344f3614..3c42e1b0a24c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt @@ -1,112 +1,54 @@ == Physical Plan == -<<<<<<< HEAD -CollectLimit (51) -+- * HashAggregate (50) - +- Exchange (49) - +- * HashAggregate (48) - +- * HashAggregate (47) - +- Exchange (46) - +- * HashAggregate (45) - +- * BroadcastHashJoin LeftSemi BuildRight (44) - :- * BroadcastHashJoin LeftSemi BuildRight (30) - : :- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.date_dim (4) - : : +- BroadcastExchange (14) - : : +- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.customer (11) - : +- BroadcastExchange (29) - : +- * HashAggregate (28) - : +- Exchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * Filter (19) - : : : +- * ColumnarToRow (18) - : : : +- Scan parquet default.catalog_sales (17) - : : +- ReusedExchange (20) - : +- ReusedExchange (23) - +- BroadcastExchange (43) - +- * HashAggregate (42) - +- Exchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * Filter (33) - : : +- * ColumnarToRow (32) - : : +- Scan parquet default.web_sales (31) - : +- ReusedExchange (34) - +- ReusedExchange (37) -======= -* HashAggregate (54) -+- Exchange (53) - +- * HashAggregate (52) - +- * HashAggregate (51) - +- * HashAggregate (50) - +- * HashAggregate (49) - +- * HashAggregate (48) - +- * HashAggregate (47) - +- Exchange (46) - +- * HashAggregate (45) - +- * BroadcastHashJoin LeftSemi BuildRight (44) - :- * BroadcastHashJoin LeftSemi BuildRight (30) - : :- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.date_dim (4) - : : +- BroadcastExchange (14) - : : +- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.customer (11) - : +- BroadcastExchange (29) - : +- * HashAggregate (28) - : +- Exchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * Filter (19) - : : : +- * ColumnarToRow (18) - : : : +- Scan parquet default.catalog_sales (17) - : : +- ReusedExchange (20) - : +- ReusedExchange (23) - +- BroadcastExchange (43) - +- * HashAggregate (42) - +- Exchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * Filter (33) - : : +- * ColumnarToRow (32) - : : +- Scan parquet default.web_sales (31) - : +- ReusedExchange (34) - +- ReusedExchange (37) ->>>>>>> master +* HashAggregate (50) ++- Exchange (49) + +- * HashAggregate (48) + +- * HashAggregate (47) + +- Exchange (46) + +- * HashAggregate (45) + +- * BroadcastHashJoin LeftSemi BuildRight (44) + :- * BroadcastHashJoin LeftSemi BuildRight (30) + : :- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Project (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.date_dim (4) + : : +- BroadcastExchange (14) + : : +- * Filter (13) + : : +- * ColumnarToRow (12) + : : +- Scan parquet default.customer (11) + : +- BroadcastExchange (29) + : +- * HashAggregate (28) + : +- Exchange (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) + : : :- * Filter (19) + : : : +- * ColumnarToRow (18) + : : : +- Scan parquet default.catalog_sales (17) + : : +- ReusedExchange (20) + : +- ReusedExchange (23) + +- BroadcastExchange (43) + +- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) + :- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * Filter (33) + : : +- * ColumnarToRow (32) + : : +- Scan parquet default.web_sales (31) + : +- ReusedExchange (34) + +- ReusedExchange (37) (1) Scan parquet default.store_sales @@ -228,7 +170,7 @@ Results [3]: [c_last_name#17, c_first_name#16, d_date#14] (27) Exchange Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -Arguments: hashpartitioning(c_last_name#17, c_first_name#16, d_date#14, 5), true, [id=#18] +Arguments: hashpartitioning(c_last_name#17, c_first_name#16, d_date#14, 5), ENSURE_REQUIREMENTS, [id=#18] (28) HashAggregate [codegen id : 6] Input [3]: [c_last_name#17, c_first_name#16, d_date#14] @@ -293,7 +235,7 @@ Results [3]: [c_last_name#26, c_first_name#25, d_date#23] (41) Exchange Input [3]: [c_last_name#26, c_first_name#25, d_date#23] -Arguments: hashpartitioning(c_last_name#26, c_first_name#25, d_date#23, 5), true, [id=#27] +Arguments: hashpartitioning(c_last_name#26, c_first_name#25, d_date#23, 5), ENSURE_REQUIREMENTS, [id=#27] (42) HashAggregate [codegen id : 10] Input [3]: [c_last_name#26, c_first_name#25, d_date#23] @@ -320,7 +262,7 @@ Results [3]: [c_last_name#9, c_first_name#8, d_date#4] (46) Exchange Input [3]: [c_last_name#9, c_first_name#8, d_date#4] -Arguments: hashpartitioning(c_last_name#9, c_first_name#8, d_date#4, 5), true, [id=#29] +Arguments: hashpartitioning(c_last_name#9, c_first_name#8, d_date#4, 5), ENSURE_REQUIREMENTS, [id=#29] (47) HashAggregate [codegen id : 12] Input [3]: [c_last_name#9, c_first_name#8, d_date#4] @@ -338,7 +280,7 @@ Results [1]: [count#31] (49) Exchange Input [1]: [count#31] -Arguments: SinglePartition, true, [id=#32] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#32] (50) HashAggregate [codegen id : 13] Input [1]: [count#31] @@ -347,10 +289,3 @@ Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#33] Results [1]: [count(1)#33 AS count(1)#34] -<<<<<<< HEAD -(51) CollectLimit -Input [1]: [count(1)#34] -Arguments: 100 - -======= ->>>>>>> master diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt index 6ca17eb4c212..980aeca8869f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt @@ -5,148 +5,72 @@ WholeStageCodegen (13) WholeStageCodegen (12) HashAggregate [count,count] HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] -<<<<<<< HEAD - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (11) - HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #2 + WholeStageCodegen (11) + HashAggregate [c_last_name,c_first_name,d_date] + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - Project [d_date,c_first_name,c_last_name] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_customer_sk] + Project [d_date,c_first_name,c_last_name] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Project [d_date_sk,d_date] + Filter [d_month_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + Filter [c_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk,d_date] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Filter [c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #6 - WholeStageCodegen (5) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (10) + BroadcastExchange #5 + WholeStageCodegen (6) HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (9) + Exchange [c_last_name,c_first_name,d_date] #6 + WholeStageCodegen (5) HashAggregate [c_last_name,c_first_name,d_date] Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_sold_date_sk,ws_bill_customer_sk] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_sold_date_sk,cs_bill_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] InputAdapter ReusedExchange [d_date_sk,d_date] #3 InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 -======= - HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (11) - HashAggregate [c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - Project [d_date,c_first_name,c_last_name] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_sold_date_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [d_date_sk,d_date] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - Filter [c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #6 - WholeStageCodegen (5) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_sold_date_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (10) + HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (10) + Exchange [c_last_name,c_first_name,d_date] #8 + WholeStageCodegen (9) HashAggregate [c_last_name,c_first_name,d_date] - InputAdapter - Exchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (9) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_sold_date_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + Project [ws_bill_customer_sk,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_bill_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 ->>>>>>> master + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] + InputAdapter + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 From 0d86060acab636bd2f19dbdec4c504924b4335fa Mon Sep 17 00:00:00 2001 From: "tanel.kiis@gmail.com" Date: Sat, 19 Dec 2020 05:13:52 +0200 Subject: [PATCH 14/22] Fix failing test --- .../spark/sql/execution/RemoveRedundantProjectsSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala index 040c5189abcb..5fad64c568e2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala @@ -192,7 +192,7 @@ abstract class RemoveRedundantProjectsSuiteBase |) |""".stripMargin - Seq(("UNION", 2, 2), ("UNION ALL", 1, 2)).foreach { case (setOperation, enabled, disabled) => + Seq(("UNION", 1, 2), ("UNION ALL", 1, 2)).foreach { case (setOperation, enabled, disabled) => val query = queryTemplate.format(setOperation) assertProjectExec(query, enabled = enabled, disabled = disabled) } From 33d60727d50d8c0809305068cde2e20a62e916c4 Mon Sep 17 00:00:00 2001 From: "tanel.kiis@gmail.com" Date: Sat, 19 Dec 2020 10:18:35 +0200 Subject: [PATCH 15/22] Remove non-deterministic restriction --- .../sql/catalyst/analysis/Analyzer.scala | 102 +++++++++--------- .../sql/catalyst/optimizer/Optimizer.scala | 10 +- .../RemoveRedundantAggregatesSuite.scala | 24 +++-- 3 files changed, 71 insertions(+), 65 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 1a5f33443d8e..5974b06be608 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -2940,56 +2940,6 @@ class Analyzer(override val catalogManager: CatalogManager) } } - /** - * Pulls out nondeterministic expressions from LogicalPlan which is not Project or Filter, - * put them into an inner Project and finally project them away at the outer Project. - */ - object PullOutNondeterministic extends Rule[LogicalPlan] { - override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { - case p if !p.resolved => p // Skip unresolved nodes. - case p: Project => p - case f: Filter => f - - case a: Aggregate if a.groupingExpressions.exists(!_.deterministic) => - val nondeterToAttr = getNondeterToAttr(a.groupingExpressions) - val newChild = Project(a.child.output ++ nondeterToAttr.values, a.child) - a.transformExpressions { case e => - nondeterToAttr.get(e).map(_.toAttribute).getOrElse(e) - }.copy(child = newChild) - - // Don't touch collect metrics. Top-level metrics are not supported (check analysis will fail) - // and we want to retain them inside the aggregate functions. - case m: CollectMetrics => m - - // todo: It's hard to write a general rule to pull out nondeterministic expressions - // from LogicalPlan, currently we only do it for UnaryNode which has same output - // schema with its child. - case p: UnaryNode if p.output == p.child.output && p.expressions.exists(!_.deterministic) => - val nondeterToAttr = getNondeterToAttr(p.expressions) - val newPlan = p.transformExpressions { case e => - nondeterToAttr.get(e).map(_.toAttribute).getOrElse(e) - } - val newChild = Project(p.child.output ++ nondeterToAttr.values, p.child) - Project(p.output, newPlan.withNewChildren(newChild :: Nil)) - } - - private def getNondeterToAttr(exprs: Seq[Expression]): Map[Expression, NamedExpression] = { - exprs.filterNot(_.deterministic).flatMap { expr => - val leafNondeterministic = expr.collect { - case n: Nondeterministic => n - case udf: UserDefinedExpression if !udf.deterministic => udf - } - leafNondeterministic.distinct.map { e => - val ne = e match { - case n: NamedExpression => n - case _ => Alias(e, "_nondeterministic")() - } - e -> ne - } - }.toMap - } - } - /** * Set the seed for random number generation. */ @@ -3858,3 +3808,55 @@ object UpdateOuterReferences extends Rule[LogicalPlan] { } } } + +/** + * Pulls out nondeterministic expressions from LogicalPlan which is not Project or Filter, + * put them into an inner Project and finally project them away at the outer Project. + */ +object PullOutNondeterministic extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperatorsUp applyLocally + + val applyLocally: PartialFunction[LogicalPlan, LogicalPlan] = { + case p if !p.resolved => p // Skip unresolved nodes. + case p: Project => p + case f: Filter => f + + case a: Aggregate if a.groupingExpressions.exists(!_.deterministic) => + val nondeterToAttr = getNondeterToAttr(a.groupingExpressions) + val newChild = Project(a.child.output ++ nondeterToAttr.values, a.child) + a.transformExpressions { case e => + nondeterToAttr.get(e).map(_.toAttribute).getOrElse(e) + }.copy(child = newChild) + + // Don't touch collect metrics. Top-level metrics are not supported (check analysis will fail) + // and we want to retain them inside the aggregate functions. + case m: CollectMetrics => m + + // todo: It's hard to write a general rule to pull out nondeterministic expressions + // from LogicalPlan, currently we only do it for UnaryNode which has same output + // schema with its child. + case p: UnaryNode if p.output == p.child.output && p.expressions.exists(!_.deterministic) => + val nondeterToAttr = getNondeterToAttr(p.expressions) + val newPlan = p.transformExpressions { case e => + nondeterToAttr.get(e).map(_.toAttribute).getOrElse(e) + } + val newChild = Project(p.child.output ++ nondeterToAttr.values, p.child) + Project(p.output, newPlan.withNewChildren(newChild :: Nil)) + } + + private def getNondeterToAttr(exprs: Seq[Expression]): Map[Expression, NamedExpression] = { + exprs.filterNot(_.deterministic).flatMap { expr => + val leafNondeterministic = expr.collect { + case n: Nondeterministic => n + case udf: UserDefinedExpression if !udf.deterministic => udf + } + leafNondeterministic.distinct.map { e => + val ne = e match { + case n: NamedExpression => n + case _ => Alias(e, "_nondeterministic")() + } + e -> ne + } + }.toMap + } +} 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 0cb1fff07d54..94dc5db4148c 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 @@ -495,25 +495,25 @@ object RemoveRedundantAggregates extends Rule[LogicalPlan] with AliasHelper { case upper @ Aggregate(_, _, lower: Aggregate) if lowerIsRedundant(upper, lower) => val aliasMap = getAliasMap(lower) - upper.copy( + val newAggregate = upper.copy( child = lower.child, groupingExpressions = upper.groupingExpressions.map(replaceAlias(_, aliasMap)), aggregateExpressions = upper.aggregateExpressions.map( replaceAliasButKeepName(_, aliasMap)) ) + + // We might have introduces non-deterministic grouping expression + PullOutNondeterministic.applyLocally.applyOrElse(newAggregate, identity[LogicalPlan]) } private def lowerIsRedundant(upper: Aggregate, lower: Aggregate): Boolean = { - val isDeterministic = !CollapseProject.haveCommonNonDeterministicOutput( - upper.aggregateExpressions, lower.aggregateExpressions) - val upperReferencesOnlyGrouping = upper.references.subsetOf(AttributeSet( lower.aggregateExpressions.filter(!isAggregate(_)).map(_.toAttribute))) val upperHasNoAggregateExpressions = upper.aggregateExpressions .forall(_.find(isAggregate).isEmpty) - isDeterministic && upperReferencesOnlyGrouping && upperHasNoAggregateExpressions + upperReferencesOnlyGrouping && upperHasNoAggregateExpressions } private def isAggregate(expr: Expression): Boolean = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala index c51efa02b4ac..3bf831bb9c24 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala @@ -96,6 +96,20 @@ class RemoveRedundantAggregatesSuite extends PlanTest { comparePlans(optimized, expected) } + test("Remove redundant aggregate with non-deterministic lower") { + val relation = LocalRelation('a.int, 'b.int) + val query = relation + .groupBy('a)('a, rand(0) as 'c) + .groupBy('a, 'c)('a, 'c) + .analyze + val expected = relation + .groupBy('a, 'c)('a, rand(0) as 'c) + .analyze + val optimized = Optimize.execute(query) + comparePlans(optimized, expected) + } + + test("Keep non-redundant aggregate - upper has agg expression") { val relation = LocalRelation('a.int, 'b.int) val query = relation @@ -117,14 +131,4 @@ class RemoveRedundantAggregatesSuite extends PlanTest { comparePlans(optimized, query) } - test("Keep redundant aggregate with non-deterministic lower") { - // This case could be simplified - val relation = LocalRelation('a.int, 'b.int) - val query = relation - .groupBy('a)('a, rand(0) as 'c) - .groupBy('a, 'c)('a, 'c) - .analyze - val optimized = Optimize.execute(query) - comparePlans(optimized, query) - } } From 3e3168a10d2366fc75db2ed5940d32cd3af51303 Mon Sep 17 00:00:00 2001 From: "tanel.kiis@gmail.com" Date: Sun, 20 Dec 2020 18:41:33 +0200 Subject: [PATCH 16/22] Address comments --- .../sql/catalyst/analysis/Analyzer.scala | 52 ------------- .../analysis/PullOutNondeterministic.scala | 74 +++++++++++++++++++ .../sql/catalyst/optimizer/Optimizer.scala | 6 +- .../RemoveRedundantAggregatesSuite.scala | 1 - 4 files changed, 79 insertions(+), 54 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/PullOutNondeterministic.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 5974b06be608..6c2bf17bbd81 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -3808,55 +3808,3 @@ object UpdateOuterReferences extends Rule[LogicalPlan] { } } } - -/** - * Pulls out nondeterministic expressions from LogicalPlan which is not Project or Filter, - * put them into an inner Project and finally project them away at the outer Project. - */ -object PullOutNondeterministic extends Rule[LogicalPlan] { - override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperatorsUp applyLocally - - val applyLocally: PartialFunction[LogicalPlan, LogicalPlan] = { - case p if !p.resolved => p // Skip unresolved nodes. - case p: Project => p - case f: Filter => f - - case a: Aggregate if a.groupingExpressions.exists(!_.deterministic) => - val nondeterToAttr = getNondeterToAttr(a.groupingExpressions) - val newChild = Project(a.child.output ++ nondeterToAttr.values, a.child) - a.transformExpressions { case e => - nondeterToAttr.get(e).map(_.toAttribute).getOrElse(e) - }.copy(child = newChild) - - // Don't touch collect metrics. Top-level metrics are not supported (check analysis will fail) - // and we want to retain them inside the aggregate functions. - case m: CollectMetrics => m - - // todo: It's hard to write a general rule to pull out nondeterministic expressions - // from LogicalPlan, currently we only do it for UnaryNode which has same output - // schema with its child. - case p: UnaryNode if p.output == p.child.output && p.expressions.exists(!_.deterministic) => - val nondeterToAttr = getNondeterToAttr(p.expressions) - val newPlan = p.transformExpressions { case e => - nondeterToAttr.get(e).map(_.toAttribute).getOrElse(e) - } - val newChild = Project(p.child.output ++ nondeterToAttr.values, p.child) - Project(p.output, newPlan.withNewChildren(newChild :: Nil)) - } - - private def getNondeterToAttr(exprs: Seq[Expression]): Map[Expression, NamedExpression] = { - exprs.filterNot(_.deterministic).flatMap { expr => - val leafNondeterministic = expr.collect { - case n: Nondeterministic => n - case udf: UserDefinedExpression if !udf.deterministic => udf - } - leafNondeterministic.distinct.map { e => - val ne = e match { - case n: NamedExpression => n - case _ => Alias(e, "_nondeterministic")() - } - e -> ne - } - }.toMap - } -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/PullOutNondeterministic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/PullOutNondeterministic.scala new file mode 100644 index 000000000000..3431c9327f1d --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/PullOutNondeterministic.scala @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.analysis + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.Rule + +/** + * Pulls out nondeterministic expressions from LogicalPlan which is not Project or Filter, + * put them into an inner Project and finally project them away at the outer Project. + */ +object PullOutNondeterministic extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperatorsUp applyLocally + + val applyLocally: PartialFunction[LogicalPlan, LogicalPlan] = { + case p if !p.resolved => p // Skip unresolved nodes. + case p: Project => p + case f: Filter => f + + case a: Aggregate if a.groupingExpressions.exists(!_.deterministic) => + val nondeterToAttr = getNondeterToAttr(a.groupingExpressions) + val newChild = Project(a.child.output ++ nondeterToAttr.values, a.child) + a.transformExpressions { case e => + nondeterToAttr.get(e).map(_.toAttribute).getOrElse(e) + }.copy(child = newChild) + + // Don't touch collect metrics. Top-level metrics are not supported (check analysis will fail) + // and we want to retain them inside the aggregate functions. + case m: CollectMetrics => m + + // todo: It's hard to write a general rule to pull out nondeterministic expressions + // from LogicalPlan, currently we only do it for UnaryNode which has same output + // schema with its child. + case p: UnaryNode if p.output == p.child.output && p.expressions.exists(!_.deterministic) => + val nondeterToAttr = getNondeterToAttr(p.expressions) + val newPlan = p.transformExpressions { case e => + nondeterToAttr.get(e).map(_.toAttribute).getOrElse(e) + } + val newChild = Project(p.child.output ++ nondeterToAttr.values, p.child) + Project(p.output, newPlan.withNewChildren(newChild :: Nil)) + } + + private def getNondeterToAttr(exprs: Seq[Expression]): Map[Expression, NamedExpression] = { + exprs.filterNot(_.deterministic).flatMap { expr => + val leafNondeterministic = expr.collect { + case n: Nondeterministic => n + case udf: UserDefinedExpression if !udf.deterministic => udf + } + leafNondeterministic.distinct.map { e => + val ne = e match { + case n: NamedExpression => n + case _ => Alias(e, "_nondeterministic")() + } + e -> ne + } + }.toMap + } +} 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 94dc5db4148c..b864fd6113de 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 @@ -503,7 +503,11 @@ object RemoveRedundantAggregates extends Rule[LogicalPlan] with AliasHelper { ) // We might have introduces non-deterministic grouping expression - PullOutNondeterministic.applyLocally.applyOrElse(newAggregate, identity[LogicalPlan]) + if (newAggregate.groupingExpressions.exists(!_.deterministic)) { + PullOutNondeterministic.applyLocally.applyOrElse(newAggregate, identity[LogicalPlan]) + } else { + newAggregate + } } private def lowerIsRedundant(upper: Aggregate, lower: Aggregate): Boolean = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala index 3bf831bb9c24..5c9b2d000f0b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala @@ -109,7 +109,6 @@ class RemoveRedundantAggregatesSuite extends PlanTest { comparePlans(optimized, expected) } - test("Keep non-redundant aggregate - upper has agg expression") { val relation = LocalRelation('a.int, 'b.int) val query = relation From 57af00504c7e985e71a60960d45c8b08f76407a7 Mon Sep 17 00:00:00 2001 From: "tanel.kiis@gmail.com" Date: Mon, 21 Dec 2020 11:13:50 +0200 Subject: [PATCH 17/22] Add UTs for python funcs --- .../sql/catalyst/optimizer/Optimizer.scala | 10 +- .../RemoveRedundantAggregatesSuite.scala | 103 +++++++++++------- 2 files changed, 67 insertions(+), 46 deletions(-) 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 b864fd6113de..85f4ac8ea2f0 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 @@ -511,13 +511,13 @@ object RemoveRedundantAggregates extends Rule[LogicalPlan] with AliasHelper { } private def lowerIsRedundant(upper: Aggregate, lower: Aggregate): Boolean = { - val upperReferencesOnlyGrouping = upper.references.subsetOf(AttributeSet( - lower.aggregateExpressions.filter(!isAggregate(_)).map(_.toAttribute))) - val upperHasNoAggregateExpressions = upper.aggregateExpressions .forall(_.find(isAggregate).isEmpty) - upperReferencesOnlyGrouping && upperHasNoAggregateExpressions + lazy val upperReferencesOnlyGrouping = upper.references.subsetOf(AttributeSet( + lower.aggregateExpressions.filter(!isAggregate(_)).map(_.toAttribute))) + + upperHasNoAggregateExpressions && upperReferencesOnlyGrouping } private def isAggregate(expr: Expression): Boolean = { @@ -809,7 +809,7 @@ object CollapseProject extends Rule[LogicalPlan] with AliasHelper { s.copy(child = p2.copy(projectList = buildCleanedProjectList(l1, p2.projectList))) } - def haveCommonNonDeterministicOutput( + private def haveCommonNonDeterministicOutput( upper: Seq[NamedExpression], lower: Seq[NamedExpression]): Boolean = { val aliases = getAliasMap(lower) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala index 5c9b2d000f0b..d19cc88be015 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala @@ -17,11 +17,14 @@ package org.apache.spark.sql.catalyst.optimizer +import org.apache.spark.api.python.PythonEvalType import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.expressions.{Expression, PythonUDF} import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.types.IntegerType class RemoveRedundantAggregatesSuite extends PlanTest { @@ -30,31 +33,43 @@ class RemoveRedundantAggregatesSuite extends PlanTest { RemoveRedundantAggregates) :: Nil } + private def aggregates(e: Expression): Seq[Expression] = { + Seq( + count(e), + PythonUDF("pyUDF", null, IntegerType, Seq(e), + PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF, udfDeterministic = true) + ) + } + test("Remove redundant aggregate") { val relation = LocalRelation('a.int, 'b.int) - val query = relation - .groupBy('a)('a, count('b)) - .groupBy('a)('a) - .analyze - val expected = relation - .groupBy('a)('a) - .analyze - val optimized = Optimize.execute(query) - comparePlans(optimized, expected) + for (agg <- aggregates('b)) { + val query = relation + .groupBy('a)('a, agg) + .groupBy('a)('a) + .analyze + val expected = relation + .groupBy('a)('a) + .analyze + val optimized = Optimize.execute(query) + comparePlans(optimized, expected) + } } test("Remove 2 redundant aggregates") { val relation = LocalRelation('a.int, 'b.int) - val query = relation - .groupBy('a)('a, count('b)) - .groupBy('a)('a) - .groupBy('a)('a) - .analyze - val expected = relation - .groupBy('a)('a) - .analyze - val optimized = Optimize.execute(query) - comparePlans(optimized, expected) + for (agg <- aggregates('b)) { + val query = relation + .groupBy('a)('a, agg) + .groupBy('a)('a) + .groupBy('a)('a) + .analyze + val expected = relation + .groupBy('a)('a) + .analyze + val optimized = Optimize.execute(query) + comparePlans(optimized, expected) + } } test("Remove redundant aggregate with different grouping") { @@ -72,15 +87,17 @@ class RemoveRedundantAggregatesSuite extends PlanTest { test("Remove redundant aggregate with aliases") { val relation = LocalRelation('a.int, 'b.int) - val query = relation - .groupBy('a + 'b)(('a + 'b) as 'c, count('b)) - .groupBy('c)('c) - .analyze - val expected = relation - .groupBy('a + 'b)(('a + 'b) as 'c) - .analyze - val optimized = Optimize.execute(query) - comparePlans(optimized, expected) + for (agg <- aggregates('b)) { + val query = relation + .groupBy('a + 'b)(('a + 'b) as 'c, agg) + .groupBy('c)('c) + .analyze + val expected = relation + .groupBy('a + 'b)(('a + 'b) as 'c) + .analyze + val optimized = Optimize.execute(query) + comparePlans(optimized, expected) + } } test("Remove redundant aggregate with non-deterministic upper") { @@ -111,23 +128,27 @@ class RemoveRedundantAggregatesSuite extends PlanTest { test("Keep non-redundant aggregate - upper has agg expression") { val relation = LocalRelation('a.int, 'b.int) - val query = relation - .groupBy('a, 'b)('a, 'b) - // The count would change if we remove the first aggregate - .groupBy('a)('a, count('b)) - .analyze - val optimized = Optimize.execute(query) - comparePlans(optimized, query) + for (agg <- aggregates('b)) { + val query = relation + .groupBy('a, 'b)('a, 'b) + // The count would change if we remove the first aggregate + .groupBy('a)('a, agg) + .analyze + val optimized = Optimize.execute(query) + comparePlans(optimized, query) + } } test("Keep non-redundant aggregate - upper references non-grouping") { val relation = LocalRelation('a.int, 'b.int) - val query = relation - .groupBy('a)('a, count('b) as 'c) - .groupBy('c)('c) - .analyze - val optimized = Optimize.execute(query) - comparePlans(optimized, query) + for (agg <- aggregates('b)) { + val query = relation + .groupBy('a)('a, agg as 'c) + .groupBy('c)('c) + .analyze + val optimized = Optimize.execute(query) + comparePlans(optimized, query) + } } } From c21dd5269c11d537ef2ca464d6a69bf44fdcbc29 Mon Sep 17 00:00:00 2001 From: Tanel Kiis Date: Tue, 5 Jan 2021 08:18:08 +0200 Subject: [PATCH 18/22] Merge branch 'master' into SPARK-33122 # Conflicts: # sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AliasHelper.scala # sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala # sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt # sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/simplified.txt # sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt # sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt # sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt # sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt # sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt # sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt --- .../q14a.sf100/explain.txt | 874 +++++++++--------- .../q14a.sf100/simplified.txt | 134 ++- .../approved-plans-v2_7/q14a/explain.txt | 826 ++++++++--------- .../approved-plans-v2_7/q14a/simplified.txt | 116 ++- 4 files changed, 957 insertions(+), 993 deletions(-) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt index 7be9447d16b4..9506aa841344 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt @@ -1,214 +1,212 @@ == Physical Plan == -TakeOrderedAndProject (210) -+- * HashAggregate (209) - +- Exchange (208) - +- * HashAggregate (207) - +- Union (206) - :- * HashAggregate (129) - : +- Exchange (128) - : +- * HashAggregate (127) - : +- Union (126) - : :- * Project (87) - : : +- * Filter (86) - : : +- * HashAggregate (85) - : : +- Exchange (84) - : : +- * HashAggregate (83) - : : +- * Project (82) - : : +- * BroadcastHashJoin Inner BuildRight (81) - : : :- * Project (71) - : : : +- * BroadcastHashJoin Inner BuildRight (70) - : : : :- SortMergeJoin LeftSemi (64) +TakeOrderedAndProject (208) ++- * HashAggregate (207) + +- Exchange (206) + +- * HashAggregate (205) + +- Union (204) + :- * HashAggregate (127) + : +- Exchange (126) + : +- * HashAggregate (125) + : +- Union (124) + : :- * Project (85) + : : +- * Filter (84) + : : +- * HashAggregate (83) + : : +- Exchange (82) + : : +- * HashAggregate (81) + : : +- * Project (80) + : : +- * BroadcastHashJoin Inner BuildRight (79) + : : :- * Project (69) + : : : +- * BroadcastHashJoin Inner BuildRight (68) + : : : :- SortMergeJoin LeftSemi (62) : : : : :- * Sort (5) : : : : : +- Exchange (4) : : : : : +- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.store_sales (1) - : : : : +- * Sort (63) - : : : : +- Exchange (62) - : : : : +- * Project (61) - : : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : : +- * Sort (61) + : : : : +- Exchange (60) + : : : : +- * Project (59) + : : : : +- * BroadcastHashJoin Inner BuildRight (58) : : : : :- * Filter (8) : : : : : +- * ColumnarToRow (7) : : : : : +- Scan parquet default.item (6) - : : : : +- BroadcastExchange (59) - : : : : +- * HashAggregate (58) - : : : : +- * HashAggregate (57) - : : : : +- * HashAggregate (56) - : : : : +- Exchange (55) - : : : : +- * HashAggregate (54) - : : : : +- SortMergeJoin LeftSemi (53) - : : : : :- SortMergeJoin LeftSemi (41) - : : : : : :- * Sort (26) - : : : : : : +- Exchange (25) - : : : : : : +- * Project (24) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : : :- * Project (18) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : : : :- * Filter (11) - : : : : : : : : +- * ColumnarToRow (10) - : : : : : : : : +- Scan parquet default.store_sales (9) - : : : : : : : +- BroadcastExchange (16) - : : : : : : : +- * Project (15) - : : : : : : : +- * Filter (14) - : : : : : : : +- * ColumnarToRow (13) - : : : : : : : +- Scan parquet default.date_dim (12) - : : : : : : +- BroadcastExchange (22) - : : : : : : +- * Filter (21) - : : : : : : +- * ColumnarToRow (20) - : : : : : : +- Scan parquet default.item (19) - : : : : : +- * Sort (40) - : : : : : +- Exchange (39) - : : : : : +- * Project (38) - : : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : : :- * Project (32) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : : :- * Filter (29) - : : : : : : : +- * ColumnarToRow (28) - : : : : : : : +- Scan parquet default.catalog_sales (27) - : : : : : : +- ReusedExchange (30) - : : : : : +- BroadcastExchange (36) - : : : : : +- * Filter (35) - : : : : : +- * ColumnarToRow (34) - : : : : : +- Scan parquet default.item (33) - : : : : +- * Sort (52) - : : : : +- Exchange (51) - : : : : +- * Project (50) - : : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : : :- * Project (47) - : : : : : +- * BroadcastHashJoin Inner BuildRight (46) - : : : : : :- * Filter (44) - : : : : : : +- * ColumnarToRow (43) - : : : : : : +- Scan parquet default.web_sales (42) - : : : : : +- ReusedExchange (45) - : : : : +- ReusedExchange (48) - : : : +- BroadcastExchange (69) - : : : +- * Project (68) - : : : +- * Filter (67) - : : : +- * ColumnarToRow (66) - : : : +- Scan parquet default.date_dim (65) - : : +- BroadcastExchange (80) - : : +- SortMergeJoin LeftSemi (79) - : : :- * Sort (76) - : : : +- Exchange (75) - : : : +- * Filter (74) - : : : +- * ColumnarToRow (73) - : : : +- Scan parquet default.item (72) - : : +- * Sort (78) - : : +- ReusedExchange (77) - : :- * Project (106) - : : +- * Filter (105) - : : +- * HashAggregate (104) - : : +- Exchange (103) - : : +- * HashAggregate (102) - : : +- * Project (101) - : : +- * BroadcastHashJoin Inner BuildRight (100) - : : :- * Project (98) - : : : +- * BroadcastHashJoin Inner BuildRight (97) - : : : :- SortMergeJoin LeftSemi (95) - : : : : :- * Sort (92) - : : : : : +- Exchange (91) - : : : : : +- * Filter (90) - : : : : : +- * ColumnarToRow (89) - : : : : : +- Scan parquet default.catalog_sales (88) - : : : : +- * Sort (94) - : : : : +- ReusedExchange (93) - : : : +- ReusedExchange (96) - : : +- ReusedExchange (99) - : +- * Project (125) - : +- * Filter (124) - : +- * HashAggregate (123) - : +- Exchange (122) - : +- * HashAggregate (121) - : +- * Project (120) - : +- * BroadcastHashJoin Inner BuildRight (119) - : :- * Project (117) - : : +- * BroadcastHashJoin Inner BuildRight (116) - : : :- SortMergeJoin LeftSemi (114) - : : : :- * Sort (111) - : : : : +- Exchange (110) - : : : : +- * Filter (109) - : : : : +- * ColumnarToRow (108) - : : : : +- Scan parquet default.web_sales (107) - : : : +- * Sort (113) - : : : +- ReusedExchange (112) - : : +- ReusedExchange (115) - : +- ReusedExchange (118) - :- * HashAggregate (148) - : +- Exchange (147) - : +- * HashAggregate (146) - : +- * HashAggregate (145) - : +- Exchange (144) - : +- * HashAggregate (143) - : +- Union (142) - : :- * Project (133) - : : +- * Filter (132) - : : +- * HashAggregate (131) - : : +- ReusedExchange (130) - : :- * Project (137) - : : +- * Filter (136) - : : +- * HashAggregate (135) - : : +- ReusedExchange (134) - : +- * Project (141) - : +- * Filter (140) - : +- * HashAggregate (139) - : +- ReusedExchange (138) - :- * HashAggregate (167) - : +- Exchange (166) - : +- * HashAggregate (165) - : +- * HashAggregate (164) - : +- Exchange (163) - : +- * HashAggregate (162) - : +- Union (161) - : :- * Project (152) - : : +- * Filter (151) - : : +- * HashAggregate (150) - : : +- ReusedExchange (149) - : :- * Project (156) - : : +- * Filter (155) - : : +- * HashAggregate (154) - : : +- ReusedExchange (153) - : +- * Project (160) - : +- * Filter (159) - : +- * HashAggregate (158) - : +- ReusedExchange (157) - :- * HashAggregate (186) - : +- Exchange (185) - : +- * HashAggregate (184) - : +- * HashAggregate (183) - : +- Exchange (182) - : +- * HashAggregate (181) - : +- Union (180) - : :- * Project (171) - : : +- * Filter (170) - : : +- * HashAggregate (169) - : : +- ReusedExchange (168) - : :- * Project (175) - : : +- * Filter (174) - : : +- * HashAggregate (173) - : : +- ReusedExchange (172) - : +- * Project (179) - : +- * Filter (178) - : +- * HashAggregate (177) - : +- ReusedExchange (176) - +- * HashAggregate (205) - +- Exchange (204) - +- * HashAggregate (203) - +- * HashAggregate (202) - +- Exchange (201) - +- * HashAggregate (200) - +- Union (199) - :- * Project (190) - : +- * Filter (189) - : +- * HashAggregate (188) - : +- ReusedExchange (187) - :- * Project (194) - : +- * Filter (193) - : +- * HashAggregate (192) - : +- ReusedExchange (191) - +- * Project (198) - +- * Filter (197) - +- * HashAggregate (196) - +- ReusedExchange (195) + : : : : +- BroadcastExchange (57) + : : : : +- * HashAggregate (56) + : : : : +- Exchange (55) + : : : : +- * HashAggregate (54) + : : : : +- SortMergeJoin LeftSemi (53) + : : : : :- SortMergeJoin LeftSemi (41) + : : : : : :- * Sort (26) + : : : : : : +- Exchange (25) + : : : : : : +- * Project (24) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : : :- * Project (18) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : : : :- * Filter (11) + : : : : : : : : +- * ColumnarToRow (10) + : : : : : : : : +- Scan parquet default.store_sales (9) + : : : : : : : +- BroadcastExchange (16) + : : : : : : : +- * Project (15) + : : : : : : : +- * Filter (14) + : : : : : : : +- * ColumnarToRow (13) + : : : : : : : +- Scan parquet default.date_dim (12) + : : : : : : +- BroadcastExchange (22) + : : : : : : +- * Filter (21) + : : : : : : +- * ColumnarToRow (20) + : : : : : : +- Scan parquet default.item (19) + : : : : : +- * Sort (40) + : : : : : +- Exchange (39) + : : : : : +- * Project (38) + : : : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : : : :- * Project (32) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : : : :- * Filter (29) + : : : : : : : +- * ColumnarToRow (28) + : : : : : : : +- Scan parquet default.catalog_sales (27) + : : : : : : +- ReusedExchange (30) + : : : : : +- BroadcastExchange (36) + : : : : : +- * Filter (35) + : : : : : +- * ColumnarToRow (34) + : : : : : +- Scan parquet default.item (33) + : : : : +- * Sort (52) + : : : : +- Exchange (51) + : : : : +- * Project (50) + : : : : +- * BroadcastHashJoin Inner BuildRight (49) + : : : : :- * Project (47) + : : : : : +- * BroadcastHashJoin Inner BuildRight (46) + : : : : : :- * Filter (44) + : : : : : : +- * ColumnarToRow (43) + : : : : : : +- Scan parquet default.web_sales (42) + : : : : : +- ReusedExchange (45) + : : : : +- ReusedExchange (48) + : : : +- BroadcastExchange (67) + : : : +- * Project (66) + : : : +- * Filter (65) + : : : +- * ColumnarToRow (64) + : : : +- Scan parquet default.date_dim (63) + : : +- BroadcastExchange (78) + : : +- SortMergeJoin LeftSemi (77) + : : :- * Sort (74) + : : : +- Exchange (73) + : : : +- * Filter (72) + : : : +- * ColumnarToRow (71) + : : : +- Scan parquet default.item (70) + : : +- * Sort (76) + : : +- ReusedExchange (75) + : :- * Project (104) + : : +- * Filter (103) + : : +- * HashAggregate (102) + : : +- Exchange (101) + : : +- * HashAggregate (100) + : : +- * Project (99) + : : +- * BroadcastHashJoin Inner BuildRight (98) + : : :- * Project (96) + : : : +- * BroadcastHashJoin Inner BuildRight (95) + : : : :- SortMergeJoin LeftSemi (93) + : : : : :- * Sort (90) + : : : : : +- Exchange (89) + : : : : : +- * Filter (88) + : : : : : +- * ColumnarToRow (87) + : : : : : +- Scan parquet default.catalog_sales (86) + : : : : +- * Sort (92) + : : : : +- ReusedExchange (91) + : : : +- ReusedExchange (94) + : : +- ReusedExchange (97) + : +- * Project (123) + : +- * Filter (122) + : +- * HashAggregate (121) + : +- Exchange (120) + : +- * HashAggregate (119) + : +- * Project (118) + : +- * BroadcastHashJoin Inner BuildRight (117) + : :- * Project (115) + : : +- * BroadcastHashJoin Inner BuildRight (114) + : : :- SortMergeJoin LeftSemi (112) + : : : :- * Sort (109) + : : : : +- Exchange (108) + : : : : +- * Filter (107) + : : : : +- * ColumnarToRow (106) + : : : : +- Scan parquet default.web_sales (105) + : : : +- * Sort (111) + : : : +- ReusedExchange (110) + : : +- ReusedExchange (113) + : +- ReusedExchange (116) + :- * HashAggregate (146) + : +- Exchange (145) + : +- * HashAggregate (144) + : +- * HashAggregate (143) + : +- Exchange (142) + : +- * HashAggregate (141) + : +- Union (140) + : :- * Project (131) + : : +- * Filter (130) + : : +- * HashAggregate (129) + : : +- ReusedExchange (128) + : :- * Project (135) + : : +- * Filter (134) + : : +- * HashAggregate (133) + : : +- ReusedExchange (132) + : +- * Project (139) + : +- * Filter (138) + : +- * HashAggregate (137) + : +- ReusedExchange (136) + :- * HashAggregate (165) + : +- Exchange (164) + : +- * HashAggregate (163) + : +- * HashAggregate (162) + : +- Exchange (161) + : +- * HashAggregate (160) + : +- Union (159) + : :- * Project (150) + : : +- * Filter (149) + : : +- * HashAggregate (148) + : : +- ReusedExchange (147) + : :- * Project (154) + : : +- * Filter (153) + : : +- * HashAggregate (152) + : : +- ReusedExchange (151) + : +- * Project (158) + : +- * Filter (157) + : +- * HashAggregate (156) + : +- ReusedExchange (155) + :- * HashAggregate (184) + : +- Exchange (183) + : +- * HashAggregate (182) + : +- * HashAggregate (181) + : +- Exchange (180) + : +- * HashAggregate (179) + : +- Union (178) + : :- * Project (169) + : : +- * Filter (168) + : : +- * HashAggregate (167) + : : +- ReusedExchange (166) + : :- * Project (173) + : : +- * Filter (172) + : : +- * HashAggregate (171) + : : +- ReusedExchange (170) + : +- * Project (177) + : +- * Filter (176) + : +- * HashAggregate (175) + : +- ReusedExchange (174) + +- * HashAggregate (203) + +- Exchange (202) + +- * HashAggregate (201) + +- * HashAggregate (200) + +- Exchange (199) + +- * HashAggregate (198) + +- Union (197) + :- * Project (188) + : +- * Filter (187) + : +- * HashAggregate (186) + : +- ReusedExchange (185) + :- * Project (192) + : +- * Filter (191) + : +- * HashAggregate (190) + : +- ReusedExchange (189) + +- * Project (196) + +- * Filter (195) + +- * HashAggregate (194) + +- ReusedExchange (193) (1) Scan parquet default.store_sales @@ -462,927 +460,913 @@ Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#14, class_id#15, category_id#16] -(57) HashAggregate [codegen id : 16] -Input [3]: [brand_id#14, class_id#15, category_id#16] -Keys [3]: [brand_id#14, class_id#15, category_id#16] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#14, class_id#15, category_id#16] - -(58) HashAggregate [codegen id : 16] -Input [3]: [brand_id#14, class_id#15, category_id#16] -Keys [3]: [brand_id#14, class_id#15, category_id#16] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#14, class_id#15, category_id#16] - -(59) BroadcastExchange +(57) BroadcastExchange Input [3]: [brand_id#14, class_id#15, category_id#16] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#26] -(60) BroadcastHashJoin [codegen id : 17] +(58) BroadcastHashJoin [codegen id : 17] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#14, class_id#15, category_id#16] Join condition: None -(61) Project [codegen id : 17] +(59) Project [codegen id : 17] Output [1]: [i_item_sk#6 AS ss_item_sk#27] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#14, class_id#15, category_id#16] -(62) Exchange +(60) Exchange Input [1]: [ss_item_sk#27] Arguments: hashpartitioning(ss_item_sk#27, 5), ENSURE_REQUIREMENTS, [id=#28] -(63) Sort [codegen id : 18] +(61) Sort [codegen id : 18] Input [1]: [ss_item_sk#27] Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 -(64) SortMergeJoin +(62) SortMergeJoin Left keys [1]: [ss_item_sk#2] Right keys [1]: [ss_item_sk#27] Join condition: None -(65) Scan parquet default.date_dim +(63) Scan parquet default.date_dim Output [3]: [d_date_sk#10, d_year#11, d_moy#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(66) ColumnarToRow [codegen id : 19] +(64) ColumnarToRow [codegen id : 19] Input [3]: [d_date_sk#10, d_year#11, d_moy#29] -(67) Filter [codegen id : 19] +(65) Filter [codegen id : 19] Input [3]: [d_date_sk#10, d_year#11, d_moy#29] Condition : ((((isnotnull(d_year#11) AND isnotnull(d_moy#29)) AND (d_year#11 = 2000)) AND (d_moy#29 = 11)) AND isnotnull(d_date_sk#10)) -(68) Project [codegen id : 19] +(66) Project [codegen id : 19] Output [1]: [d_date_sk#10] Input [3]: [d_date_sk#10, d_year#11, d_moy#29] -(69) BroadcastExchange +(67) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] -(70) BroadcastHashJoin [codegen id : 38] +(68) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(71) Project [codegen id : 38] +(69) Project [codegen id : 38] Output [3]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4] Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(72) Scan parquet default.item +(70) Scan parquet default.item Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(73) ColumnarToRow [codegen id : 20] +(71) ColumnarToRow [codegen id : 20] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(74) Filter [codegen id : 20] +(72) Filter [codegen id : 20] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : isnotnull(i_item_sk#6) -(75) Exchange +(73) Exchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: hashpartitioning(i_item_sk#6, 5), ENSURE_REQUIREMENTS, [id=#31] -(76) Sort [codegen id : 21] +(74) Sort [codegen id : 21] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: [i_item_sk#6 ASC NULLS FIRST], false, 0 -(77) ReusedExchange [Reuses operator id: 62] +(75) ReusedExchange [Reuses operator id: 60] Output [1]: [ss_item_sk#27] -(78) Sort [codegen id : 37] +(76) Sort [codegen id : 37] Input [1]: [ss_item_sk#27] Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 -(79) SortMergeJoin +(77) SortMergeJoin Left keys [1]: [i_item_sk#6] Right keys [1]: [ss_item_sk#27] Join condition: None -(80) BroadcastExchange +(78) BroadcastExchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#32] -(81) BroadcastHashJoin [codegen id : 38] +(79) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#6] Join condition: None -(82) Project [codegen id : 38] +(80) Project [codegen id : 38] Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#7, i_class_id#8, i_category_id#9] Input [7]: [ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(83) HashAggregate [codegen id : 38] +(81) HashAggregate [codegen id : 38] Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#7, i_class_id#8, i_category_id#9] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#33, isEmpty#34, count#35] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#36, isEmpty#37, count#38] -(84) Exchange +(82) Exchange Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#36, isEmpty#37, count#38] Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#39] -(85) HashAggregate [codegen id : 39] +(83) HashAggregate [codegen id : 39] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#36, isEmpty#37, count#38] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40, count(1)#41] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40 AS sales#42, count(1)#41 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#40 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44] -(86) Filter [codegen id : 39] +(84) Filter [codegen id : 39] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44 as decimal(32,6)) > cast(Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(87) Project [codegen id : 39] +(85) Project [codegen id : 39] Output [6]: [store AS channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#44] -(88) Scan parquet default.catalog_sales +(86) Scan parquet default.catalog_sales Output [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(89) ColumnarToRow [codegen id : 40] +(87) ColumnarToRow [codegen id : 40] Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] -(90) Filter [codegen id : 40] +(88) Filter [codegen id : 40] Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] Condition : (isnotnull(cs_item_sk#19) AND isnotnull(cs_sold_date_sk#18)) -(91) Exchange +(89) Exchange Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] Arguments: hashpartitioning(cs_item_sk#19, 5), ENSURE_REQUIREMENTS, [id=#50] -(92) Sort [codegen id : 41] +(90) Sort [codegen id : 41] Input [4]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49] Arguments: [cs_item_sk#19 ASC NULLS FIRST], false, 0 -(93) ReusedExchange [Reuses operator id: 62] +(91) ReusedExchange [Reuses operator id: 60] Output [1]: [ss_item_sk#27] -(94) Sort [codegen id : 57] +(92) Sort [codegen id : 57] Input [1]: [ss_item_sk#27] Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 -(95) SortMergeJoin +(93) SortMergeJoin Left keys [1]: [cs_item_sk#19] Right keys [1]: [ss_item_sk#27] Join condition: None -(96) ReusedExchange [Reuses operator id: 69] +(94) ReusedExchange [Reuses operator id: 67] Output [1]: [d_date_sk#10] -(97) BroadcastHashJoin [codegen id : 77] +(95) BroadcastHashJoin [codegen id : 77] Left keys [1]: [cs_sold_date_sk#18] Right keys [1]: [d_date_sk#10] Join condition: None -(98) Project [codegen id : 77] +(96) Project [codegen id : 77] Output [3]: [cs_item_sk#19, cs_quantity#48, cs_list_price#49] Input [5]: [cs_sold_date_sk#18, cs_item_sk#19, cs_quantity#48, cs_list_price#49, d_date_sk#10] -(99) ReusedExchange [Reuses operator id: 80] +(97) ReusedExchange [Reuses operator id: 78] Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(100) BroadcastHashJoin [codegen id : 77] +(98) BroadcastHashJoin [codegen id : 77] Left keys [1]: [cs_item_sk#19] Right keys [1]: [i_item_sk#6] Join condition: None -(101) Project [codegen id : 77] +(99) Project [codegen id : 77] Output [5]: [cs_quantity#48, cs_list_price#49, i_brand_id#7, i_class_id#8, i_category_id#9] Input [7]: [cs_item_sk#19, cs_quantity#48, cs_list_price#49, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(102) HashAggregate [codegen id : 77] +(100) HashAggregate [codegen id : 77] Input [5]: [cs_quantity#48, cs_list_price#49, i_brand_id#7, i_class_id#8, i_category_id#9] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#51, isEmpty#52, count#53] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#54, isEmpty#55, count#56] -(103) Exchange +(101) Exchange Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#54, isEmpty#55, count#56] Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#57] -(104) HashAggregate [codegen id : 78] +(102) HashAggregate [codegen id : 78] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#54, isEmpty#55, count#56] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58, count(1)#59] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sales#60, count(1)#59 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62] -(105) Filter [codegen id : 78] +(103) Filter [codegen id : 78] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(106) Project [codegen id : 78] +(104) Project [codegen id : 78] Output [6]: [catalog AS channel#63, i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#62] -(107) Scan parquet default.web_sales +(105) Scan parquet default.web_sales Output [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(108) ColumnarToRow [codegen id : 79] +(106) ColumnarToRow [codegen id : 79] Input [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] -(109) Filter [codegen id : 79] +(107) Filter [codegen id : 79] Input [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] Condition : (isnotnull(ws_item_sk#23) AND isnotnull(ws_sold_date_sk#22)) -(110) Exchange +(108) Exchange Input [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] Arguments: hashpartitioning(ws_item_sk#23, 5), ENSURE_REQUIREMENTS, [id=#66] -(111) Sort [codegen id : 80] +(109) Sort [codegen id : 80] Input [4]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65] Arguments: [ws_item_sk#23 ASC NULLS FIRST], false, 0 -(112) ReusedExchange [Reuses operator id: 62] +(110) ReusedExchange [Reuses operator id: 60] Output [1]: [ss_item_sk#27] -(113) Sort [codegen id : 96] +(111) Sort [codegen id : 96] Input [1]: [ss_item_sk#27] Arguments: [ss_item_sk#27 ASC NULLS FIRST], false, 0 -(114) SortMergeJoin +(112) SortMergeJoin Left keys [1]: [ws_item_sk#23] Right keys [1]: [ss_item_sk#27] Join condition: None -(115) ReusedExchange [Reuses operator id: 69] +(113) ReusedExchange [Reuses operator id: 67] Output [1]: [d_date_sk#10] -(116) BroadcastHashJoin [codegen id : 116] +(114) BroadcastHashJoin [codegen id : 116] Left keys [1]: [ws_sold_date_sk#22] Right keys [1]: [d_date_sk#10] Join condition: None -(117) Project [codegen id : 116] +(115) Project [codegen id : 116] Output [3]: [ws_item_sk#23, ws_quantity#64, ws_list_price#65] Input [5]: [ws_sold_date_sk#22, ws_item_sk#23, ws_quantity#64, ws_list_price#65, d_date_sk#10] -(118) ReusedExchange [Reuses operator id: 80] +(116) ReusedExchange [Reuses operator id: 78] Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(119) BroadcastHashJoin [codegen id : 116] +(117) BroadcastHashJoin [codegen id : 116] Left keys [1]: [ws_item_sk#23] Right keys [1]: [i_item_sk#6] Join condition: None -(120) Project [codegen id : 116] +(118) Project [codegen id : 116] Output [5]: [ws_quantity#64, ws_list_price#65, i_brand_id#7, i_class_id#8, i_category_id#9] Input [7]: [ws_item_sk#23, ws_quantity#64, ws_list_price#65, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(121) HashAggregate [codegen id : 116] +(119) HashAggregate [codegen id : 116] Input [5]: [ws_quantity#64, ws_list_price#65, i_brand_id#7, i_class_id#8, i_category_id#9] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#67, isEmpty#68, count#69] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#70, isEmpty#71, count#72] -(122) Exchange +(120) Exchange Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#70, isEmpty#71, count#72] Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#73] -(123) HashAggregate [codegen id : 117] +(121) HashAggregate [codegen id : 117] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#70, isEmpty#71, count#72] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74, count(1)#75] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74 AS sales#76, count(1)#75 AS number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#74 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78] -(124) Filter [codegen id : 117] +(122) Filter [codegen id : 117] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(125) Project [codegen id : 117] +(123) Project [codegen id : 117] Output [6]: [web AS channel#79, i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#78] -(126) Union +(124) Union -(127) HashAggregate [codegen id : 118] +(125) HashAggregate [codegen id : 118] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(sales#42), partial_sum(number_sales#43)] Aggregate Attributes [3]: [sum#80, isEmpty#81, sum#82] Results [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#83, isEmpty#84, sum#85] -(128) Exchange +(126) Exchange Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#83, isEmpty#84, sum#85] Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#86] -(129) HashAggregate [codegen id : 119] +(127) HashAggregate [codegen id : 119] Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#83, isEmpty#84, sum#85] Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(sales#42), sum(number_sales#43)] Aggregate Attributes [2]: [sum(sales#42)#87, sum(number_sales#43)#88] Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum(sales#42)#87 AS sum_sales#89, sum(number_sales#43)#88 AS number_sales#90] -(130) ReusedExchange [Reuses operator id: 84] +(128) ReusedExchange [Reuses operator id: 82] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#91, isEmpty#92, count#93] -(131) HashAggregate [codegen id : 158] +(129) HashAggregate [codegen id : 158] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#91, isEmpty#92, count#93] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#94, count(1)#95] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#94 AS sales#42, count(1)#95 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#94 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96] -(132) Filter [codegen id : 158] +(130) Filter [codegen id : 158] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(133) Project [codegen id : 158] +(131) Project [codegen id : 158] Output [6]: [store AS channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#96] -(134) ReusedExchange [Reuses operator id: 103] +(132) ReusedExchange [Reuses operator id: 101] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#97, isEmpty#98, count#99] -(135) HashAggregate [codegen id : 197] +(133) HashAggregate [codegen id : 197] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#97, isEmpty#98, count#99] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#100, count(1)#101] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#100 AS sales#60, count(1)#101 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#100 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102] -(136) Filter [codegen id : 197] +(134) Filter [codegen id : 197] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(137) Project [codegen id : 197] +(135) Project [codegen id : 197] Output [6]: [catalog AS channel#103, i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#102] -(138) ReusedExchange [Reuses operator id: 122] +(136) ReusedExchange [Reuses operator id: 120] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#104, isEmpty#105, count#106] -(139) HashAggregate [codegen id : 236] +(137) HashAggregate [codegen id : 236] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#104, isEmpty#105, count#106] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#107, count(1)#108] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sales#76, count(1)#108 AS number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#109] -(140) Filter [codegen id : 236] +(138) Filter [codegen id : 236] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#109] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#109) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#109 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(141) Project [codegen id : 236] +(139) Project [codegen id : 236] Output [6]: [web AS channel#110, i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#109] -(142) Union +(140) Union -(143) HashAggregate [codegen id : 237] +(141) HashAggregate [codegen id : 237] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(sales#42), partial_sum(number_sales#43)] Aggregate Attributes [3]: [sum#111, isEmpty#112, sum#113] Results [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#114, isEmpty#115, sum#116] -(144) Exchange +(142) Exchange Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#114, isEmpty#115, sum#116] Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#117] -(145) HashAggregate [codegen id : 238] +(143) HashAggregate [codegen id : 238] Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#114, isEmpty#115, sum#116] Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(sales#42), sum(number_sales#43)] Aggregate Attributes [2]: [sum(sales#42)#118, sum(number_sales#43)#119] Results [5]: [channel#47, i_brand_id#7, i_class_id#8, sum(sales#42)#118 AS sum_sales#89, sum(number_sales#43)#119 AS number_sales#90] -(146) HashAggregate [codegen id : 238] +(144) HashAggregate [codegen id : 238] Input [5]: [channel#47, i_brand_id#7, i_class_id#8, sum_sales#89, number_sales#90] Keys [3]: [channel#47, i_brand_id#7, i_class_id#8] Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)] Aggregate Attributes [3]: [sum#120, isEmpty#121, sum#122] Results [6]: [channel#47, i_brand_id#7, i_class_id#8, sum#123, isEmpty#124, sum#125] -(147) Exchange +(145) Exchange Input [6]: [channel#47, i_brand_id#7, i_class_id#8, sum#123, isEmpty#124, sum#125] Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, 5), ENSURE_REQUIREMENTS, [id=#126] -(148) HashAggregate [codegen id : 239] +(146) HashAggregate [codegen id : 239] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, sum#123, isEmpty#124, sum#125] Keys [3]: [channel#47, i_brand_id#7, i_class_id#8] Functions [2]: [sum(sum_sales#89), sum(number_sales#90)] Aggregate Attributes [2]: [sum(sum_sales#89)#127, sum(number_sales#90)#128] Results [6]: [channel#47, i_brand_id#7, i_class_id#8, null AS i_category_id#129, sum(sum_sales#89)#127 AS sum(sum_sales)#130, sum(number_sales#90)#128 AS sum(number_sales)#131] -(149) ReusedExchange [Reuses operator id: 84] +(147) ReusedExchange [Reuses operator id: 82] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#132, isEmpty#133, count#134] -(150) HashAggregate [codegen id : 278] +(148) HashAggregate [codegen id : 278] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#132, isEmpty#133, count#134] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#135, count(1)#136] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#135 AS sales#42, count(1)#136 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#135 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#137] -(151) Filter [codegen id : 278] +(149) Filter [codegen id : 278] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#137] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#137) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#137 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(152) Project [codegen id : 278] +(150) Project [codegen id : 278] Output [6]: [store AS channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#137] -(153) ReusedExchange [Reuses operator id: 103] +(151) ReusedExchange [Reuses operator id: 101] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#138, isEmpty#139, count#140] -(154) HashAggregate [codegen id : 317] +(152) HashAggregate [codegen id : 317] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#138, isEmpty#139, count#140] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#141, count(1)#142] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#141 AS sales#60, count(1)#142 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#141 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#143] -(155) Filter [codegen id : 317] +(153) Filter [codegen id : 317] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#143] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#143) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#143 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(156) Project [codegen id : 317] +(154) Project [codegen id : 317] Output [6]: [catalog AS channel#144, i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#143] -(157) ReusedExchange [Reuses operator id: 122] +(155) ReusedExchange [Reuses operator id: 120] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#145, isEmpty#146, count#147] -(158) HashAggregate [codegen id : 356] +(156) HashAggregate [codegen id : 356] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#145, isEmpty#146, count#147] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#148, count(1)#149] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#148 AS sales#76, count(1)#149 AS number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#148 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#150] -(159) Filter [codegen id : 356] +(157) Filter [codegen id : 356] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#150] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#150) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#150 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(160) Project [codegen id : 356] +(158) Project [codegen id : 356] Output [6]: [web AS channel#151, i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#150] -(161) Union +(159) Union -(162) HashAggregate [codegen id : 357] +(160) HashAggregate [codegen id : 357] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(sales#42), partial_sum(number_sales#43)] Aggregate Attributes [3]: [sum#152, isEmpty#153, sum#154] Results [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#155, isEmpty#156, sum#157] -(163) Exchange +(161) Exchange Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#155, isEmpty#156, sum#157] Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#158] -(164) HashAggregate [codegen id : 358] +(162) HashAggregate [codegen id : 358] Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#155, isEmpty#156, sum#157] Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(sales#42), sum(number_sales#43)] Aggregate Attributes [2]: [sum(sales#42)#159, sum(number_sales#43)#160] Results [4]: [channel#47, i_brand_id#7, sum(sales#42)#159 AS sum_sales#89, sum(number_sales#43)#160 AS number_sales#90] -(165) HashAggregate [codegen id : 358] +(163) HashAggregate [codegen id : 358] Input [4]: [channel#47, i_brand_id#7, sum_sales#89, number_sales#90] Keys [2]: [channel#47, i_brand_id#7] Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)] Aggregate Attributes [3]: [sum#161, isEmpty#162, sum#163] Results [5]: [channel#47, i_brand_id#7, sum#164, isEmpty#165, sum#166] -(166) Exchange +(164) Exchange Input [5]: [channel#47, i_brand_id#7, sum#164, isEmpty#165, sum#166] Arguments: hashpartitioning(channel#47, i_brand_id#7, 5), ENSURE_REQUIREMENTS, [id=#167] -(167) HashAggregate [codegen id : 359] +(165) HashAggregate [codegen id : 359] Input [5]: [channel#47, i_brand_id#7, sum#164, isEmpty#165, sum#166] Keys [2]: [channel#47, i_brand_id#7] Functions [2]: [sum(sum_sales#89), sum(number_sales#90)] Aggregate Attributes [2]: [sum(sum_sales#89)#168, sum(number_sales#90)#169] Results [6]: [channel#47, i_brand_id#7, null AS i_class_id#170, null AS i_category_id#171, sum(sum_sales#89)#168 AS sum(sum_sales)#172, sum(number_sales#90)#169 AS sum(number_sales)#173] -(168) ReusedExchange [Reuses operator id: 84] +(166) ReusedExchange [Reuses operator id: 82] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#174, isEmpty#175, count#176] -(169) HashAggregate [codegen id : 398] +(167) HashAggregate [codegen id : 398] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#174, isEmpty#175, count#176] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#177, count(1)#178] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#177 AS sales#42, count(1)#178 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#177 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#179] -(170) Filter [codegen id : 398] +(168) Filter [codegen id : 398] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#179] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#179) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#179 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(171) Project [codegen id : 398] +(169) Project [codegen id : 398] Output [6]: [store AS channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#179] -(172) ReusedExchange [Reuses operator id: 103] +(170) ReusedExchange [Reuses operator id: 101] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#180, isEmpty#181, count#182] -(173) HashAggregate [codegen id : 437] +(171) HashAggregate [codegen id : 437] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#180, isEmpty#181, count#182] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#183, count(1)#184] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#183 AS sales#60, count(1)#184 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#183 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#185] -(174) Filter [codegen id : 437] +(172) Filter [codegen id : 437] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#185] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#185) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#185 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(175) Project [codegen id : 437] +(173) Project [codegen id : 437] Output [6]: [catalog AS channel#186, i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#185] -(176) ReusedExchange [Reuses operator id: 122] +(174) ReusedExchange [Reuses operator id: 120] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#187, isEmpty#188, count#189] -(177) HashAggregate [codegen id : 476] +(175) HashAggregate [codegen id : 476] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#187, isEmpty#188, count#189] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#190, count(1)#191] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#190 AS sales#76, count(1)#191 AS number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#190 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#192] -(178) Filter [codegen id : 476] +(176) Filter [codegen id : 476] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#192] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#192) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#192 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(179) Project [codegen id : 476] +(177) Project [codegen id : 476] Output [6]: [web AS channel#193, i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#192] -(180) Union +(178) Union -(181) HashAggregate [codegen id : 477] +(179) HashAggregate [codegen id : 477] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(sales#42), partial_sum(number_sales#43)] Aggregate Attributes [3]: [sum#194, isEmpty#195, sum#196] Results [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#197, isEmpty#198, sum#199] -(182) Exchange +(180) Exchange Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#197, isEmpty#198, sum#199] Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#200] -(183) HashAggregate [codegen id : 478] +(181) HashAggregate [codegen id : 478] Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#197, isEmpty#198, sum#199] Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(sales#42), sum(number_sales#43)] Aggregate Attributes [2]: [sum(sales#42)#201, sum(number_sales#43)#202] Results [3]: [channel#47, sum(sales#42)#201 AS sum_sales#89, sum(number_sales#43)#202 AS number_sales#90] -(184) HashAggregate [codegen id : 478] +(182) HashAggregate [codegen id : 478] Input [3]: [channel#47, sum_sales#89, number_sales#90] Keys [1]: [channel#47] Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)] Aggregate Attributes [3]: [sum#203, isEmpty#204, sum#205] Results [4]: [channel#47, sum#206, isEmpty#207, sum#208] -(185) Exchange +(183) Exchange Input [4]: [channel#47, sum#206, isEmpty#207, sum#208] Arguments: hashpartitioning(channel#47, 5), ENSURE_REQUIREMENTS, [id=#209] -(186) HashAggregate [codegen id : 479] +(184) HashAggregate [codegen id : 479] Input [4]: [channel#47, sum#206, isEmpty#207, sum#208] Keys [1]: [channel#47] Functions [2]: [sum(sum_sales#89), sum(number_sales#90)] Aggregate Attributes [2]: [sum(sum_sales#89)#210, sum(number_sales#90)#211] Results [6]: [channel#47, null AS i_brand_id#212, null AS i_class_id#213, null AS i_category_id#214, sum(sum_sales#89)#210 AS sum(sum_sales)#215, sum(number_sales#90)#211 AS sum(number_sales)#216] -(187) ReusedExchange [Reuses operator id: 84] +(185) ReusedExchange [Reuses operator id: 82] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#217, isEmpty#218, count#219] -(188) HashAggregate [codegen id : 518] +(186) HashAggregate [codegen id : 518] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#217, isEmpty#218, count#219] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#220, count(1)#221] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#220 AS sales#42, count(1)#221 AS number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#220 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#222] -(189) Filter [codegen id : 518] +(187) Filter [codegen id : 518] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#222] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#222) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#222 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(190) Project [codegen id : 518] +(188) Project [codegen id : 518] Output [6]: [store AS channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#222] -(191) ReusedExchange [Reuses operator id: 103] +(189) ReusedExchange [Reuses operator id: 101] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#223, isEmpty#224, count#225] -(192) HashAggregate [codegen id : 557] +(190) HashAggregate [codegen id : 557] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#223, isEmpty#224, count#225] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#226, count(1)#227] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#226 AS sales#60, count(1)#227 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#226 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#228] -(193) Filter [codegen id : 557] +(191) Filter [codegen id : 557] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#228] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#228) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#228 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(194) Project [codegen id : 557] +(192) Project [codegen id : 557] Output [6]: [catalog AS channel#229, i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#49 as decimal(12,2)))), DecimalType(18,2), true))#228] -(195) ReusedExchange [Reuses operator id: 122] +(193) ReusedExchange [Reuses operator id: 120] Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#230, isEmpty#231, count#232] -(196) HashAggregate [codegen id : 596] +(194) HashAggregate [codegen id : 596] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#230, isEmpty#231, count#232] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#233, count(1)#234] Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#233 AS sales#76, count(1)#234 AS number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#233 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#235] -(197) Filter [codegen id : 596] +(195) Filter [codegen id : 596] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#235] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#235) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#235 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#45, [id=#46] as decimal(32,6)))) -(198) Project [codegen id : 596] +(196) Project [codegen id : 596] Output [6]: [web AS channel#236, i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77] Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#76, number_sales#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#235] -(199) Union +(197) Union -(200) HashAggregate [codegen id : 597] +(198) HashAggregate [codegen id : 597] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sales#42, number_sales#43] Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(sales#42), partial_sum(number_sales#43)] Aggregate Attributes [3]: [sum#237, isEmpty#238, sum#239] Results [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#240, isEmpty#241, sum#242] -(201) Exchange +(199) Exchange Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#240, isEmpty#241, sum#242] Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#243] -(202) HashAggregate [codegen id : 598] +(200) HashAggregate [codegen id : 598] Input [7]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum#240, isEmpty#241, sum#242] Keys [4]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(sales#42), sum(number_sales#43)] Aggregate Attributes [2]: [sum(sales#42)#244, sum(number_sales#43)#245] Results [2]: [sum(sales#42)#244 AS sum_sales#89, sum(number_sales#43)#245 AS number_sales#90] -(203) HashAggregate [codegen id : 598] +(201) HashAggregate [codegen id : 598] Input [2]: [sum_sales#89, number_sales#90] Keys: [] Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)] Aggregate Attributes [3]: [sum#246, isEmpty#247, sum#248] Results [3]: [sum#249, isEmpty#250, sum#251] -(204) Exchange +(202) Exchange Input [3]: [sum#249, isEmpty#250, sum#251] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#252] -(205) HashAggregate [codegen id : 599] +(203) HashAggregate [codegen id : 599] Input [3]: [sum#249, isEmpty#250, sum#251] Keys: [] Functions [2]: [sum(sum_sales#89), sum(number_sales#90)] Aggregate Attributes [2]: [sum(sum_sales#89)#253, sum(number_sales#90)#254] Results [6]: [null AS channel#255, null AS i_brand_id#256, null AS i_class_id#257, null AS i_category_id#258, sum(sum_sales#89)#253 AS sum(sum_sales)#259, sum(number_sales#90)#254 AS sum(number_sales)#260] -(206) Union +(204) Union -(207) HashAggregate [codegen id : 600] +(205) HashAggregate [codegen id : 600] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Functions: [] Aggregate Attributes: [] Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] -(208) Exchange +(206) Exchange Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Arguments: hashpartitioning(channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90, 5), ENSURE_REQUIREMENTS, [id=#261] -(209) HashAggregate [codegen id : 601] +(207) HashAggregate [codegen id : 601] Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Keys [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Functions: [] Aggregate Attributes: [] Results [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] -(210) TakeOrderedAndProject +(208) TakeOrderedAndProject Input [6]: [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] Arguments: 100, [channel#47 ASC NULLS FIRST, i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#47, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#89, number_sales#90] ===== Subqueries ===== -Subquery:1 Hosting operator id = 86 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* HashAggregate (240) -+- Exchange (239) - +- * HashAggregate (238) - +- Union (237) - :- * Project (220) - : +- * BroadcastHashJoin Inner BuildRight (219) - : :- * Filter (213) - : : +- * ColumnarToRow (212) - : : +- Scan parquet default.store_sales (211) - : +- BroadcastExchange (218) - : +- * Project (217) - : +- * Filter (216) - : +- * ColumnarToRow (215) - : +- Scan parquet default.date_dim (214) - :- * Project (230) - : +- * BroadcastHashJoin Inner BuildRight (229) - : :- * Filter (223) - : : +- * ColumnarToRow (222) - : : +- Scan parquet default.catalog_sales (221) - : +- BroadcastExchange (228) - : +- * Project (227) - : +- * Filter (226) - : +- * ColumnarToRow (225) - : +- Scan parquet default.date_dim (224) - +- * Project (236) - +- * BroadcastHashJoin Inner BuildRight (235) - :- * Filter (233) - : +- * ColumnarToRow (232) - : +- Scan parquet default.web_sales (231) - +- ReusedExchange (234) - - -(211) Scan parquet default.store_sales +Subquery:1 Hosting operator id = 84 Hosting Expression = Subquery scalar-subquery#45, [id=#46] +* HashAggregate (238) ++- Exchange (237) + +- * HashAggregate (236) + +- Union (235) + :- * Project (218) + : +- * BroadcastHashJoin Inner BuildRight (217) + : :- * Filter (211) + : : +- * ColumnarToRow (210) + : : +- Scan parquet default.store_sales (209) + : +- BroadcastExchange (216) + : +- * Project (215) + : +- * Filter (214) + : +- * ColumnarToRow (213) + : +- Scan parquet default.date_dim (212) + :- * Project (228) + : +- * BroadcastHashJoin Inner BuildRight (227) + : :- * Filter (221) + : : +- * ColumnarToRow (220) + : : +- Scan parquet default.catalog_sales (219) + : +- BroadcastExchange (226) + : +- * Project (225) + : +- * Filter (224) + : +- * ColumnarToRow (223) + : +- Scan parquet default.date_dim (222) + +- * Project (234) + +- * BroadcastHashJoin Inner BuildRight (233) + :- * Filter (231) + : +- * ColumnarToRow (230) + : +- Scan parquet default.web_sales (229) + +- ReusedExchange (232) + + +(209) Scan parquet default.store_sales Output [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(212) ColumnarToRow [codegen id : 2] +(210) ColumnarToRow [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] -(213) Filter [codegen id : 2] +(211) Filter [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Condition : isnotnull(ss_sold_date_sk#1) -(214) Scan parquet default.date_dim +(212) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_year#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(215) ColumnarToRow [codegen id : 1] +(213) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] -(216) Filter [codegen id : 1] +(214) Filter [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] Condition : (((isnotnull(d_year#11) AND (d_year#11 >= 1999)) AND (d_year#11 <= 2001)) AND isnotnull(d_date_sk#10)) -(217) Project [codegen id : 1] +(215) Project [codegen id : 1] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_year#11] -(218) BroadcastExchange +(216) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#262] -(219) BroadcastHashJoin [codegen id : 2] +(217) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(220) Project [codegen id : 2] +(218) Project [codegen id : 2] Output [2]: [ss_quantity#3 AS quantity#263, ss_list_price#4 AS list_price#264] Input [4]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(221) Scan parquet default.catalog_sales +(219) Scan parquet default.catalog_sales Output [3]: [cs_sold_date_sk#18, cs_quantity#48, cs_list_price#49] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(222) ColumnarToRow [codegen id : 4] +(220) ColumnarToRow [codegen id : 4] Input [3]: [cs_sold_date_sk#18, cs_quantity#48, cs_list_price#49] -(223) Filter [codegen id : 4] +(221) Filter [codegen id : 4] Input [3]: [cs_sold_date_sk#18, cs_quantity#48, cs_list_price#49] Condition : isnotnull(cs_sold_date_sk#18) -(224) Scan parquet default.date_dim +(222) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_year#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(225) ColumnarToRow [codegen id : 3] +(223) ColumnarToRow [codegen id : 3] Input [2]: [d_date_sk#10, d_year#11] -(226) Filter [codegen id : 3] +(224) Filter [codegen id : 3] Input [2]: [d_date_sk#10, d_year#11] Condition : (((isnotnull(d_year#11) AND (d_year#11 >= 1998)) AND (d_year#11 <= 2000)) AND isnotnull(d_date_sk#10)) -(227) Project [codegen id : 3] +(225) Project [codegen id : 3] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_year#11] -(228) BroadcastExchange +(226) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#265] -(229) BroadcastHashJoin [codegen id : 4] +(227) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#18] Right keys [1]: [d_date_sk#10] Join condition: None -(230) Project [codegen id : 4] +(228) Project [codegen id : 4] Output [2]: [cs_quantity#48 AS quantity#266, cs_list_price#49 AS list_price#267] Input [4]: [cs_sold_date_sk#18, cs_quantity#48, cs_list_price#49, d_date_sk#10] -(231) Scan parquet default.web_sales +(229) Scan parquet default.web_sales Output [3]: [ws_sold_date_sk#22, ws_quantity#64, ws_list_price#65] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(232) ColumnarToRow [codegen id : 6] +(230) ColumnarToRow [codegen id : 6] Input [3]: [ws_sold_date_sk#22, ws_quantity#64, ws_list_price#65] -(233) Filter [codegen id : 6] +(231) Filter [codegen id : 6] Input [3]: [ws_sold_date_sk#22, ws_quantity#64, ws_list_price#65] Condition : isnotnull(ws_sold_date_sk#22) -(234) ReusedExchange [Reuses operator id: 228] +(232) ReusedExchange [Reuses operator id: 226] Output [1]: [d_date_sk#10] -(235) BroadcastHashJoin [codegen id : 6] +(233) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#22] Right keys [1]: [d_date_sk#10] Join condition: None -(236) Project [codegen id : 6] +(234) Project [codegen id : 6] Output [2]: [ws_quantity#64 AS quantity#268, ws_list_price#65 AS list_price#269] Input [4]: [ws_sold_date_sk#22, ws_quantity#64, ws_list_price#65, d_date_sk#10] -(237) Union +(235) Union -(238) HashAggregate [codegen id : 7] +(236) HashAggregate [codegen id : 7] Input [2]: [quantity#263, list_price#264] Keys: [] Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#263 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#264 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [2]: [sum#270, count#271] Results [2]: [sum#272, count#273] -(239) Exchange +(237) Exchange Input [2]: [sum#272, count#273] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#274] -(240) HashAggregate [codegen id : 8] +(238) HashAggregate [codegen id : 8] Input [2]: [sum#272, count#273] Keys: [] Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#263 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#264 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#263 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#264 as decimal(12,2)))), DecimalType(18,2), true))#275] Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#263 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#264 as decimal(12,2)))), DecimalType(18,2), true))#275 AS average_sales#276] -Subquery:2 Hosting operator id = 105 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:2 Hosting operator id = 103 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:3 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:3 Hosting operator id = 122 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:4 Hosting operator id = 132 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:4 Hosting operator id = 130 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:5 Hosting operator id = 136 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:5 Hosting operator id = 134 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:6 Hosting operator id = 140 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:6 Hosting operator id = 138 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:7 Hosting operator id = 151 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:7 Hosting operator id = 149 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:8 Hosting operator id = 155 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:8 Hosting operator id = 153 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:9 Hosting operator id = 159 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:9 Hosting operator id = 157 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:10 Hosting operator id = 170 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:10 Hosting operator id = 168 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:11 Hosting operator id = 174 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:11 Hosting operator id = 172 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:12 Hosting operator id = 178 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:12 Hosting operator id = 176 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:13 Hosting operator id = 189 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:13 Hosting operator id = 187 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:14 Hosting operator id = 193 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:14 Hosting operator id = 191 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:15 Hosting operator id = 197 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:15 Hosting operator id = 195 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt index c63f1b8a7564..44a7bcd7cd9f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt @@ -101,82 +101,80 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastExchange #6 WholeStageCodegen (16) HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (15) - HashAggregate [brand_id,class_id,category_id] - InputAdapter - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (6) - Sort [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #8 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] + InputAdapter + Exchange [brand_id,class_id,category_id] #7 + WholeStageCodegen (15) + HashAggregate [brand_id,class_id,category_id] + InputAdapter + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (6) + Sort [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #8 + WholeStageCodegen (5) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk,ss_sold_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #9 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] InputAdapter - BroadcastExchange #10 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (10) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #11 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] + BroadcastExchange #9 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (10) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #11 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk,cs_sold_date_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #9 + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] InputAdapter - BroadcastExchange #12 - WholeStageCodegen (8) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (14) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #13 - WholeStageCodegen (13) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk,ws_sold_date_sk] + ReusedExchange [d_date_sk] #9 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (8) + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (14) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #13 + WholeStageCodegen (13) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk,ws_sold_date_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #9 + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + ReusedExchange [d_date_sk] #9 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 InputAdapter BroadcastExchange #14 WholeStageCodegen (19) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt index 4e60a9b6b154..db16a22e74d3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt @@ -1,198 +1,196 @@ == Physical Plan == -TakeOrderedAndProject (194) -+- * HashAggregate (193) - +- Exchange (192) - +- * HashAggregate (191) - +- Union (190) - :- * HashAggregate (113) - : +- Exchange (112) - : +- * HashAggregate (111) - : +- Union (110) - : :- * Project (77) - : : +- * Filter (76) - : : +- * HashAggregate (75) - : : +- Exchange (74) - : : +- * HashAggregate (73) - : : +- * Project (72) - : : +- * BroadcastHashJoin Inner BuildRight (71) - : : :- * Project (65) - : : : +- * BroadcastHashJoin Inner BuildRight (64) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (57) +TakeOrderedAndProject (192) ++- * HashAggregate (191) + +- Exchange (190) + +- * HashAggregate (189) + +- Union (188) + :- * HashAggregate (111) + : +- Exchange (110) + : +- * HashAggregate (109) + : +- Union (108) + : :- * Project (75) + : : +- * Filter (74) + : : +- * HashAggregate (73) + : : +- Exchange (72) + : : +- * HashAggregate (71) + : : +- * Project (70) + : : +- * BroadcastHashJoin Inner BuildRight (69) + : : :- * Project (63) + : : : +- * BroadcastHashJoin Inner BuildRight (62) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (55) : : : : :- * Filter (3) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet default.store_sales (1) - : : : : +- BroadcastExchange (56) - : : : : +- * Project (55) - : : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : : +- BroadcastExchange (54) + : : : : +- * Project (53) + : : : : +- * BroadcastHashJoin Inner BuildRight (52) : : : : :- * Filter (6) : : : : : +- * ColumnarToRow (5) : : : : : +- Scan parquet default.item (4) - : : : : +- BroadcastExchange (53) - : : : : +- * HashAggregate (52) - : : : : +- * HashAggregate (51) - : : : : +- * HashAggregate (50) - : : : : +- Exchange (49) - : : : : +- * HashAggregate (48) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : : :- * BroadcastHashJoin LeftSemi BuildRight (36) - : : : : : :- * Project (22) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : : : : :- * Project (15) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : : :- * Filter (9) - : : : : : : : : +- * ColumnarToRow (8) - : : : : : : : : +- Scan parquet default.store_sales (7) - : : : : : : : +- BroadcastExchange (13) - : : : : : : : +- * Filter (12) - : : : : : : : +- * ColumnarToRow (11) - : : : : : : : +- Scan parquet default.item (10) - : : : : : : +- BroadcastExchange (20) - : : : : : : +- * Project (19) - : : : : : : +- * Filter (18) - : : : : : : +- * ColumnarToRow (17) - : : : : : : +- Scan parquet default.date_dim (16) - : : : : : +- BroadcastExchange (35) - : : : : : +- * Project (34) - : : : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : : : :- * Project (31) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : : : : :- * Filter (25) - : : : : : : : +- * ColumnarToRow (24) - : : : : : : : +- Scan parquet default.catalog_sales (23) - : : : : : : +- BroadcastExchange (29) - : : : : : : +- * Filter (28) - : : : : : : +- * ColumnarToRow (27) - : : : : : : +- Scan parquet default.item (26) - : : : : : +- ReusedExchange (32) - : : : : +- BroadcastExchange (46) - : : : : +- * Project (45) - : : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : : :- * Project (42) - : : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : : :- * Filter (39) - : : : : : : +- * ColumnarToRow (38) - : : : : : : +- Scan parquet default.web_sales (37) - : : : : : +- ReusedExchange (40) - : : : : +- ReusedExchange (43) - : : : +- BroadcastExchange (63) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (62) - : : : :- * Filter (60) - : : : : +- * ColumnarToRow (59) - : : : : +- Scan parquet default.item (58) - : : : +- ReusedExchange (61) - : : +- BroadcastExchange (70) - : : +- * Project (69) - : : +- * Filter (68) - : : +- * ColumnarToRow (67) - : : +- Scan parquet default.date_dim (66) - : :- * Project (93) - : : +- * Filter (92) - : : +- * HashAggregate (91) - : : +- Exchange (90) - : : +- * HashAggregate (89) - : : +- * Project (88) - : : +- * BroadcastHashJoin Inner BuildRight (87) - : : :- * Project (85) - : : : +- * BroadcastHashJoin Inner BuildRight (84) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (82) - : : : : :- * Filter (80) - : : : : : +- * ColumnarToRow (79) - : : : : : +- Scan parquet default.catalog_sales (78) - : : : : +- ReusedExchange (81) - : : : +- ReusedExchange (83) - : : +- ReusedExchange (86) - : +- * Project (109) - : +- * Filter (108) - : +- * HashAggregate (107) - : +- Exchange (106) - : +- * HashAggregate (105) - : +- * Project (104) - : +- * BroadcastHashJoin Inner BuildRight (103) - : :- * Project (101) - : : +- * BroadcastHashJoin Inner BuildRight (100) - : : :- * BroadcastHashJoin LeftSemi BuildRight (98) - : : : :- * Filter (96) - : : : : +- * ColumnarToRow (95) - : : : : +- Scan parquet default.web_sales (94) - : : : +- ReusedExchange (97) - : : +- ReusedExchange (99) - : +- ReusedExchange (102) - :- * HashAggregate (132) - : +- Exchange (131) - : +- * HashAggregate (130) - : +- * HashAggregate (129) - : +- Exchange (128) - : +- * HashAggregate (127) - : +- Union (126) - : :- * Project (117) - : : +- * Filter (116) - : : +- * HashAggregate (115) - : : +- ReusedExchange (114) - : :- * Project (121) - : : +- * Filter (120) - : : +- * HashAggregate (119) - : : +- ReusedExchange (118) - : +- * Project (125) - : +- * Filter (124) - : +- * HashAggregate (123) - : +- ReusedExchange (122) - :- * HashAggregate (151) - : +- Exchange (150) - : +- * HashAggregate (149) - : +- * HashAggregate (148) - : +- Exchange (147) - : +- * HashAggregate (146) - : +- Union (145) - : :- * Project (136) - : : +- * Filter (135) - : : +- * HashAggregate (134) - : : +- ReusedExchange (133) - : :- * Project (140) - : : +- * Filter (139) - : : +- * HashAggregate (138) - : : +- ReusedExchange (137) - : +- * Project (144) - : +- * Filter (143) - : +- * HashAggregate (142) - : +- ReusedExchange (141) - :- * HashAggregate (170) - : +- Exchange (169) - : +- * HashAggregate (168) - : +- * HashAggregate (167) - : +- Exchange (166) - : +- * HashAggregate (165) - : +- Union (164) - : :- * Project (155) - : : +- * Filter (154) - : : +- * HashAggregate (153) - : : +- ReusedExchange (152) - : :- * Project (159) - : : +- * Filter (158) - : : +- * HashAggregate (157) - : : +- ReusedExchange (156) - : +- * Project (163) - : +- * Filter (162) - : +- * HashAggregate (161) - : +- ReusedExchange (160) - +- * HashAggregate (189) - +- Exchange (188) - +- * HashAggregate (187) - +- * HashAggregate (186) - +- Exchange (185) - +- * HashAggregate (184) - +- Union (183) - :- * Project (174) - : +- * Filter (173) - : +- * HashAggregate (172) - : +- ReusedExchange (171) - :- * Project (178) - : +- * Filter (177) - : +- * HashAggregate (176) - : +- ReusedExchange (175) - +- * Project (182) - +- * Filter (181) - +- * HashAggregate (180) - +- ReusedExchange (179) + : : : : +- BroadcastExchange (51) + : : : : +- * HashAggregate (50) + : : : : +- Exchange (49) + : : : : +- * HashAggregate (48) + : : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) + : : : : :- * BroadcastHashJoin LeftSemi BuildRight (36) + : : : : : :- * Project (22) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : : : : :- * Project (15) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : : : :- * Filter (9) + : : : : : : : : +- * ColumnarToRow (8) + : : : : : : : : +- Scan parquet default.store_sales (7) + : : : : : : : +- BroadcastExchange (13) + : : : : : : : +- * Filter (12) + : : : : : : : +- * ColumnarToRow (11) + : : : : : : : +- Scan parquet default.item (10) + : : : : : : +- BroadcastExchange (20) + : : : : : : +- * Project (19) + : : : : : : +- * Filter (18) + : : : : : : +- * ColumnarToRow (17) + : : : : : : +- Scan parquet default.date_dim (16) + : : : : : +- BroadcastExchange (35) + : : : : : +- * Project (34) + : : : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : : : :- * Project (31) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (30) + : : : : : : :- * Filter (25) + : : : : : : : +- * ColumnarToRow (24) + : : : : : : : +- Scan parquet default.catalog_sales (23) + : : : : : : +- BroadcastExchange (29) + : : : : : : +- * Filter (28) + : : : : : : +- * ColumnarToRow (27) + : : : : : : +- Scan parquet default.item (26) + : : : : : +- ReusedExchange (32) + : : : : +- BroadcastExchange (46) + : : : : +- * Project (45) + : : : : +- * BroadcastHashJoin Inner BuildRight (44) + : : : : :- * Project (42) + : : : : : +- * BroadcastHashJoin Inner BuildRight (41) + : : : : : :- * Filter (39) + : : : : : : +- * ColumnarToRow (38) + : : : : : : +- Scan parquet default.web_sales (37) + : : : : : +- ReusedExchange (40) + : : : : +- ReusedExchange (43) + : : : +- BroadcastExchange (61) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (60) + : : : :- * Filter (58) + : : : : +- * ColumnarToRow (57) + : : : : +- Scan parquet default.item (56) + : : : +- ReusedExchange (59) + : : +- BroadcastExchange (68) + : : +- * Project (67) + : : +- * Filter (66) + : : +- * ColumnarToRow (65) + : : +- Scan parquet default.date_dim (64) + : :- * Project (91) + : : +- * Filter (90) + : : +- * HashAggregate (89) + : : +- Exchange (88) + : : +- * HashAggregate (87) + : : +- * Project (86) + : : +- * BroadcastHashJoin Inner BuildRight (85) + : : :- * Project (83) + : : : +- * BroadcastHashJoin Inner BuildRight (82) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (80) + : : : : :- * Filter (78) + : : : : : +- * ColumnarToRow (77) + : : : : : +- Scan parquet default.catalog_sales (76) + : : : : +- ReusedExchange (79) + : : : +- ReusedExchange (81) + : : +- ReusedExchange (84) + : +- * Project (107) + : +- * Filter (106) + : +- * HashAggregate (105) + : +- Exchange (104) + : +- * HashAggregate (103) + : +- * Project (102) + : +- * BroadcastHashJoin Inner BuildRight (101) + : :- * Project (99) + : : +- * BroadcastHashJoin Inner BuildRight (98) + : : :- * BroadcastHashJoin LeftSemi BuildRight (96) + : : : :- * Filter (94) + : : : : +- * ColumnarToRow (93) + : : : : +- Scan parquet default.web_sales (92) + : : : +- ReusedExchange (95) + : : +- ReusedExchange (97) + : +- ReusedExchange (100) + :- * HashAggregate (130) + : +- Exchange (129) + : +- * HashAggregate (128) + : +- * HashAggregate (127) + : +- Exchange (126) + : +- * HashAggregate (125) + : +- Union (124) + : :- * Project (115) + : : +- * Filter (114) + : : +- * HashAggregate (113) + : : +- ReusedExchange (112) + : :- * Project (119) + : : +- * Filter (118) + : : +- * HashAggregate (117) + : : +- ReusedExchange (116) + : +- * Project (123) + : +- * Filter (122) + : +- * HashAggregate (121) + : +- ReusedExchange (120) + :- * HashAggregate (149) + : +- Exchange (148) + : +- * HashAggregate (147) + : +- * HashAggregate (146) + : +- Exchange (145) + : +- * HashAggregate (144) + : +- Union (143) + : :- * Project (134) + : : +- * Filter (133) + : : +- * HashAggregate (132) + : : +- ReusedExchange (131) + : :- * Project (138) + : : +- * Filter (137) + : : +- * HashAggregate (136) + : : +- ReusedExchange (135) + : +- * Project (142) + : +- * Filter (141) + : +- * HashAggregate (140) + : +- ReusedExchange (139) + :- * HashAggregate (168) + : +- Exchange (167) + : +- * HashAggregate (166) + : +- * HashAggregate (165) + : +- Exchange (164) + : +- * HashAggregate (163) + : +- Union (162) + : :- * Project (153) + : : +- * Filter (152) + : : +- * HashAggregate (151) + : : +- ReusedExchange (150) + : :- * Project (157) + : : +- * Filter (156) + : : +- * HashAggregate (155) + : : +- ReusedExchange (154) + : +- * Project (161) + : +- * Filter (160) + : +- * HashAggregate (159) + : +- ReusedExchange (158) + +- * HashAggregate (187) + +- Exchange (186) + +- * HashAggregate (185) + +- * HashAggregate (184) + +- Exchange (183) + +- * HashAggregate (182) + +- Union (181) + :- * Project (172) + : +- * Filter (171) + : +- * HashAggregate (170) + : +- ReusedExchange (169) + :- * Project (176) + : +- * Filter (175) + : +- * HashAggregate (174) + : +- ReusedExchange (173) + +- * Project (180) + +- * Filter (179) + +- * HashAggregate (178) + +- ReusedExchange (177) (1) Scan parquet default.store_sales @@ -422,887 +420,873 @@ Functions: [] Aggregate Attributes: [] Results [3]: [brand_id#13, class_id#14, category_id#15] -(51) HashAggregate [codegen id : 10] -Input [3]: [brand_id#13, class_id#14, category_id#15] -Keys [3]: [brand_id#13, class_id#14, category_id#15] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#13, class_id#14, category_id#15] - -(52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#13, class_id#14, category_id#15] -Keys [3]: [brand_id#13, class_id#14, category_id#15] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#13, class_id#14, category_id#15] - -(53) BroadcastExchange +(51) BroadcastExchange Input [3]: [brand_id#13, class_id#14, category_id#15] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#24] -(54) BroadcastHashJoin [codegen id : 11] +(52) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Right keys [3]: [brand_id#13, class_id#14, category_id#15] Join condition: None -(55) Project [codegen id : 11] +(53) Project [codegen id : 11] Output [1]: [i_item_sk#5 AS ss_item_sk#25] Input [7]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8, brand_id#13, class_id#14, category_id#15] -(56) BroadcastExchange +(54) BroadcastExchange Input [1]: [ss_item_sk#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] -(57) BroadcastHashJoin [codegen id : 25] +(55) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#2] Right keys [1]: [ss_item_sk#25] Join condition: None -(58) Scan parquet default.item +(56) Scan parquet default.item Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(59) ColumnarToRow [codegen id : 23] +(57) ColumnarToRow [codegen id : 23] Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(60) Filter [codegen id : 23] +(58) Filter [codegen id : 23] Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Condition : isnotnull(i_item_sk#5) -(61) ReusedExchange [Reuses operator id: 56] +(59) ReusedExchange [Reuses operator id: 54] Output [1]: [ss_item_sk#25] -(62) BroadcastHashJoin [codegen id : 23] +(60) BroadcastHashJoin [codegen id : 23] Left keys [1]: [i_item_sk#5] Right keys [1]: [ss_item_sk#25] Join condition: None -(63) BroadcastExchange +(61) BroadcastExchange Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] -(64) BroadcastHashJoin [codegen id : 25] +(62) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#2] Right keys [1]: [i_item_sk#5] Join condition: None -(65) Project [codegen id : 25] +(63) Project [codegen id : 25] Output [6]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8] Input [8]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(66) Scan parquet default.date_dim +(64) Scan parquet default.date_dim Output [3]: [d_date_sk#10, d_year#11, d_moy#28] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(67) ColumnarToRow [codegen id : 24] +(65) ColumnarToRow [codegen id : 24] Input [3]: [d_date_sk#10, d_year#11, d_moy#28] -(68) Filter [codegen id : 24] +(66) Filter [codegen id : 24] Input [3]: [d_date_sk#10, d_year#11, d_moy#28] Condition : ((((isnotnull(d_year#11) AND isnotnull(d_moy#28)) AND (d_year#11 = 2000)) AND (d_moy#28 = 11)) AND isnotnull(d_date_sk#10)) -(69) Project [codegen id : 24] +(67) Project [codegen id : 24] Output [1]: [d_date_sk#10] Input [3]: [d_date_sk#10, d_year#11, d_moy#28] -(70) BroadcastExchange +(68) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] -(71) BroadcastHashJoin [codegen id : 25] +(69) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(72) Project [codegen id : 25] +(70) Project [codegen id : 25] Output [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8] Input [7]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8, d_date_sk#10] -(73) HashAggregate [codegen id : 25] +(71) HashAggregate [codegen id : 25] Input [5]: [ss_quantity#3, ss_list_price#4, i_brand_id#6, i_class_id#7, i_category_id#8] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#30, isEmpty#31, count#32] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#33, isEmpty#34, count#35] -(74) Exchange +(72) Exchange Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#33, isEmpty#34, count#35] Arguments: hashpartitioning(i_brand_id#6, i_class_id#7, i_category_id#8, 5), ENSURE_REQUIREMENTS, [id=#36] -(75) HashAggregate [codegen id : 26] +(73) HashAggregate [codegen id : 26] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#33, isEmpty#34, count#35] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37, count(1)#38] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sales#39, count(1)#38 AS number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41] -(76) Filter [codegen id : 26] +(74) Filter [codegen id : 26] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41 as decimal(32,6)) > cast(Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(77) Project [codegen id : 26] +(75) Project [codegen id : 26] Output [6]: [store AS channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#41] -(78) Scan parquet default.catalog_sales +(76) Scan parquet default.catalog_sales Output [4]: [cs_sold_date_sk#16, cs_item_sk#17, cs_quantity#45, cs_list_price#46] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(79) ColumnarToRow [codegen id : 51] +(77) ColumnarToRow [codegen id : 51] Input [4]: [cs_sold_date_sk#16, cs_item_sk#17, cs_quantity#45, cs_list_price#46] -(80) Filter [codegen id : 51] +(78) Filter [codegen id : 51] Input [4]: [cs_sold_date_sk#16, cs_item_sk#17, cs_quantity#45, cs_list_price#46] Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_sold_date_sk#16)) -(81) ReusedExchange [Reuses operator id: 56] +(79) ReusedExchange [Reuses operator id: 54] Output [1]: [ss_item_sk#25] -(82) BroadcastHashJoin [codegen id : 51] +(80) BroadcastHashJoin [codegen id : 51] Left keys [1]: [cs_item_sk#17] Right keys [1]: [ss_item_sk#25] Join condition: None -(83) ReusedExchange [Reuses operator id: 63] +(81) ReusedExchange [Reuses operator id: 61] Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(84) BroadcastHashJoin [codegen id : 51] +(82) BroadcastHashJoin [codegen id : 51] Left keys [1]: [cs_item_sk#17] Right keys [1]: [i_item_sk#5] Join condition: None -(85) Project [codegen id : 51] +(83) Project [codegen id : 51] Output [6]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46, i_brand_id#6, i_class_id#7, i_category_id#8] Input [8]: [cs_sold_date_sk#16, cs_item_sk#17, cs_quantity#45, cs_list_price#46, i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(86) ReusedExchange [Reuses operator id: 70] +(84) ReusedExchange [Reuses operator id: 68] Output [1]: [d_date_sk#10] -(87) BroadcastHashJoin [codegen id : 51] +(85) BroadcastHashJoin [codegen id : 51] Left keys [1]: [cs_sold_date_sk#16] Right keys [1]: [d_date_sk#10] Join condition: None -(88) Project [codegen id : 51] +(86) Project [codegen id : 51] Output [5]: [cs_quantity#45, cs_list_price#46, i_brand_id#6, i_class_id#7, i_category_id#8] Input [7]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46, i_brand_id#6, i_class_id#7, i_category_id#8, d_date_sk#10] -(89) HashAggregate [codegen id : 51] +(87) HashAggregate [codegen id : 51] Input [5]: [cs_quantity#45, cs_list_price#46, i_brand_id#6, i_class_id#7, i_category_id#8] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#47, isEmpty#48, count#49] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#50, isEmpty#51, count#52] -(90) Exchange +(88) Exchange Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#50, isEmpty#51, count#52] Arguments: hashpartitioning(i_brand_id#6, i_class_id#7, i_category_id#8, 5), ENSURE_REQUIREMENTS, [id=#53] -(91) HashAggregate [codegen id : 52] +(89) HashAggregate [codegen id : 52] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#50, isEmpty#51, count#52] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54, count(1)#55] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sales#56, count(1)#55 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58] -(92) Filter [codegen id : 52] +(90) Filter [codegen id : 52] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(93) Project [codegen id : 52] +(91) Project [codegen id : 52] Output [6]: [catalog AS channel#59, i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#58] -(94) Scan parquet default.web_sales +(92) Scan parquet default.web_sales Output [4]: [ws_sold_date_sk#20, ws_item_sk#21, ws_quantity#60, ws_list_price#61] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(95) ColumnarToRow [codegen id : 77] +(93) ColumnarToRow [codegen id : 77] Input [4]: [ws_sold_date_sk#20, ws_item_sk#21, ws_quantity#60, ws_list_price#61] -(96) Filter [codegen id : 77] +(94) Filter [codegen id : 77] Input [4]: [ws_sold_date_sk#20, ws_item_sk#21, ws_quantity#60, ws_list_price#61] Condition : (isnotnull(ws_item_sk#21) AND isnotnull(ws_sold_date_sk#20)) -(97) ReusedExchange [Reuses operator id: 56] +(95) ReusedExchange [Reuses operator id: 54] Output [1]: [ss_item_sk#25] -(98) BroadcastHashJoin [codegen id : 77] +(96) BroadcastHashJoin [codegen id : 77] Left keys [1]: [ws_item_sk#21] Right keys [1]: [ss_item_sk#25] Join condition: None -(99) ReusedExchange [Reuses operator id: 63] +(97) ReusedExchange [Reuses operator id: 61] Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(100) BroadcastHashJoin [codegen id : 77] +(98) BroadcastHashJoin [codegen id : 77] Left keys [1]: [ws_item_sk#21] Right keys [1]: [i_item_sk#5] Join condition: None -(101) Project [codegen id : 77] +(99) Project [codegen id : 77] Output [6]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61, i_brand_id#6, i_class_id#7, i_category_id#8] Input [8]: [ws_sold_date_sk#20, ws_item_sk#21, ws_quantity#60, ws_list_price#61, i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -(102) ReusedExchange [Reuses operator id: 70] +(100) ReusedExchange [Reuses operator id: 68] Output [1]: [d_date_sk#10] -(103) BroadcastHashJoin [codegen id : 77] +(101) BroadcastHashJoin [codegen id : 77] Left keys [1]: [ws_sold_date_sk#20] Right keys [1]: [d_date_sk#10] Join condition: None -(104) Project [codegen id : 77] +(102) Project [codegen id : 77] Output [5]: [ws_quantity#60, ws_list_price#61, i_brand_id#6, i_class_id#7, i_category_id#8] Input [7]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61, i_brand_id#6, i_class_id#7, i_category_id#8, d_date_sk#10] -(105) HashAggregate [codegen id : 77] +(103) HashAggregate [codegen id : 77] Input [5]: [ws_quantity#60, ws_list_price#61, i_brand_id#6, i_class_id#7, i_category_id#8] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] Aggregate Attributes [3]: [sum#62, isEmpty#63, count#64] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#65, isEmpty#66, count#67] -(106) Exchange +(104) Exchange Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#65, isEmpty#66, count#67] Arguments: hashpartitioning(i_brand_id#6, i_class_id#7, i_category_id#8, 5), ENSURE_REQUIREMENTS, [id=#68] -(107) HashAggregate [codegen id : 78] +(105) HashAggregate [codegen id : 78] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#65, isEmpty#66, count#67] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69, count(1)#70] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sales#71, count(1)#70 AS number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73] -(108) Filter [codegen id : 78] +(106) Filter [codegen id : 78] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(109) Project [codegen id : 78] +(107) Project [codegen id : 78] Output [6]: [web AS channel#74, i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#73] -(110) Union +(108) Union -(111) HashAggregate [codegen id : 79] +(109) HashAggregate [codegen id : 79] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [partial_sum(sales#39), partial_sum(number_sales#40)] Aggregate Attributes [3]: [sum#75, isEmpty#76, sum#77] Results [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#78, isEmpty#79, sum#80] -(112) Exchange +(110) Exchange Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#78, isEmpty#79, sum#80] Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, 5), ENSURE_REQUIREMENTS, [id=#81] -(113) HashAggregate [codegen id : 80] +(111) HashAggregate [codegen id : 80] Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#78, isEmpty#79, sum#80] Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(sales#39), sum(number_sales#40)] Aggregate Attributes [2]: [sum(sales#39)#82, sum(number_sales#40)#83] Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum(sales#39)#82 AS sum_sales#84, sum(number_sales#40)#83 AS number_sales#85] -(114) ReusedExchange [Reuses operator id: 74] +(112) ReusedExchange [Reuses operator id: 72] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#86, isEmpty#87, count#88] -(115) HashAggregate [codegen id : 106] +(113) HashAggregate [codegen id : 106] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#86, isEmpty#87, count#88] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#89, count(1)#90] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#89 AS sales#39, count(1)#90 AS number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#89 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#91] -(116) Filter [codegen id : 106] +(114) Filter [codegen id : 106] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#91] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#91) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#91 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(117) Project [codegen id : 106] +(115) Project [codegen id : 106] Output [6]: [store AS channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#91] -(118) ReusedExchange [Reuses operator id: 90] +(116) ReusedExchange [Reuses operator id: 88] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#92, isEmpty#93, count#94] -(119) HashAggregate [codegen id : 132] +(117) HashAggregate [codegen id : 132] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#92, isEmpty#93, count#94] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#95, count(1)#96] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#95 AS sales#56, count(1)#96 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#95 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#97] -(120) Filter [codegen id : 132] +(118) Filter [codegen id : 132] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#97] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#97) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#97 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(121) Project [codegen id : 132] +(119) Project [codegen id : 132] Output [6]: [catalog AS channel#98, i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#97] -(122) ReusedExchange [Reuses operator id: 106] +(120) ReusedExchange [Reuses operator id: 104] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#99, isEmpty#100, count#101] -(123) HashAggregate [codegen id : 158] +(121) HashAggregate [codegen id : 158] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#99, isEmpty#100, count#101] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#102, count(1)#103] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#102 AS sales#71, count(1)#103 AS number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#102 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#104] -(124) Filter [codegen id : 158] +(122) Filter [codegen id : 158] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#104] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#104) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#104 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(125) Project [codegen id : 158] +(123) Project [codegen id : 158] Output [6]: [web AS channel#105, i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#104] -(126) Union +(124) Union -(127) HashAggregate [codegen id : 159] +(125) HashAggregate [codegen id : 159] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [partial_sum(sales#39), partial_sum(number_sales#40)] Aggregate Attributes [3]: [sum#106, isEmpty#107, sum#108] Results [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#109, isEmpty#110, sum#111] -(128) Exchange +(126) Exchange Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#109, isEmpty#110, sum#111] Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, 5), ENSURE_REQUIREMENTS, [id=#112] -(129) HashAggregate [codegen id : 160] +(127) HashAggregate [codegen id : 160] Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#109, isEmpty#110, sum#111] Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(sales#39), sum(number_sales#40)] Aggregate Attributes [2]: [sum(sales#39)#113, sum(number_sales#40)#114] Results [5]: [channel#44, i_brand_id#6, i_class_id#7, sum(sales#39)#113 AS sum_sales#84, sum(number_sales#40)#114 AS number_sales#85] -(130) HashAggregate [codegen id : 160] +(128) HashAggregate [codegen id : 160] Input [5]: [channel#44, i_brand_id#6, i_class_id#7, sum_sales#84, number_sales#85] Keys [3]: [channel#44, i_brand_id#6, i_class_id#7] Functions [2]: [partial_sum(sum_sales#84), partial_sum(number_sales#85)] Aggregate Attributes [3]: [sum#115, isEmpty#116, sum#117] Results [6]: [channel#44, i_brand_id#6, i_class_id#7, sum#118, isEmpty#119, sum#120] -(131) Exchange +(129) Exchange Input [6]: [channel#44, i_brand_id#6, i_class_id#7, sum#118, isEmpty#119, sum#120] Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, 5), ENSURE_REQUIREMENTS, [id=#121] -(132) HashAggregate [codegen id : 161] +(130) HashAggregate [codegen id : 161] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, sum#118, isEmpty#119, sum#120] Keys [3]: [channel#44, i_brand_id#6, i_class_id#7] Functions [2]: [sum(sum_sales#84), sum(number_sales#85)] Aggregate Attributes [2]: [sum(sum_sales#84)#122, sum(number_sales#85)#123] Results [6]: [channel#44, i_brand_id#6, i_class_id#7, null AS i_category_id#124, sum(sum_sales#84)#122 AS sum(sum_sales)#125, sum(number_sales#85)#123 AS sum(number_sales)#126] -(133) ReusedExchange [Reuses operator id: 74] +(131) ReusedExchange [Reuses operator id: 72] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#127, isEmpty#128, count#129] -(134) HashAggregate [codegen id : 187] +(132) HashAggregate [codegen id : 187] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#127, isEmpty#128, count#129] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#130, count(1)#131] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#130 AS sales#39, count(1)#131 AS number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#130 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#132] -(135) Filter [codegen id : 187] +(133) Filter [codegen id : 187] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#132] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#132) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#132 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(136) Project [codegen id : 187] +(134) Project [codegen id : 187] Output [6]: [store AS channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#132] -(137) ReusedExchange [Reuses operator id: 90] +(135) ReusedExchange [Reuses operator id: 88] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#133, isEmpty#134, count#135] -(138) HashAggregate [codegen id : 213] +(136) HashAggregate [codegen id : 213] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#133, isEmpty#134, count#135] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#136, count(1)#137] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#136 AS sales#56, count(1)#137 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#136 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#138] -(139) Filter [codegen id : 213] +(137) Filter [codegen id : 213] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#138] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#138) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#138 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(140) Project [codegen id : 213] +(138) Project [codegen id : 213] Output [6]: [catalog AS channel#139, i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#138] -(141) ReusedExchange [Reuses operator id: 106] +(139) ReusedExchange [Reuses operator id: 104] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#140, isEmpty#141, count#142] -(142) HashAggregate [codegen id : 239] +(140) HashAggregate [codegen id : 239] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#140, isEmpty#141, count#142] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#143, count(1)#144] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#143 AS sales#71, count(1)#144 AS number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#143 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#145] -(143) Filter [codegen id : 239] +(141) Filter [codegen id : 239] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#145] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#145) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#145 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(144) Project [codegen id : 239] +(142) Project [codegen id : 239] Output [6]: [web AS channel#146, i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#145] -(145) Union +(143) Union -(146) HashAggregate [codegen id : 240] +(144) HashAggregate [codegen id : 240] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [partial_sum(sales#39), partial_sum(number_sales#40)] Aggregate Attributes [3]: [sum#147, isEmpty#148, sum#149] Results [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#150, isEmpty#151, sum#152] -(147) Exchange +(145) Exchange Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#150, isEmpty#151, sum#152] Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, 5), ENSURE_REQUIREMENTS, [id=#153] -(148) HashAggregate [codegen id : 241] +(146) HashAggregate [codegen id : 241] Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#150, isEmpty#151, sum#152] Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(sales#39), sum(number_sales#40)] Aggregate Attributes [2]: [sum(sales#39)#154, sum(number_sales#40)#155] Results [4]: [channel#44, i_brand_id#6, sum(sales#39)#154 AS sum_sales#84, sum(number_sales#40)#155 AS number_sales#85] -(149) HashAggregate [codegen id : 241] +(147) HashAggregate [codegen id : 241] Input [4]: [channel#44, i_brand_id#6, sum_sales#84, number_sales#85] Keys [2]: [channel#44, i_brand_id#6] Functions [2]: [partial_sum(sum_sales#84), partial_sum(number_sales#85)] Aggregate Attributes [3]: [sum#156, isEmpty#157, sum#158] Results [5]: [channel#44, i_brand_id#6, sum#159, isEmpty#160, sum#161] -(150) Exchange +(148) Exchange Input [5]: [channel#44, i_brand_id#6, sum#159, isEmpty#160, sum#161] Arguments: hashpartitioning(channel#44, i_brand_id#6, 5), ENSURE_REQUIREMENTS, [id=#162] -(151) HashAggregate [codegen id : 242] +(149) HashAggregate [codegen id : 242] Input [5]: [channel#44, i_brand_id#6, sum#159, isEmpty#160, sum#161] Keys [2]: [channel#44, i_brand_id#6] Functions [2]: [sum(sum_sales#84), sum(number_sales#85)] Aggregate Attributes [2]: [sum(sum_sales#84)#163, sum(number_sales#85)#164] Results [6]: [channel#44, i_brand_id#6, null AS i_class_id#165, null AS i_category_id#166, sum(sum_sales#84)#163 AS sum(sum_sales)#167, sum(number_sales#85)#164 AS sum(number_sales)#168] -(152) ReusedExchange [Reuses operator id: 74] +(150) ReusedExchange [Reuses operator id: 72] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#169, isEmpty#170, count#171] -(153) HashAggregate [codegen id : 268] +(151) HashAggregate [codegen id : 268] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#169, isEmpty#170, count#171] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#172, count(1)#173] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#172 AS sales#39, count(1)#173 AS number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#172 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#174] -(154) Filter [codegen id : 268] +(152) Filter [codegen id : 268] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#174] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#174) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#174 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(155) Project [codegen id : 268] +(153) Project [codegen id : 268] Output [6]: [store AS channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#174] -(156) ReusedExchange [Reuses operator id: 90] +(154) ReusedExchange [Reuses operator id: 88] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#175, isEmpty#176, count#177] -(157) HashAggregate [codegen id : 294] +(155) HashAggregate [codegen id : 294] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#175, isEmpty#176, count#177] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#178, count(1)#179] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#178 AS sales#56, count(1)#179 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#178 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#180] -(158) Filter [codegen id : 294] +(156) Filter [codegen id : 294] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#180] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#180) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#180 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(159) Project [codegen id : 294] +(157) Project [codegen id : 294] Output [6]: [catalog AS channel#181, i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#180] -(160) ReusedExchange [Reuses operator id: 106] +(158) ReusedExchange [Reuses operator id: 104] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#182, isEmpty#183, count#184] -(161) HashAggregate [codegen id : 320] +(159) HashAggregate [codegen id : 320] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#182, isEmpty#183, count#184] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#185, count(1)#186] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#185 AS sales#71, count(1)#186 AS number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#185 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#187] -(162) Filter [codegen id : 320] +(160) Filter [codegen id : 320] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#187] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#187) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#187 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(163) Project [codegen id : 320] +(161) Project [codegen id : 320] Output [6]: [web AS channel#188, i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#187] -(164) Union +(162) Union -(165) HashAggregate [codegen id : 321] +(163) HashAggregate [codegen id : 321] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [partial_sum(sales#39), partial_sum(number_sales#40)] Aggregate Attributes [3]: [sum#189, isEmpty#190, sum#191] Results [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#192, isEmpty#193, sum#194] -(166) Exchange +(164) Exchange Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#192, isEmpty#193, sum#194] Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, 5), ENSURE_REQUIREMENTS, [id=#195] -(167) HashAggregate [codegen id : 322] +(165) HashAggregate [codegen id : 322] Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#192, isEmpty#193, sum#194] Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(sales#39), sum(number_sales#40)] Aggregate Attributes [2]: [sum(sales#39)#196, sum(number_sales#40)#197] Results [3]: [channel#44, sum(sales#39)#196 AS sum_sales#84, sum(number_sales#40)#197 AS number_sales#85] -(168) HashAggregate [codegen id : 322] +(166) HashAggregate [codegen id : 322] Input [3]: [channel#44, sum_sales#84, number_sales#85] Keys [1]: [channel#44] Functions [2]: [partial_sum(sum_sales#84), partial_sum(number_sales#85)] Aggregate Attributes [3]: [sum#198, isEmpty#199, sum#200] Results [4]: [channel#44, sum#201, isEmpty#202, sum#203] -(169) Exchange +(167) Exchange Input [4]: [channel#44, sum#201, isEmpty#202, sum#203] Arguments: hashpartitioning(channel#44, 5), ENSURE_REQUIREMENTS, [id=#204] -(170) HashAggregate [codegen id : 323] +(168) HashAggregate [codegen id : 323] Input [4]: [channel#44, sum#201, isEmpty#202, sum#203] Keys [1]: [channel#44] Functions [2]: [sum(sum_sales#84), sum(number_sales#85)] Aggregate Attributes [2]: [sum(sum_sales#84)#205, sum(number_sales#85)#206] Results [6]: [channel#44, null AS i_brand_id#207, null AS i_class_id#208, null AS i_category_id#209, sum(sum_sales#84)#205 AS sum(sum_sales)#210, sum(number_sales#85)#206 AS sum(number_sales)#211] -(171) ReusedExchange [Reuses operator id: 74] +(169) ReusedExchange [Reuses operator id: 72] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#212, isEmpty#213, count#214] -(172) HashAggregate [codegen id : 349] +(170) HashAggregate [codegen id : 349] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#212, isEmpty#213, count#214] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#215, count(1)#216] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#215 AS sales#39, count(1)#216 AS number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#215 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#217] -(173) Filter [codegen id : 349] +(171) Filter [codegen id : 349] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#217] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#217) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#217 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(174) Project [codegen id : 349] +(172) Project [codegen id : 349] Output [6]: [store AS channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#217] -(175) ReusedExchange [Reuses operator id: 90] +(173) ReusedExchange [Reuses operator id: 88] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#218, isEmpty#219, count#220] -(176) HashAggregate [codegen id : 375] +(174) HashAggregate [codegen id : 375] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#218, isEmpty#219, count#220] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#221, count(1)#222] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#221 AS sales#56, count(1)#222 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#221 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#223] -(177) Filter [codegen id : 375] +(175) Filter [codegen id : 375] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#223] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#223) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#223 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(178) Project [codegen id : 375] +(176) Project [codegen id : 375] Output [6]: [catalog AS channel#224, i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#46 as decimal(12,2)))), DecimalType(18,2), true))#223] -(179) ReusedExchange [Reuses operator id: 106] +(177) ReusedExchange [Reuses operator id: 104] Output [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#225, isEmpty#226, count#227] -(180) HashAggregate [codegen id : 401] +(178) HashAggregate [codegen id : 401] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum#225, isEmpty#226, count#227] Keys [3]: [i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#228, count(1)#229] Results [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#228 AS sales#71, count(1)#229 AS number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#228 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#230] -(181) Filter [codegen id : 401] +(179) Filter [codegen id : 401] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#230] Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#230) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#230 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#42, [id=#43] as decimal(32,6)))) -(182) Project [codegen id : 401] +(180) Project [codegen id : 401] Output [6]: [web AS channel#231, i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72] Input [6]: [i_brand_id#6, i_class_id#7, i_category_id#8, sales#71, number_sales#72, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#230] -(183) Union +(181) Union -(184) HashAggregate [codegen id : 402] +(182) HashAggregate [codegen id : 402] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sales#39, number_sales#40] Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [partial_sum(sales#39), partial_sum(number_sales#40)] Aggregate Attributes [3]: [sum#232, isEmpty#233, sum#234] Results [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#235, isEmpty#236, sum#237] -(185) Exchange +(183) Exchange Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#235, isEmpty#236, sum#237] Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, 5), ENSURE_REQUIREMENTS, [id=#238] -(186) HashAggregate [codegen id : 403] +(184) HashAggregate [codegen id : 403] Input [7]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum#235, isEmpty#236, sum#237] Keys [4]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8] Functions [2]: [sum(sales#39), sum(number_sales#40)] Aggregate Attributes [2]: [sum(sales#39)#239, sum(number_sales#40)#240] Results [2]: [sum(sales#39)#239 AS sum_sales#84, sum(number_sales#40)#240 AS number_sales#85] -(187) HashAggregate [codegen id : 403] +(185) HashAggregate [codegen id : 403] Input [2]: [sum_sales#84, number_sales#85] Keys: [] Functions [2]: [partial_sum(sum_sales#84), partial_sum(number_sales#85)] Aggregate Attributes [3]: [sum#241, isEmpty#242, sum#243] Results [3]: [sum#244, isEmpty#245, sum#246] -(188) Exchange +(186) Exchange Input [3]: [sum#244, isEmpty#245, sum#246] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#247] -(189) HashAggregate [codegen id : 404] +(187) HashAggregate [codegen id : 404] Input [3]: [sum#244, isEmpty#245, sum#246] Keys: [] Functions [2]: [sum(sum_sales#84), sum(number_sales#85)] Aggregate Attributes [2]: [sum(sum_sales#84)#248, sum(number_sales#85)#249] Results [6]: [null AS channel#250, null AS i_brand_id#251, null AS i_class_id#252, null AS i_category_id#253, sum(sum_sales#84)#248 AS sum(sum_sales)#254, sum(number_sales#85)#249 AS sum(number_sales)#255] -(190) Union +(188) Union -(191) HashAggregate [codegen id : 405] +(189) HashAggregate [codegen id : 405] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Keys [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Functions: [] Aggregate Attributes: [] Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] -(192) Exchange +(190) Exchange Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Arguments: hashpartitioning(channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85, 5), ENSURE_REQUIREMENTS, [id=#256] -(193) HashAggregate [codegen id : 406] +(191) HashAggregate [codegen id : 406] Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Keys [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Functions: [] Aggregate Attributes: [] Results [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] -(194) TakeOrderedAndProject +(192) TakeOrderedAndProject Input [6]: [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] Arguments: 100, [channel#44 ASC NULLS FIRST, i_brand_id#6 ASC NULLS FIRST, i_class_id#7 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST], [channel#44, i_brand_id#6, i_class_id#7, i_category_id#8, sum_sales#84, number_sales#85] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#42, [id=#43] -* HashAggregate (224) -+- Exchange (223) - +- * HashAggregate (222) - +- Union (221) - :- * Project (204) - : +- * BroadcastHashJoin Inner BuildRight (203) - : :- * Filter (197) - : : +- * ColumnarToRow (196) - : : +- Scan parquet default.store_sales (195) - : +- BroadcastExchange (202) - : +- * Project (201) - : +- * Filter (200) - : +- * ColumnarToRow (199) - : +- Scan parquet default.date_dim (198) - :- * Project (214) - : +- * BroadcastHashJoin Inner BuildRight (213) - : :- * Filter (207) - : : +- * ColumnarToRow (206) - : : +- Scan parquet default.catalog_sales (205) - : +- BroadcastExchange (212) - : +- * Project (211) - : +- * Filter (210) - : +- * ColumnarToRow (209) - : +- Scan parquet default.date_dim (208) - +- * Project (220) - +- * BroadcastHashJoin Inner BuildRight (219) - :- * Filter (217) - : +- * ColumnarToRow (216) - : +- Scan parquet default.web_sales (215) - +- ReusedExchange (218) - - -(195) Scan parquet default.store_sales +Subquery:1 Hosting operator id = 74 Hosting Expression = Subquery scalar-subquery#42, [id=#43] +* HashAggregate (222) ++- Exchange (221) + +- * HashAggregate (220) + +- Union (219) + :- * Project (202) + : +- * BroadcastHashJoin Inner BuildRight (201) + : :- * Filter (195) + : : +- * ColumnarToRow (194) + : : +- Scan parquet default.store_sales (193) + : +- BroadcastExchange (200) + : +- * Project (199) + : +- * Filter (198) + : +- * ColumnarToRow (197) + : +- Scan parquet default.date_dim (196) + :- * Project (212) + : +- * BroadcastHashJoin Inner BuildRight (211) + : :- * Filter (205) + : : +- * ColumnarToRow (204) + : : +- Scan parquet default.catalog_sales (203) + : +- BroadcastExchange (210) + : +- * Project (209) + : +- * Filter (208) + : +- * ColumnarToRow (207) + : +- Scan parquet default.date_dim (206) + +- * Project (218) + +- * BroadcastHashJoin Inner BuildRight (217) + :- * Filter (215) + : +- * ColumnarToRow (214) + : +- Scan parquet default.web_sales (213) + +- ReusedExchange (216) + + +(193) Scan parquet default.store_sales Output [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(196) ColumnarToRow [codegen id : 2] +(194) ColumnarToRow [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] -(197) Filter [codegen id : 2] +(195) Filter [codegen id : 2] Input [3]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4] Condition : isnotnull(ss_sold_date_sk#1) -(198) Scan parquet default.date_dim +(196) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_year#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(199) ColumnarToRow [codegen id : 1] +(197) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] -(200) Filter [codegen id : 1] +(198) Filter [codegen id : 1] Input [2]: [d_date_sk#10, d_year#11] Condition : (((isnotnull(d_year#11) AND (d_year#11 >= 1999)) AND (d_year#11 <= 2001)) AND isnotnull(d_date_sk#10)) -(201) Project [codegen id : 1] +(199) Project [codegen id : 1] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_year#11] -(202) BroadcastExchange +(200) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#257] -(203) BroadcastHashJoin [codegen id : 2] +(201) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#1] Right keys [1]: [d_date_sk#10] Join condition: None -(204) Project [codegen id : 2] +(202) Project [codegen id : 2] Output [2]: [ss_quantity#3 AS quantity#258, ss_list_price#4 AS list_price#259] Input [4]: [ss_sold_date_sk#1, ss_quantity#3, ss_list_price#4, d_date_sk#10] -(205) Scan parquet default.catalog_sales +(203) Scan parquet default.catalog_sales Output [3]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(206) ColumnarToRow [codegen id : 4] +(204) ColumnarToRow [codegen id : 4] Input [3]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46] -(207) Filter [codegen id : 4] +(205) Filter [codegen id : 4] Input [3]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46] Condition : isnotnull(cs_sold_date_sk#16) -(208) Scan parquet default.date_dim +(206) Scan parquet default.date_dim Output [2]: [d_date_sk#10, d_year#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(209) ColumnarToRow [codegen id : 3] +(207) ColumnarToRow [codegen id : 3] Input [2]: [d_date_sk#10, d_year#11] -(210) Filter [codegen id : 3] +(208) Filter [codegen id : 3] Input [2]: [d_date_sk#10, d_year#11] Condition : (((isnotnull(d_year#11) AND (d_year#11 >= 1998)) AND (d_year#11 <= 2000)) AND isnotnull(d_date_sk#10)) -(211) Project [codegen id : 3] +(209) Project [codegen id : 3] Output [1]: [d_date_sk#10] Input [2]: [d_date_sk#10, d_year#11] -(212) BroadcastExchange +(210) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#260] -(213) BroadcastHashJoin [codegen id : 4] +(211) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#16] Right keys [1]: [d_date_sk#10] Join condition: None -(214) Project [codegen id : 4] +(212) Project [codegen id : 4] Output [2]: [cs_quantity#45 AS quantity#261, cs_list_price#46 AS list_price#262] Input [4]: [cs_sold_date_sk#16, cs_quantity#45, cs_list_price#46, d_date_sk#10] -(215) Scan parquet default.web_sales +(213) Scan parquet default.web_sales Output [3]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(216) ColumnarToRow [codegen id : 6] +(214) ColumnarToRow [codegen id : 6] Input [3]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61] -(217) Filter [codegen id : 6] +(215) Filter [codegen id : 6] Input [3]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61] Condition : isnotnull(ws_sold_date_sk#20) -(218) ReusedExchange [Reuses operator id: 212] +(216) ReusedExchange [Reuses operator id: 210] Output [1]: [d_date_sk#10] -(219) BroadcastHashJoin [codegen id : 6] +(217) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#20] Right keys [1]: [d_date_sk#10] Join condition: None -(220) Project [codegen id : 6] +(218) Project [codegen id : 6] Output [2]: [ws_quantity#60 AS quantity#263, ws_list_price#61 AS list_price#264] Input [4]: [ws_sold_date_sk#20, ws_quantity#60, ws_list_price#61, d_date_sk#10] -(221) Union +(219) Union -(222) HashAggregate [codegen id : 7] +(220) HashAggregate [codegen id : 7] Input [2]: [quantity#258, list_price#259] Keys: [] Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#258 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#259 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [2]: [sum#265, count#266] Results [2]: [sum#267, count#268] -(223) Exchange +(221) Exchange Input [2]: [sum#267, count#268] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#269] -(224) HashAggregate [codegen id : 8] +(222) HashAggregate [codegen id : 8] Input [2]: [sum#267, count#268] Keys: [] Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#258 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#259 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#258 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#259 as decimal(12,2)))), DecimalType(18,2), true))#270] Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#258 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#259 as decimal(12,2)))), DecimalType(18,2), true))#270 AS average_sales#271] -Subquery:2 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:2 Hosting operator id = 90 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:3 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:3 Hosting operator id = 106 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:4 Hosting operator id = 116 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:4 Hosting operator id = 114 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:5 Hosting operator id = 120 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:5 Hosting operator id = 118 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:6 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:6 Hosting operator id = 122 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:7 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:7 Hosting operator id = 133 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:8 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:8 Hosting operator id = 137 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:9 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:9 Hosting operator id = 141 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:10 Hosting operator id = 154 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:10 Hosting operator id = 152 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:11 Hosting operator id = 158 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:11 Hosting operator id = 156 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:12 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:12 Hosting operator id = 160 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:13 Hosting operator id = 173 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:13 Hosting operator id = 171 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:14 Hosting operator id = 177 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:14 Hosting operator id = 175 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] -Subquery:15 Hosting operator id = 181 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:15 Hosting operator id = 179 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt index 18484308feca..3e13d3bdffa7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt @@ -93,72 +93,70 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastExchange #5 WholeStageCodegen (10) HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (9) - HashAggregate [brand_id,class_id,category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] + InputAdapter + Exchange [brand_id,class_id,category_id] #6 + WholeStageCodegen (9) + HashAggregate [brand_id,class_id,category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk,ss_sold_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #7 - WholeStageCodegen (1) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] + BroadcastExchange #7 + WholeStageCodegen (1) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] + BroadcastExchange #8 + WholeStageCodegen (2) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (5) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk,cs_sold_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #10 - WholeStageCodegen (3) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (8) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk,ws_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] + BroadcastExchange #10 + WholeStageCodegen (3) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [d_date_sk] #8 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (8) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk,ws_sold_date_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] InputAdapter - ReusedExchange [d_date_sk] #8 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + InputAdapter + ReusedExchange [d_date_sk] #8 InputAdapter BroadcastExchange #12 WholeStageCodegen (23) From b41519448e1db46d6c11aeb56396c4af4b0e1c51 Mon Sep 17 00:00:00 2001 From: "tanel.kiis@gmail.com" Date: Tue, 12 Jan 2021 00:27:04 +0200 Subject: [PATCH 19/22] upperRefsOnlyDeterministicNonAgg --- .../spark/sql/catalyst/optimizer/Optimizer.scala | 13 +++++++++---- .../optimizer/RemoveRedundantAggregatesSuite.scala | 13 +++++++++++-- 2 files changed, 20 insertions(+), 6 deletions(-) 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 60366df3335d..dd78633a30e1 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 @@ -516,10 +516,15 @@ object RemoveRedundantAggregates extends Rule[LogicalPlan] with AliasHelper { val upperHasNoAggregateExpressions = upper.aggregateExpressions .forall(_.find(isAggregate).isEmpty) - lazy val upperReferencesOnlyGrouping = upper.references.subsetOf(AttributeSet( - lower.aggregateExpressions.filter(!isAggregate(_)).map(_.toAttribute))) - - upperHasNoAggregateExpressions && upperReferencesOnlyGrouping + lazy val upperRefsOnlyDeterministicNonAgg = upper.references.subsetOf(AttributeSet( + lower + .aggregateExpressions + .filter(!isAggregate(_)) + .filter(_.deterministic) + .map(_.toAttribute) + )) + + upperHasNoAggregateExpressions && upperRefsOnlyDeterministicNonAgg } private def isAggregate(expr: Expression): Boolean = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala index d19cc88be015..d376c31ef965 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala @@ -116,7 +116,7 @@ class RemoveRedundantAggregatesSuite extends PlanTest { test("Remove redundant aggregate with non-deterministic lower") { val relation = LocalRelation('a.int, 'b.int) val query = relation - .groupBy('a)('a, rand(0) as 'c) + .groupBy('a, 'c)('a, rand(0) as 'c) .groupBy('a, 'c)('a, 'c) .analyze val expected = relation @@ -139,7 +139,7 @@ class RemoveRedundantAggregatesSuite extends PlanTest { } } - test("Keep non-redundant aggregate - upper references non-grouping") { + test("Keep non-redundant aggregate - upper references agg expression") { val relation = LocalRelation('a.int, 'b.int) for (agg <- aggregates('b)) { val query = relation @@ -151,4 +151,13 @@ class RemoveRedundantAggregatesSuite extends PlanTest { } } + test("Keep non-redundant aggregate - upper references non-deterministic non-grouping") { + val relation = LocalRelation('a.int, 'b.int) + val query = relation + .groupBy('a)('a, ('a + rand(0)) as 'c) + .groupBy('a, 'c)('a, 'c) + .analyze + val optimized = Optimize.execute(query) + comparePlans(optimized, query) + } } From 797d48fcd76269c06f0dc832d2019bf9619bcbf2 Mon Sep 17 00:00:00 2001 From: "tanel.kiis@gmail.com" Date: Tue, 12 Jan 2021 00:29:10 +0200 Subject: [PATCH 20/22] upperRefsOnlyDeterministicNonAgg --- .../org/apache/spark/sql/catalyst/optimizer/Optimizer.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 dd78633a30e1..e1aa37211e80 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 @@ -519,8 +519,8 @@ object RemoveRedundantAggregates extends Rule[LogicalPlan] with AliasHelper { lazy val upperRefsOnlyDeterministicNonAgg = upper.references.subsetOf(AttributeSet( lower .aggregateExpressions - .filter(!isAggregate(_)) .filter(_.deterministic) + .filter(!isAggregate(_)) .map(_.toAttribute) )) From e202987ed5d6b78aa0a91264232a9a42ca0c3dbe Mon Sep 17 00:00:00 2001 From: "tanel.kiis@gmail.com" Date: Sat, 16 Jan 2021 20:02:53 +0200 Subject: [PATCH 21/22] Cleanup --- .../org/apache/spark/sql/catalyst/optimizer/Optimizer.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) 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 a9996781c9bd..c5bfaf222e79 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 @@ -514,8 +514,7 @@ object RemoveRedundantAggregates extends Rule[LogicalPlan] with AliasHelper { } private def lowerIsRedundant(upper: Aggregate, lower: Aggregate): Boolean = { - val upperHasNoAggregateExpressions = upper.aggregateExpressions - .forall(_.find(isAggregate).isEmpty) + val upperHasNoAggregateExpressions = !upper.aggregateExpressions.exists(isAggregate) lazy val upperRefsOnlyDeterministicNonAgg = upper.references.subsetOf(AttributeSet( lower From 07e758d25e0840bf40129a36ad5bdb90a005058a Mon Sep 17 00:00:00 2001 From: "tanel.kiis@gmail.com" Date: Fri, 19 Mar 2021 06:24:16 +0200 Subject: [PATCH 22/22] Trigger build