diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 26114090f..f9caee9d9 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -149,6 +149,17 @@ object CometConf { .booleanConf .createWithDefault(false) + val COMET_SHUFFLE_ENFORCE_MODE_ENABLED: ConfigEntry[Boolean] = + conf("spark.comet.shuffle.enforceMode.enabled") + .doc( + "Comet shuffle doesn't support Spark AQE coalesce partitions. If AQE coalesce " + + "partitions is enabled, Comet shuffle won't be triggered even enabled. This config " + + "is used to enforce Comet to trigger shuffle even if AQE coalesce partitions is " + + "enabled. This is for testing purpose only.") + .internal() + .booleanConf + .createWithDefault(false) + val COMET_EXEC_BROADCAST_FORCE_ENABLED: ConfigEntry[Boolean] = conf(s"$COMET_EXEC_CONFIG_PREFIX.broadcast.enabled") .doc( diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala index 1e78013c7..0b93f3206 100644 --- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala +++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala @@ -916,7 +916,10 @@ object CometSparkSessionExtensions extends Logging { private[comet] def isCometShuffleEnabled(conf: SQLConf): Boolean = COMET_EXEC_SHUFFLE_ENABLED.get(conf) && (conf.contains("spark.shuffle.manager") && conf.getConfString("spark.shuffle.manager") == - "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") + "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") && + // TODO: AQE coalesce partitions feature causes Comet shuffle memory leak. + // We should disable Comet shuffle when AQE coalesce partitions is enabled. + (!conf.coalesceShufflePartitionsEnabled || COMET_SHUFFLE_ENFORCE_MODE_ENABLED.get()) private[comet] def isCometScanEnabled(conf: SQLConf): Boolean = { COMET_SCAN_ENABLED.get(conf) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt index 1762c8c6c..d8ed9c06e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt @@ -13,20 +13,20 @@ : : :- * SortMergeJoin LeftAnti (19) : : : :- * Project (13) : : : : +- * SortMergeJoin LeftSemi (12) - : : : : :- * ColumnarToRow (6) - : : : : : +- CometSort (5) - : : : : : +- CometExchange (4) + : : : : :- * Sort (6) + : : : : : +- Exchange (5) + : : : : : +- * ColumnarToRow (4) : : : : : +- CometProject (3) : : : : : +- CometFilter (2) : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : +- * ColumnarToRow (11) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) + : : : : +- * Sort (11) + : : : : +- Exchange (10) + : : : : +- * ColumnarToRow (9) : : : : +- CometProject (8) : : : : +- CometScan parquet spark_catalog.default.catalog_sales (7) - : : : +- * ColumnarToRow (18) - : : : +- CometSort (17) - : : : +- CometExchange (16) + : : : +- * Sort (18) + : : : +- Exchange (17) + : : : +- * ColumnarToRow (16) : : : +- CometProject (15) : : : +- CometScan parquet spark_catalog.default.catalog_returns (14) : : +- BroadcastExchange (24) @@ -61,16 +61,16 @@ Condition : ((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -(4) CometExchange +(4) ColumnarToRow [codegen id : 1] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: hashpartitioning(cs_order_number#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(5) CometSort +(5) Exchange Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_order_number#5 ASC NULLS FIRST] +Arguments: hashpartitioning(cs_order_number#5, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(6) ColumnarToRow [codegen id : 1] +(6) Sort [codegen id : 2] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] +Arguments: [cs_order_number#5 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.catalog_sales Output [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11] @@ -82,24 +82,24 @@ ReadSchema: struct Input [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11] Arguments: [cs_warehouse_sk#9, cs_order_number#10], [cs_warehouse_sk#9, cs_order_number#10] -(9) CometExchange +(9) ColumnarToRow [codegen id : 3] Input [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: hashpartitioning(cs_order_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(10) CometSort +(10) Exchange Input [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: [cs_warehouse_sk#9, cs_order_number#10], [cs_order_number#10 ASC NULLS FIRST] +Arguments: hashpartitioning(cs_order_number#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(11) ColumnarToRow [codegen id : 2] +(11) Sort [codegen id : 4] Input [2]: [cs_warehouse_sk#9, cs_order_number#10] +Arguments: [cs_order_number#10 ASC NULLS FIRST], false, 0 -(12) SortMergeJoin [codegen id : 3] +(12) SortMergeJoin [codegen id : 5] Left keys [1]: [cs_order_number#5] Right keys [1]: [cs_order_number#10] Join type: LeftSemi Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#9) -(13) Project [codegen id : 3] +(13) Project [codegen id : 5] Output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] @@ -113,18 +113,18 @@ ReadSchema: struct Input [2]: [cr_order_number#12, cr_returned_date_sk#13] Arguments: [cr_order_number#12], [cr_order_number#12] -(16) CometExchange +(16) ColumnarToRow [codegen id : 6] Input [1]: [cr_order_number#12] -Arguments: hashpartitioning(cr_order_number#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(17) CometSort +(17) Exchange Input [1]: [cr_order_number#12] -Arguments: [cr_order_number#12], [cr_order_number#12 ASC NULLS FIRST] +Arguments: hashpartitioning(cr_order_number#12, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(18) ColumnarToRow [codegen id : 4] +(18) Sort [codegen id : 7] Input [1]: [cr_order_number#12] +Arguments: [cr_order_number#12 ASC NULLS FIRST], false, 0 -(19) SortMergeJoin [codegen id : 8] +(19) SortMergeJoin [codegen id : 11] Left keys [1]: [cs_order_number#5] Right keys [1]: [cr_order_number#12] Join type: LeftAnti @@ -145,20 +145,20 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2002-02-01)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(23) ColumnarToRow [codegen id : 5] +(23) ColumnarToRow [codegen id : 8] Input [1]: [d_date_sk#14] (24) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(25) BroadcastHashJoin [codegen id : 8] +(25) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_date_sk#1] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(26) Project [codegen id : 8] +(26) Project [codegen id : 11] Output [5]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#14] @@ -177,20 +177,20 @@ Condition : ((isnotnull(ca_state#17) AND (ca_state#17 = GA)) AND isnotnull(ca_ad Input [2]: [ca_address_sk#16, ca_state#17] Arguments: [ca_address_sk#16], [ca_address_sk#16] -(30) ColumnarToRow [codegen id : 6] +(30) ColumnarToRow [codegen id : 9] Input [1]: [ca_address_sk#16] (31) BroadcastExchange Input [1]: [ca_address_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(32) BroadcastHashJoin [codegen id : 8] +(32) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_addr_sk#2] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(33) Project [codegen id : 8] +(33) Project [codegen id : 11] Output [4]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [6]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#16] @@ -209,38 +209,38 @@ Condition : ((isnotnull(cc_county#19) AND (cc_county#19 = Williamson County)) AN Input [2]: [cc_call_center_sk#18, cc_county#19] Arguments: [cc_call_center_sk#18], [cc_call_center_sk#18] -(37) ColumnarToRow [codegen id : 7] +(37) ColumnarToRow [codegen id : 10] Input [1]: [cc_call_center_sk#18] (38) BroadcastExchange Input [1]: [cc_call_center_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(39) BroadcastHashJoin [codegen id : 8] +(39) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_call_center_sk#3] Right keys [1]: [cc_call_center_sk#18] Join type: Inner Join condition: None -(40) Project [codegen id : 8] +(40) Project [codegen id : 11] Output [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [5]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#18] -(41) HashAggregate [codegen id : 8] +(41) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Keys [1]: [cs_order_number#5] Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#20, sum(UnscaledValue(cs_net_profit#7))#21] Results [3]: [cs_order_number#5, sum#22, sum#23] -(42) HashAggregate [codegen id : 8] +(42) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, sum#22, sum#23] Keys [1]: [cs_order_number#5] Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#20, sum(UnscaledValue(cs_net_profit#7))#21] Results [3]: [cs_order_number#5, sum#22, sum#23] -(43) HashAggregate [codegen id : 8] +(43) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, sum#22, sum#23] Keys: [] Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] @@ -251,7 +251,7 @@ Results [3]: [sum#22, sum#23, count#25] Input [3]: [sum#22, sum#23, count#25] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(45) HashAggregate [codegen id : 9] +(45) HashAggregate [codegen id : 12] Input [3]: [sum#22, sum#23, count#25] Keys: [] Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt index 213726372..a55c182be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (9) +WholeStageCodegen (12) HashAggregate [sum,sum,count] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] InputAdapter Exchange #1 - WholeStageCodegen (8) + WholeStageCodegen (11) HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] HashAggregate [cs_order_number,cs_ext_ship_cost,cs_net_profit] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] @@ -14,37 +14,43 @@ WholeStageCodegen (9) BroadcastHashJoin [cs_ship_date_sk,d_date_sk] SortMergeJoin [cs_order_number,cr_order_number] InputAdapter - WholeStageCodegen (3) + WholeStageCodegen (5) Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] SortMergeJoin [cs_order_number,cs_order_number,cs_warehouse_sk,cs_warehouse_sk] InputAdapter - WholeStageCodegen (1) - ColumnarToRow + WholeStageCodegen (2) + Sort [cs_order_number] InputAdapter - CometSort [cs_order_number] - CometExchange [cs_order_number] #2 - CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] + Exchange [cs_order_number] #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] InputAdapter - WholeStageCodegen (2) - ColumnarToRow + WholeStageCodegen (4) + Sort [cs_order_number] InputAdapter - CometSort [cs_order_number] - CometExchange [cs_order_number] #3 - CometProject [cs_warehouse_sk,cs_order_number] - CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] + Exchange [cs_order_number] #3 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [cs_warehouse_sk,cs_order_number] + CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] InputAdapter - WholeStageCodegen (4) - ColumnarToRow + WholeStageCodegen (7) + Sort [cr_order_number] InputAdapter - CometSort [cr_order_number] - CometExchange [cr_order_number] #4 - CometProject [cr_order_number] - CometScan parquet spark_catalog.default.catalog_returns [cr_order_number,cr_returned_date_sk] + Exchange [cr_order_number] #4 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometProject [cr_order_number] + CometScan parquet spark_catalog.default.catalog_returns [cr_order_number,cr_returned_date_sk] InputAdapter BroadcastExchange #5 - WholeStageCodegen (5) + WholeStageCodegen (8) ColumnarToRow InputAdapter CometProject [d_date_sk] @@ -52,7 +58,7 @@ WholeStageCodegen (9) CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 - WholeStageCodegen (6) + WholeStageCodegen (9) ColumnarToRow InputAdapter CometProject [ca_address_sk] @@ -60,7 +66,7 @@ WholeStageCodegen (9) CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 - WholeStageCodegen (7) + WholeStageCodegen (10) ColumnarToRow InputAdapter CometProject [cc_call_center_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt index 86227b832..8ffb43cf5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt @@ -52,9 +52,9 @@ TakeOrderedAndProject (87) : : : +- CometScan parquet spark_catalog.default.customer (30) : : +- BroadcastExchange (54) : : +- * SortMergeJoin LeftSemi (53) - : : :- * ColumnarToRow (47) - : : : +- CometSort (46) - : : : +- CometExchange (45) + : : :- * Sort (47) + : : : +- Exchange (46) + : : : +- * ColumnarToRow (45) : : : +- CometFilter (44) : : : +- CometScan parquet spark_catalog.default.customer (43) : : +- * Sort (52) @@ -283,7 +283,7 @@ Input [2]: [c_customer_sk#24, ssales#30] Input [1]: [c_customer_sk#24] Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 -(42) SortMergeJoin [codegen id : 16] +(42) SortMergeJoin [codegen id : 17] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#24] Join type: LeftSemi @@ -300,40 +300,40 @@ ReadSchema: struct Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] Condition : isnotnull(c_customer_sk#33) -(45) CometExchange +(45) ColumnarToRow [codegen id : 10] Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] -Arguments: hashpartitioning(c_customer_sk#33, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(46) CometSort +(46) Exchange Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] -Arguments: [c_customer_sk#33, c_first_name#34, c_last_name#35], [c_customer_sk#33 ASC NULLS FIRST] +Arguments: hashpartitioning(c_customer_sk#33, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(47) ColumnarToRow [codegen id : 10] +(47) Sort [codegen id : 11] Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] +Arguments: [c_customer_sk#33 ASC NULLS FIRST], false, 0 (48) ReusedExchange [Reuses operator id: 37] Output [3]: [c_customer_sk#24, sum#27, isEmpty#28] -(49) HashAggregate [codegen id : 13] +(49) HashAggregate [codegen id : 14] Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] Keys [1]: [c_customer_sk#24] Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] -(50) Filter [codegen id : 13] +(50) Filter [codegen id : 14] Input [2]: [c_customer_sk#24, ssales#30] Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32]))) -(51) Project [codegen id : 13] +(51) Project [codegen id : 14] Output [1]: [c_customer_sk#24] Input [2]: [c_customer_sk#24, ssales#30] -(52) Sort [codegen id : 13] +(52) Sort [codegen id : 14] Input [1]: [c_customer_sk#24] Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 -(53) SortMergeJoin [codegen id : 14] +(53) SortMergeJoin [codegen id : 15] Left keys [1]: [c_customer_sk#33] Right keys [1]: [c_customer_sk#24] Join type: LeftSemi @@ -343,30 +343,30 @@ Join condition: None Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(55) BroadcastHashJoin [codegen id : 16] +(55) BroadcastHashJoin [codegen id : 17] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#33] Join type: Inner Join condition: None -(56) Project [codegen id : 16] +(56) Project [codegen id : 17] Output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#34, c_last_name#35] Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#33, c_first_name#34, c_last_name#35] (57) ReusedExchange [Reuses operator id: 92] Output [1]: [d_date_sk#36] -(58) BroadcastHashJoin [codegen id : 16] +(58) BroadcastHashJoin [codegen id : 17] Left keys [1]: [cs_sold_date_sk#5] Right keys [1]: [d_date_sk#36] Join type: Inner Join condition: None -(59) Project [codegen id : 16] +(59) Project [codegen id : 17] Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#34, c_last_name#35] Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#34, c_last_name#35, d_date_sk#36] -(60) HashAggregate [codegen id : 16] +(60) HashAggregate [codegen id : 17] Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#34, c_last_name#35] Keys [2]: [c_last_name#35, c_first_name#34] Functions [1]: [partial_sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] @@ -377,7 +377,7 @@ Results [4]: [c_last_name#35, c_first_name#34, sum#39, isEmpty#40] Input [4]: [c_last_name#35, c_first_name#34, sum#39, isEmpty#40] Arguments: hashpartitioning(c_last_name#35, c_first_name#34, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(62) HashAggregate [codegen id : 17] +(62) HashAggregate [codegen id : 18] Input [4]: [c_last_name#35, c_first_name#34, sum#39, isEmpty#40] Keys [2]: [c_last_name#35, c_first_name#34] Functions [1]: [sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] @@ -396,19 +396,19 @@ ReadSchema: struct (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32]))) -(74) Project [codegen id : 26] +(74) Project [codegen id : 27] Output [1]: [c_customer_sk#24] Input [2]: [c_customer_sk#24, ssales#30] -(75) Sort [codegen id : 26] +(75) Sort [codegen id : 27] Input [1]: [c_customer_sk#24] Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 -(76) SortMergeJoin [codegen id : 33] +(76) SortMergeJoin [codegen id : 35] Left keys [1]: [ws_bill_customer_sk#44] Right keys [1]: [c_customer_sk#24] Join type: LeftSemi @@ -451,30 +451,30 @@ Join condition: None (77) ReusedExchange [Reuses operator id: 54] Output [3]: [c_customer_sk#49, c_first_name#50, c_last_name#51] -(78) BroadcastHashJoin [codegen id : 33] +(78) BroadcastHashJoin [codegen id : 35] Left keys [1]: [ws_bill_customer_sk#44] Right keys [1]: [c_customer_sk#49] Join type: Inner Join condition: None -(79) Project [codegen id : 33] +(79) Project [codegen id : 35] Output [5]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, c_first_name#50, c_last_name#51] Input [7]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, c_customer_sk#49, c_first_name#50, c_last_name#51] (80) ReusedExchange [Reuses operator id: 92] Output [1]: [d_date_sk#52] -(81) BroadcastHashJoin [codegen id : 33] +(81) BroadcastHashJoin [codegen id : 35] Left keys [1]: [ws_sold_date_sk#47] Right keys [1]: [d_date_sk#52] Join type: Inner Join condition: None -(82) Project [codegen id : 33] +(82) Project [codegen id : 35] Output [4]: [ws_quantity#45, ws_list_price#46, c_first_name#50, c_last_name#51] Input [6]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, c_first_name#50, c_last_name#51, d_date_sk#52] -(83) HashAggregate [codegen id : 33] +(83) HashAggregate [codegen id : 35] Input [4]: [ws_quantity#45, ws_list_price#46, c_first_name#50, c_last_name#51] Keys [2]: [c_last_name#51, c_first_name#50] Functions [1]: [partial_sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))] @@ -485,7 +485,7 @@ Results [4]: [c_last_name#51, c_first_name#50, sum#55, isEmpty#56] Input [4]: [c_last_name#51, c_first_name#50, sum#55, isEmpty#56] Arguments: hashpartitioning(c_last_name#51, c_first_name#50, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(85) HashAggregate [codegen id : 34] +(85) HashAggregate [codegen id : 36] Input [4]: [c_last_name#51, c_first_name#50, sum#55, isEmpty#56] Keys [2]: [c_last_name#51, c_first_name#50] Functions [1]: [sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt index 9cbb568a9..49ddeaef8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] Union - WholeStageCodegen (17) + WholeStageCodegen (18) HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),sales,sum,isEmpty] InputAdapter Exchange [c_last_name,c_first_name] #1 - WholeStageCodegen (16) + WholeStageCodegen (17) HashAggregate [c_last_name,c_first_name,cs_quantity,cs_list_price] [sum,isEmpty,sum,isEmpty] Project [cs_quantity,cs_list_price,c_first_name,c_last_name] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] @@ -124,18 +124,20 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] CometScan parquet spark_catalog.default.customer [c_customer_sk] InputAdapter BroadcastExchange #13 - WholeStageCodegen (14) + WholeStageCodegen (15) SortMergeJoin [c_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (10) - ColumnarToRow + WholeStageCodegen (11) + Sort [c_customer_sk] InputAdapter - CometSort [c_customer_sk] - CometExchange [c_customer_sk] #14 - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] + Exchange [c_customer_sk] #14 + WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter - WholeStageCodegen (13) + WholeStageCodegen (14) Sort [c_customer_sk] Project [c_customer_sk] Filter [ssales] @@ -145,11 +147,11 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ReusedExchange [c_customer_sk,sum,isEmpty] #8 InputAdapter ReusedExchange [d_date_sk] #3 - WholeStageCodegen (34) + WholeStageCodegen (36) HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),sales,sum,isEmpty] InputAdapter Exchange [c_last_name,c_first_name] #15 - WholeStageCodegen (33) + WholeStageCodegen (35) HashAggregate [c_last_name,c_first_name,ws_quantity,ws_list_price] [sum,isEmpty,sum,isEmpty] Project [ws_quantity,ws_list_price,c_first_name,c_last_name] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] @@ -157,11 +159,11 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (23) + WholeStageCodegen (24) Sort [ws_bill_customer_sk] InputAdapter Exchange [ws_bill_customer_sk] #16 - WholeStageCodegen (22) + WholeStageCodegen (23) Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] BroadcastHashJoin [ws_item_sk,item_sk] ColumnarToRow @@ -172,7 +174,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] InputAdapter ReusedExchange [item_sk] #4 InputAdapter - WholeStageCodegen (26) + WholeStageCodegen (27) Sort [c_customer_sk] Project [c_customer_sk] Filter [ssales] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt index 6e03148d5..d2534e9be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt @@ -1,49 +1,50 @@ == Physical Plan == -* Filter (45) -+- * HashAggregate (44) - +- Exchange (43) - +- * HashAggregate (42) - +- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (20) - : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : :- * ColumnarToRow (13) - : : : : +- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometScan parquet spark_catalog.default.store_returns (6) - : : : +- BroadcastExchange (18) - : : : +- * ColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometFilter (15) - : : : +- CometScan parquet spark_catalog.default.store (14) - : : +- BroadcastExchange (24) - : : +- * ColumnarToRow (23) - : : +- CometFilter (22) - : : +- CometScan parquet spark_catalog.default.item (21) - : +- BroadcastExchange (30) - : +- * ColumnarToRow (29) - : +- CometFilter (28) - : +- CometScan parquet spark_catalog.default.customer (27) - +- BroadcastExchange (36) - +- * ColumnarToRow (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.customer_address (33) +* Filter (46) ++- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * Project (21) + : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : :- * Project (14) + : : : : +- * SortMergeJoin Inner (13) + : : : : :- * Sort (6) + : : : : : +- Exchange (5) + : : : : : +- * ColumnarToRow (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- * Sort (12) + : : : : +- Exchange (11) + : : : : +- * ColumnarToRow (10) + : : : : +- CometProject (9) + : : : : +- CometFilter (8) + : : : : +- CometScan parquet spark_catalog.default.store_returns (7) + : : : +- BroadcastExchange (19) + : : : +- * ColumnarToRow (18) + : : : +- CometProject (17) + : : : +- CometFilter (16) + : : : +- CometScan parquet spark_catalog.default.store (15) + : : +- BroadcastExchange (25) + : : +- * ColumnarToRow (24) + : : +- CometFilter (23) + : : +- CometScan parquet spark_catalog.default.item (22) + : +- BroadcastExchange (31) + : +- * ColumnarToRow (30) + : +- CometFilter (29) + : +- CometScan parquet spark_catalog.default.customer (28) + +- BroadcastExchange (37) + +- * ColumnarToRow (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.customer_address (34) (unknown) Scan parquet spark_catalog.default.store_sales @@ -61,13 +62,16 @@ Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND is Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -(4) CometExchange +(4) ColumnarToRow [codegen id : 1] Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(5) CometSort +(5) Exchange Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] +Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(6) Sort [codegen id : 2] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Arguments: [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.store_returns Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] @@ -76,33 +80,34 @@ Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(7) CometFilter +(8) CometFilter Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) -(8) CometProject +(9) CometProject Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] -(9) CometExchange +(10) ColumnarToRow [codegen id : 3] Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(10) CometSort +(11) Exchange Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] +Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(11) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner +(12) Sort [codegen id : 4] +Input [2]: [sr_item_sk#7, sr_ticket_number#8] +Arguments: [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST], false, 0 -(12) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +(13) SortMergeJoin [codegen id : 9] +Left keys [2]: [ss_ticket_number#4, ss_item_sk#1] +Right keys [2]: [sr_ticket_number#8, sr_item_sk#7] +Join type: Inner +Join condition: None -(13) ColumnarToRow [codegen id : 5] -Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +(14) Project [codegen id : 9] +Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] (unknown) Scan parquet spark_catalog.default.store Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] @@ -111,28 +116,28 @@ Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(15) CometFilter +(16) CometFilter Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(s_zip#14)) -(16) CometProject +(17) CometProject Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Arguments: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14], [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(17) ColumnarToRow [codegen id : 1] +(18) ColumnarToRow [codegen id : 5] Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(18) BroadcastExchange +(19) BroadcastExchange Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(19) BroadcastHashJoin [codegen id : 5] +(20) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#10] Join type: Inner Join condition: None -(20) Project [codegen id : 5] +(21) Project [codegen id : 9] Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] @@ -143,24 +148,24 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale ), IsNotNull(i_item_sk)] ReadSchema: struct -(22) CometFilter +(23) CometFilter Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Condition : ((isnotnull(i_color#18) AND (i_color#18 = pale )) AND isnotnull(i_item_sk#15)) -(23) ColumnarToRow [codegen id : 2] +(24) ColumnarToRow [codegen id : 6] Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(24) BroadcastExchange +(25) BroadcastExchange Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(25) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#15] Join type: Inner Join condition: None -(26) Project [codegen id : 5] +(27) Project [codegen id : 9] Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] @@ -171,24 +176,24 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct -(28) CometFilter +(29) CometFilter Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_birth_country#24)) -(29) ColumnarToRow [codegen id : 3] +(30) ColumnarToRow [codegen id : 7] Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] -(30) BroadcastExchange +(31) BroadcastExchange Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(31) BroadcastHashJoin [codegen id : 5] +(32) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_customer_sk#2] Right keys [1]: [c_customer_sk#21] Join type: Inner Join condition: None -(32) Project [codegen id : 5] +(33) Project [codegen id : 9] Output [12]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] @@ -199,70 +204,70 @@ Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct -(34) CometFilter +(35) CometFilter Input [3]: [ca_state#25, ca_zip#26, ca_country#27] Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) -(35) ColumnarToRow [codegen id : 4] +(36) ColumnarToRow [codegen id : 8] Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -(36) BroadcastExchange +(37) BroadcastExchange Input [3]: [ca_state#25, ca_zip#26, ca_country#27] Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [plan_id=6] -(37) BroadcastHashJoin [codegen id : 5] +(38) BroadcastHashJoin [codegen id : 9] Left keys [2]: [c_birth_country#24, s_zip#14] Right keys [2]: [upper(ca_country#27), ca_zip#26] Join type: Inner Join condition: None -(38) Project [codegen id : 5] +(39) Project [codegen id : 9] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] Input [15]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24, ca_state#25, ca_zip#26, ca_country#27] -(39) HashAggregate [codegen id : 5] +(40) HashAggregate [codegen id : 9] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum#28] Results [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] -(40) Exchange +(41) Exchange Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(41) HashAggregate [codegen id : 6] +(42) HashAggregate [codegen id : 10] Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#30] Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#30,17,2) AS netpaid#31] -(42) HashAggregate [codegen id : 6] +(43) HashAggregate [codegen id : 10] Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, netpaid#31] Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11] Functions [1]: [partial_sum(netpaid#31)] Aggregate Attributes [2]: [sum#32, isEmpty#33] Results [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] -(43) Exchange +(44) Exchange Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(44) HashAggregate [codegen id : 7] +(45) HashAggregate [codegen id : 11] Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11] Functions [1]: [sum(netpaid#31)] Aggregate Attributes [1]: [sum(netpaid#31)#36] Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, sum(netpaid#31)#36 AS paid#37] -(45) Filter [codegen id : 7] +(46) Filter [codegen id : 11] Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, paid#37] Condition : (isnotnull(paid#37) AND (cast(paid#37 as decimal(33,8)) > cast(Subquery scalar-subquery#38, [id=#39] as decimal(33,8)))) ===== Subqueries ===== -Subquery:1 Hosting operator id = 45 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#38, [id=#39] * HashAggregate (73) +- Exchange (72) +- * HashAggregate (71) @@ -277,13 +282,12 @@ Subquery:1 Hosting operator id = 45 Hosting Expression = Subquery scalar-subquer : : +- * BroadcastHashJoin Inner BuildRight (60) : : :- * Project (55) : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : :- * ColumnarToRow (52) - : : : : +- CometProject (51) - : : : : +- CometSortMergeJoin (50) - : : : : :- CometSort (47) - : : : : : +- ReusedExchange (46) - : : : : +- CometSort (49) - : : : : +- ReusedExchange (48) + : : : :- * Project (52) + : : : : +- * SortMergeJoin Inner (51) + : : : : :- * Sort (48) + : : : : : +- ReusedExchange (47) + : : : : +- * Sort (50) + : : : : +- ReusedExchange (49) : : : +- ReusedExchange (53) : : +- BroadcastExchange (59) : : +- * ColumnarToRow (58) @@ -293,42 +297,40 @@ Subquery:1 Hosting operator id = 45 Hosting Expression = Subquery scalar-subquer +- ReusedExchange (65) -(46) ReusedExchange [Reuses operator id: 4] +(47) ReusedExchange [Reuses operator id: 5] Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -(47) CometSort +(48) Sort [codegen id : 2] Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] +Arguments: [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST], false, 0 -(48) ReusedExchange [Reuses operator id: 9] +(49) ReusedExchange [Reuses operator id: 11] Output [2]: [sr_item_sk#7, sr_ticket_number#8] -(49) CometSort +(50) Sort [codegen id : 4] Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] +Arguments: [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST], false, 0 -(50) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner +(51) SortMergeJoin [codegen id : 9] +Left keys [2]: [ss_ticket_number#4, ss_item_sk#1] +Right keys [2]: [sr_ticket_number#8, sr_item_sk#7] +Join type: Inner +Join condition: None -(51) CometProject +(52) Project [codegen id : 9] +Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] - -(52) ColumnarToRow [codegen id : 5] -Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -(53) ReusedExchange [Reuses operator id: 18] +(53) ReusedExchange [Reuses operator id: 19] Output [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(54) BroadcastHashJoin [codegen id : 5] +(54) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#10] Join type: Inner Join condition: None -(55) Project [codegen id : 5] +(55) Project [codegen id : 9] Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] @@ -343,50 +345,50 @@ ReadSchema: struct -(7) CometFilter +(8) CometFilter Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) -(8) CometProject +(9) CometProject Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] -(9) CometExchange +(10) ColumnarToRow [codegen id : 3] Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(10) CometSort +(11) Exchange Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] +Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(11) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner +(12) Sort [codegen id : 4] +Input [2]: [sr_item_sk#7, sr_ticket_number#8] +Arguments: [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST], false, 0 -(12) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +(13) SortMergeJoin [codegen id : 9] +Left keys [2]: [ss_ticket_number#4, ss_item_sk#1] +Right keys [2]: [sr_ticket_number#8, sr_item_sk#7] +Join type: Inner +Join condition: None -(13) ColumnarToRow [codegen id : 5] -Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +(14) Project [codegen id : 9] +Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] (unknown) Scan parquet spark_catalog.default.store Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] @@ -111,28 +116,28 @@ Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(15) CometFilter +(16) CometFilter Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(s_zip#14)) -(16) CometProject +(17) CometProject Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Arguments: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14], [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(17) ColumnarToRow [codegen id : 1] +(18) ColumnarToRow [codegen id : 5] Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(18) BroadcastExchange +(19) BroadcastExchange Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(19) BroadcastHashJoin [codegen id : 5] +(20) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#10] Join type: Inner Join condition: None -(20) Project [codegen id : 5] +(21) Project [codegen id : 9] Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] @@ -143,24 +148,24 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_color), EqualTo(i_color,chiffon ), IsNotNull(i_item_sk)] ReadSchema: struct -(22) CometFilter +(23) CometFilter Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Condition : ((isnotnull(i_color#18) AND (i_color#18 = chiffon )) AND isnotnull(i_item_sk#15)) -(23) ColumnarToRow [codegen id : 2] +(24) ColumnarToRow [codegen id : 6] Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(24) BroadcastExchange +(25) BroadcastExchange Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(25) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#15] Join type: Inner Join condition: None -(26) Project [codegen id : 5] +(27) Project [codegen id : 9] Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] @@ -171,24 +176,24 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct -(28) CometFilter +(29) CometFilter Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_birth_country#24)) -(29) ColumnarToRow [codegen id : 3] +(30) ColumnarToRow [codegen id : 7] Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] -(30) BroadcastExchange +(31) BroadcastExchange Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(31) BroadcastHashJoin [codegen id : 5] +(32) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_customer_sk#2] Right keys [1]: [c_customer_sk#21] Join type: Inner Join condition: None -(32) Project [codegen id : 5] +(33) Project [codegen id : 9] Output [12]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] @@ -199,70 +204,70 @@ Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct -(34) CometFilter +(35) CometFilter Input [3]: [ca_state#25, ca_zip#26, ca_country#27] Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) -(35) ColumnarToRow [codegen id : 4] +(36) ColumnarToRow [codegen id : 8] Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -(36) BroadcastExchange +(37) BroadcastExchange Input [3]: [ca_state#25, ca_zip#26, ca_country#27] Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [plan_id=6] -(37) BroadcastHashJoin [codegen id : 5] +(38) BroadcastHashJoin [codegen id : 9] Left keys [2]: [c_birth_country#24, s_zip#14] Right keys [2]: [upper(ca_country#27), ca_zip#26] Join type: Inner Join condition: None -(38) Project [codegen id : 5] +(39) Project [codegen id : 9] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] Input [15]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24, ca_state#25, ca_zip#26, ca_country#27] -(39) HashAggregate [codegen id : 5] +(40) HashAggregate [codegen id : 9] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum#28] Results [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] -(40) Exchange +(41) Exchange Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(41) HashAggregate [codegen id : 6] +(42) HashAggregate [codegen id : 10] Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#30] Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#30,17,2) AS netpaid#31] -(42) HashAggregate [codegen id : 6] +(43) HashAggregate [codegen id : 10] Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, netpaid#31] Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11] Functions [1]: [partial_sum(netpaid#31)] Aggregate Attributes [2]: [sum#32, isEmpty#33] Results [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] -(43) Exchange +(44) Exchange Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(44) HashAggregate [codegen id : 7] +(45) HashAggregate [codegen id : 11] Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11] Functions [1]: [sum(netpaid#31)] Aggregate Attributes [1]: [sum(netpaid#31)#36] Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, sum(netpaid#31)#36 AS paid#37] -(45) Filter [codegen id : 7] +(46) Filter [codegen id : 11] Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, paid#37] Condition : (isnotnull(paid#37) AND (cast(paid#37 as decimal(33,8)) > cast(Subquery scalar-subquery#38, [id=#39] as decimal(33,8)))) ===== Subqueries ===== -Subquery:1 Hosting operator id = 45 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#38, [id=#39] * HashAggregate (73) +- Exchange (72) +- * HashAggregate (71) @@ -277,13 +282,12 @@ Subquery:1 Hosting operator id = 45 Hosting Expression = Subquery scalar-subquer : : +- * BroadcastHashJoin Inner BuildRight (60) : : :- * Project (55) : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : :- * ColumnarToRow (52) - : : : : +- CometProject (51) - : : : : +- CometSortMergeJoin (50) - : : : : :- CometSort (47) - : : : : : +- ReusedExchange (46) - : : : : +- CometSort (49) - : : : : +- ReusedExchange (48) + : : : :- * Project (52) + : : : : +- * SortMergeJoin Inner (51) + : : : : :- * Sort (48) + : : : : : +- ReusedExchange (47) + : : : : +- * Sort (50) + : : : : +- ReusedExchange (49) : : : +- ReusedExchange (53) : : +- BroadcastExchange (59) : : +- * ColumnarToRow (58) @@ -293,42 +297,40 @@ Subquery:1 Hosting operator id = 45 Hosting Expression = Subquery scalar-subquer +- ReusedExchange (65) -(46) ReusedExchange [Reuses operator id: 4] +(47) ReusedExchange [Reuses operator id: 5] Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -(47) CometSort +(48) Sort [codegen id : 2] Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] +Arguments: [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST], false, 0 -(48) ReusedExchange [Reuses operator id: 9] +(49) ReusedExchange [Reuses operator id: 11] Output [2]: [sr_item_sk#7, sr_ticket_number#8] -(49) CometSort +(50) Sort [codegen id : 4] Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] +Arguments: [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST], false, 0 -(50) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner +(51) SortMergeJoin [codegen id : 9] +Left keys [2]: [ss_ticket_number#4, ss_item_sk#1] +Right keys [2]: [sr_ticket_number#8, sr_item_sk#7] +Join type: Inner +Join condition: None -(51) CometProject +(52) Project [codegen id : 9] +Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] - -(52) ColumnarToRow [codegen id : 5] -Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -(53) ReusedExchange [Reuses operator id: 18] +(53) ReusedExchange [Reuses operator id: 19] Output [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(54) BroadcastHashJoin [codegen id : 5] +(54) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#10] Join type: Inner Join condition: None -(55) Project [codegen id : 5] +(55) Project [codegen id : 9] Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] @@ -343,50 +345,50 @@ ReadSchema: struct -(6) CometFilter +(7) CometFilter Input [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] Condition : (isnotnull(cr_order_number#8) AND isnotnull(cr_item_sk#7)) -(7) CometProject +(8) CometProject Input [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] Arguments: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9], [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -(8) CometExchange +(9) ColumnarToRow [codegen id : 3] +Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] + +(10) Exchange Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: hashpartitioning(cr_order_number#8, cr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +Arguments: hashpartitioning(cr_order_number#8, cr_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(9) CometSort +(11) Sort [codegen id : 4] Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9], [cr_order_number#8 ASC NULLS FIRST, cr_item_sk#7 ASC NULLS FIRST] +Arguments: [cr_order_number#8 ASC NULLS FIRST, cr_item_sk#7 ASC NULLS FIRST], false, 0 -(10) CometSortMergeJoin -Left output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Right output [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cs_order_number#3, cs_item_sk#2], [cr_order_number#8, cr_item_sk#7], LeftOuter +(12) SortMergeJoin [codegen id : 8] +Left keys [2]: [cs_order_number#3, cs_item_sk#2] +Right keys [2]: [cr_order_number#8, cr_item_sk#7] +Join type: LeftOuter +Join condition: None -(11) CometProject +(13) Project [codegen id : 8] +Output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9] Input [8]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5, cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9], [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9] - -(12) ColumnarToRow [codegen id : 4] -Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9] (unknown) Scan parquet spark_catalog.default.warehouse Output [2]: [w_warehouse_sk#11, w_state#12] @@ -95,24 +100,24 @@ Location [not included in comparison]/{warehouse_dir}/warehouse] PushedFilters: [IsNotNull(w_warehouse_sk)] ReadSchema: struct -(14) CometFilter +(15) CometFilter Input [2]: [w_warehouse_sk#11, w_state#12] Condition : isnotnull(w_warehouse_sk#11) -(15) ColumnarToRow [codegen id : 1] +(16) ColumnarToRow [codegen id : 5] Input [2]: [w_warehouse_sk#11, w_state#12] -(16) BroadcastExchange +(17) BroadcastExchange Input [2]: [w_warehouse_sk#11, w_state#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(17) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_warehouse_sk#1] Right keys [1]: [w_warehouse_sk#11] Join type: Inner Join condition: None -(18) Project [codegen id : 4] +(19) Project [codegen id : 8] Output [5]: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12] Input [7]: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_warehouse_sk#11, w_state#12] @@ -123,73 +128,73 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,0.99), LessThanOrEqual(i_current_price,1.49), IsNotNull(i_item_sk)] ReadSchema: struct -(20) CometFilter +(21) CometFilter Input [3]: [i_item_sk#13, i_item_id#14, i_current_price#15] Condition : (((isnotnull(i_current_price#15) AND (i_current_price#15 >= 0.99)) AND (i_current_price#15 <= 1.49)) AND isnotnull(i_item_sk#13)) -(21) CometProject +(22) CometProject Input [3]: [i_item_sk#13, i_item_id#14, i_current_price#15] Arguments: [i_item_sk#13, i_item_id#14], [i_item_sk#13, i_item_id#14] -(22) ColumnarToRow [codegen id : 2] +(23) ColumnarToRow [codegen id : 6] Input [2]: [i_item_sk#13, i_item_id#14] -(23) BroadcastExchange +(24) BroadcastExchange Input [2]: [i_item_sk#13, i_item_id#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(24) BroadcastHashJoin [codegen id : 4] +(25) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_item_sk#2] Right keys [1]: [i_item_sk#13] Join type: Inner Join condition: None -(25) Project [codegen id : 4] +(26) Project [codegen id : 8] Output [5]: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12, i_item_id#14] Input [7]: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12, i_item_sk#13, i_item_id#14] -(26) ReusedExchange [Reuses operator id: 36] +(27) ReusedExchange [Reuses operator id: 37] Output [2]: [d_date_sk#16, d_date#17] -(27) BroadcastHashJoin [codegen id : 4] +(28) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_sold_date_sk#5] Right keys [1]: [d_date_sk#16] Join type: Inner Join condition: None -(28) Project [codegen id : 4] +(29) Project [codegen id : 8] Output [5]: [cs_sales_price#4, cr_refunded_cash#9, w_state#12, i_item_id#14, d_date#17] Input [7]: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12, i_item_id#14, d_date_sk#16, d_date#17] -(29) HashAggregate [codegen id : 4] +(30) HashAggregate [codegen id : 8] Input [5]: [cs_sales_price#4, cr_refunded_cash#9, w_state#12, i_item_id#14, d_date#17] Keys [2]: [w_state#12, i_item_id#14] Functions [2]: [partial_sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END), partial_sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)] Aggregate Attributes [4]: [sum#18, isEmpty#19, sum#20, isEmpty#21] Results [6]: [w_state#12, i_item_id#14, sum#22, isEmpty#23, sum#24, isEmpty#25] -(30) Exchange +(31) Exchange Input [6]: [w_state#12, i_item_id#14, sum#22, isEmpty#23, sum#24, isEmpty#25] Arguments: hashpartitioning(w_state#12, i_item_id#14, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(31) HashAggregate [codegen id : 5] +(32) HashAggregate [codegen id : 9] Input [6]: [w_state#12, i_item_id#14, sum#22, isEmpty#23, sum#24, isEmpty#25] Keys [2]: [w_state#12, i_item_id#14] Functions [2]: [sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END), sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)] Aggregate Attributes [2]: [sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#26, sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#27] Results [4]: [w_state#12, i_item_id#14, sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#26 AS sales_before#28, sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#27 AS sales_after#29] -(32) TakeOrderedAndProject +(33) TakeOrderedAndProject Input [4]: [w_state#12, i_item_id#14, sales_before#28, sales_after#29] Arguments: 100, [w_state#12 ASC NULLS FIRST, i_item_id#14 ASC NULLS FIRST], [w_state#12, i_item_id#14, sales_before#28, sales_after#29] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (36) -+- * ColumnarToRow (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.date_dim (33) +BroadcastExchange (37) ++- * ColumnarToRow (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.date_dim (34) (unknown) Scan parquet spark_catalog.default.date_dim @@ -199,14 +204,14 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] ReadSchema: struct -(34) CometFilter +(35) CometFilter Input [2]: [d_date_sk#16, d_date#17] Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 2000-02-10)) AND (d_date#17 <= 2000-04-10)) AND isnotnull(d_date_sk#16)) -(35) ColumnarToRow [codegen id : 1] +(36) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#16, d_date#17] -(36) BroadcastExchange +(37) BroadcastExchange Input [2]: [d_date_sk#16, d_date#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt index 98d39c58d..10e0735b4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt @@ -1,9 +1,9 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] - WholeStageCodegen (5) + WholeStageCodegen (9) HashAggregate [w_state,i_item_id,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_date < 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END),sales_before,sales_after,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [w_state,i_item_id] #1 - WholeStageCodegen (4) + WholeStageCodegen (8) HashAggregate [w_state,i_item_id,d_date,cs_sales_price,cr_refunded_cash] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] Project [cs_sales_price,cr_refunded_cash,w_state,i_item_id,d_date] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] @@ -11,36 +11,46 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] BroadcastHashJoin [cs_item_sk,i_item_sk] Project [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state] BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - ColumnarToRow - InputAdapter - CometProject [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash] - CometSortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometSort [cs_order_number,cs_item_sk] - CometExchange [cs_order_number,cs_item_sk] #2 - CometFilter [cs_warehouse_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [cr_order_number,cr_item_sk] - CometExchange [cr_order_number,cr_item_sk] #4 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] - CometFilter [cr_order_number,cr_item_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] + Project [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash] + SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [cs_order_number,cs_item_sk] + InputAdapter + Exchange [cs_order_number,cs_item_sk] #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [cs_warehouse_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + WholeStageCodegen (4) + Sort [cr_order_number,cr_item_sk] + InputAdapter + Exchange [cr_order_number,cr_item_sk] #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] + CometFilter [cr_order_number,cr_item_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] InputAdapter BroadcastExchange #5 - WholeStageCodegen (1) + WholeStageCodegen (5) ColumnarToRow InputAdapter CometFilter [w_warehouse_sk] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] InputAdapter BroadcastExchange #6 - WholeStageCodegen (2) + WholeStageCodegen (6) ColumnarToRow InputAdapter CometProject [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt index 230899077..7174e2746 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt @@ -10,11 +10,11 @@ TakeOrderedAndProject (20) : +- CometFilter (2) : +- CometScan parquet spark_catalog.default.item (1) +- BroadcastExchange (14) - +- * ColumnarToRow (13) - +- CometProject (12) - +- CometFilter (11) - +- CometHashAggregate (10) - +- CometExchange (9) + +- * Project (13) + +- * Filter (12) + +- * HashAggregate (11) + +- Exchange (10) + +- * ColumnarToRow (9) +- CometHashAggregate (8) +- CometProject (7) +- CometFilter (6) @@ -36,7 +36,7 @@ Condition : (((isnotnull(i_manufact_id#1) AND (i_manufact_id#1 >= 738)) AND (i_m Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] Arguments: [i_manufact#2, i_product_name#3], [i_manufact#2, i_product_name#3] -(4) ColumnarToRow [codegen id : 2] +(4) ColumnarToRow [codegen id : 3] Input [2]: [i_manufact#2, i_product_name#3] (unknown) Scan parquet spark_catalog.default.item @@ -59,41 +59,43 @@ Input [1]: [i_manufact#5] Keys [1]: [i_manufact#5] Functions [1]: [partial_count(1)] -(9) CometExchange +(9) ColumnarToRow [codegen id : 1] Input [2]: [i_manufact#5, count#9] -Arguments: hashpartitioning(i_manufact#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(10) CometHashAggregate +(10) Exchange +Input [2]: [i_manufact#5, count#9] +Arguments: hashpartitioning(i_manufact#5, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(11) HashAggregate [codegen id : 2] Input [2]: [i_manufact#5, count#9] Keys [1]: [i_manufact#5] Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#10] +Results [2]: [count(1)#10 AS item_cnt#11, i_manufact#5] -(11) CometFilter -Input [2]: [item_cnt#10, i_manufact#5] -Condition : (item_cnt#10 > 0) +(12) Filter [codegen id : 2] +Input [2]: [item_cnt#11, i_manufact#5] +Condition : (item_cnt#11 > 0) -(12) CometProject -Input [2]: [item_cnt#10, i_manufact#5] -Arguments: [i_manufact#5], [i_manufact#5] - -(13) ColumnarToRow [codegen id : 1] -Input [1]: [i_manufact#5] +(13) Project [codegen id : 2] +Output [1]: [i_manufact#5] +Input [2]: [item_cnt#11, i_manufact#5] (14) BroadcastExchange Input [1]: [i_manufact#5] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] -(15) BroadcastHashJoin [codegen id : 2] +(15) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_manufact#2] Right keys [1]: [i_manufact#5] Join type: Inner Join condition: None -(16) Project [codegen id : 2] +(16) Project [codegen id : 3] Output [1]: [i_product_name#3] Input [3]: [i_manufact#2, i_product_name#3, i_manufact#5] -(17) HashAggregate [codegen id : 2] +(17) HashAggregate [codegen id : 3] Input [1]: [i_product_name#3] Keys [1]: [i_product_name#3] Functions: [] @@ -104,7 +106,7 @@ Results [1]: [i_product_name#3] Input [1]: [i_product_name#3] Arguments: hashpartitioning(i_product_name#3, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(19) HashAggregate [codegen id : 3] +(19) HashAggregate [codegen id : 4] Input [1]: [i_product_name#3] Keys [1]: [i_product_name#3] Functions: [] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt index 6c8ffadfd..e31217066 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt @@ -1,9 +1,9 @@ TakeOrderedAndProject [i_product_name] - WholeStageCodegen (3) + WholeStageCodegen (4) HashAggregate [i_product_name] InputAdapter Exchange [i_product_name] #1 - WholeStageCodegen (2) + WholeStageCodegen (3) HashAggregate [i_product_name] Project [i_product_name] BroadcastHashJoin [i_manufact,i_manufact] @@ -14,14 +14,16 @@ TakeOrderedAndProject [i_product_name] CometScan parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] InputAdapter BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [i_manufact] - CometFilter [item_cnt] - CometHashAggregate [i_manufact,count] - CometExchange [i_manufact] #3 - CometHashAggregate [i_manufact] - CometProject [i_manufact] - CometFilter [i_category,i_color,i_units,i_size,i_manufact] - CometScan parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] + WholeStageCodegen (2) + Project [i_manufact] + Filter [item_cnt] + HashAggregate [i_manufact,count] [count(1),item_cnt,count] + InputAdapter + Exchange [i_manufact] #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_manufact] + CometProject [i_manufact] + CometFilter [i_category,i_color,i_units,i_size,i_manufact] + CometScan parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt index d7f13b274..7a7d66737 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt @@ -1,37 +1,36 @@ == Physical Plan == -TakeOrderedAndProject (33) -+- * Project (32) - +- * BroadcastHashJoin Inner BuildRight (31) - :- * Project (29) - : +- * BroadcastHashJoin Inner BuildRight (28) - : :- * Project (23) - : : +- * SortMergeJoin Inner (22) +TakeOrderedAndProject (32) ++- * Project (31) + +- * BroadcastHashJoin Inner BuildRight (30) + :- * Project (28) + : +- * BroadcastHashJoin Inner BuildRight (27) + : :- * Project (22) + : : +- * SortMergeJoin Inner (21) : : :- * Sort (14) : : : +- * Project (13) : : : +- * Filter (12) : : : +- Window (11) - : : : +- * ColumnarToRow (10) - : : : +- CometSort (9) - : : : +- CometExchange (8) - : : : +- CometFilter (7) - : : : +- CometHashAggregate (6) - : : : +- CometExchange (5) + : : : +- * Sort (10) + : : : +- Exchange (9) + : : : +- * Filter (8) + : : : +- * HashAggregate (7) + : : : +- Exchange (6) + : : : +- * ColumnarToRow (5) : : : +- CometHashAggregate (4) : : : +- CometProject (3) : : : +- CometFilter (2) : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : +- * Sort (21) - : : +- * Project (20) - : : +- * Filter (19) - : : +- Window (18) - : : +- * ColumnarToRow (17) - : : +- CometSort (16) - : : +- ReusedExchange (15) - : +- BroadcastExchange (27) - : +- * ColumnarToRow (26) - : +- CometFilter (25) - : +- CometScan parquet spark_catalog.default.item (24) - +- ReusedExchange (30) + : : +- * Sort (20) + : : +- * Project (19) + : : +- * Filter (18) + : : +- Window (17) + : : +- * Sort (16) + : : +- ReusedExchange (15) + : +- BroadcastExchange (26) + : +- * ColumnarToRow (25) + : +- CometFilter (24) + : +- CometScan parquet spark_catalog.default.item (23) + +- ReusedExchange (29) (unknown) Scan parquet spark_catalog.default.store_sales @@ -54,169 +53,170 @@ Input [2]: [ss_item_sk#1, ss_net_profit#3] Keys [1]: [ss_item_sk#1] Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#3))] -(5) CometExchange +(5) ColumnarToRow [codegen id : 1] Input [3]: [ss_item_sk#1, sum#5, count#6] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(6) CometHashAggregate +(6) Exchange +Input [3]: [ss_item_sk#1, sum#5, count#6] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(7) HashAggregate [codegen id : 2] Input [3]: [ss_item_sk#1, sum#5, count#6] Keys [1]: [ss_item_sk#1] Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#3))#7] +Results [2]: [ss_item_sk#1 AS item_sk#8, cast((avg(UnscaledValue(ss_net_profit#3))#7 / 100.0) as decimal(11,6)) AS rank_col#9] -(7) CometFilter -Input [2]: [item_sk#7, rank_col#8] -Condition : (isnotnull(rank_col#8) AND (cast(rank_col#8 as decimal(13,7)) > (0.9 * Subquery scalar-subquery#9, [id=#10]))) +(8) Filter [codegen id : 2] +Input [2]: [item_sk#8, rank_col#9] +Condition : (isnotnull(rank_col#9) AND (cast(rank_col#9 as decimal(13,7)) > (0.9 * Subquery scalar-subquery#10, [id=#11]))) -(8) CometExchange -Input [2]: [item_sk#7, rank_col#8] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(9) Exchange +Input [2]: [item_sk#8, rank_col#9] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] -(9) CometSort -Input [2]: [item_sk#7, rank_col#8] -Arguments: [item_sk#7, rank_col#8], [rank_col#8 ASC NULLS FIRST] - -(10) ColumnarToRow [codegen id : 1] -Input [2]: [item_sk#7, rank_col#8] +(10) Sort [codegen id : 3] +Input [2]: [item_sk#8, rank_col#9] +Arguments: [rank_col#9 ASC NULLS FIRST], false, 0 (11) Window -Input [2]: [item_sk#7, rank_col#8] -Arguments: [rank(rank_col#8) windowspecdefinition(rank_col#8 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#11], [rank_col#8 ASC NULLS FIRST] - -(12) Filter [codegen id : 2] -Input [3]: [item_sk#7, rank_col#8, rnk#11] -Condition : ((rnk#11 < 11) AND isnotnull(item_sk#7)) +Input [2]: [item_sk#8, rank_col#9] +Arguments: [rank(rank_col#9) windowspecdefinition(rank_col#9 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#12], [rank_col#9 ASC NULLS FIRST] -(13) Project [codegen id : 2] -Output [2]: [item_sk#7, rnk#11] -Input [3]: [item_sk#7, rank_col#8, rnk#11] +(12) Filter [codegen id : 4] +Input [3]: [item_sk#8, rank_col#9, rnk#12] +Condition : ((rnk#12 < 11) AND isnotnull(item_sk#8)) -(14) Sort [codegen id : 2] -Input [2]: [item_sk#7, rnk#11] -Arguments: [rnk#11 ASC NULLS FIRST], false, 0 +(13) Project [codegen id : 4] +Output [2]: [item_sk#8, rnk#12] +Input [3]: [item_sk#8, rank_col#9, rnk#12] -(15) ReusedExchange [Reuses operator id: 8] -Output [2]: [item_sk#12, rank_col#13] +(14) Sort [codegen id : 4] +Input [2]: [item_sk#8, rnk#12] +Arguments: [rnk#12 ASC NULLS FIRST], false, 0 -(16) CometSort -Input [2]: [item_sk#12, rank_col#13] -Arguments: [item_sk#12, rank_col#13], [rank_col#13 DESC NULLS LAST] +(15) ReusedExchange [Reuses operator id: 9] +Output [2]: [item_sk#13, rank_col#14] -(17) ColumnarToRow [codegen id : 3] -Input [2]: [item_sk#12, rank_col#13] +(16) Sort [codegen id : 7] +Input [2]: [item_sk#13, rank_col#14] +Arguments: [rank_col#14 DESC NULLS LAST], false, 0 -(18) Window -Input [2]: [item_sk#12, rank_col#13] -Arguments: [rank(rank_col#13) windowspecdefinition(rank_col#13 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#14], [rank_col#13 DESC NULLS LAST] +(17) Window +Input [2]: [item_sk#13, rank_col#14] +Arguments: [rank(rank_col#14) windowspecdefinition(rank_col#14 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#15], [rank_col#14 DESC NULLS LAST] -(19) Filter [codegen id : 4] -Input [3]: [item_sk#12, rank_col#13, rnk#14] -Condition : ((rnk#14 < 11) AND isnotnull(item_sk#12)) +(18) Filter [codegen id : 8] +Input [3]: [item_sk#13, rank_col#14, rnk#15] +Condition : ((rnk#15 < 11) AND isnotnull(item_sk#13)) -(20) Project [codegen id : 4] -Output [2]: [item_sk#12, rnk#14] -Input [3]: [item_sk#12, rank_col#13, rnk#14] +(19) Project [codegen id : 8] +Output [2]: [item_sk#13, rnk#15] +Input [3]: [item_sk#13, rank_col#14, rnk#15] -(21) Sort [codegen id : 4] -Input [2]: [item_sk#12, rnk#14] -Arguments: [rnk#14 ASC NULLS FIRST], false, 0 +(20) Sort [codegen id : 8] +Input [2]: [item_sk#13, rnk#15] +Arguments: [rnk#15 ASC NULLS FIRST], false, 0 -(22) SortMergeJoin [codegen id : 7] -Left keys [1]: [rnk#11] -Right keys [1]: [rnk#14] +(21) SortMergeJoin [codegen id : 11] +Left keys [1]: [rnk#12] +Right keys [1]: [rnk#15] Join type: Inner Join condition: None -(23) Project [codegen id : 7] -Output [3]: [item_sk#7, rnk#11, item_sk#12] -Input [4]: [item_sk#7, rnk#11, item_sk#12, rnk#14] +(22) Project [codegen id : 11] +Output [3]: [item_sk#8, rnk#12, item_sk#13] +Input [4]: [item_sk#8, rnk#12, item_sk#13, rnk#15] (unknown) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#15, i_product_name#16] +Output [2]: [i_item_sk#16, i_product_name#17] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(25) CometFilter -Input [2]: [i_item_sk#15, i_product_name#16] -Condition : isnotnull(i_item_sk#15) +(24) CometFilter +Input [2]: [i_item_sk#16, i_product_name#17] +Condition : isnotnull(i_item_sk#16) -(26) ColumnarToRow [codegen id : 5] -Input [2]: [i_item_sk#15, i_product_name#16] +(25) ColumnarToRow [codegen id : 9] +Input [2]: [i_item_sk#16, i_product_name#17] -(27) BroadcastExchange -Input [2]: [i_item_sk#15, i_product_name#16] +(26) BroadcastExchange +Input [2]: [i_item_sk#16, i_product_name#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(28) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [item_sk#7] -Right keys [1]: [i_item_sk#15] +(27) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [item_sk#8] +Right keys [1]: [i_item_sk#16] Join type: Inner Join condition: None -(29) Project [codegen id : 7] -Output [3]: [rnk#11, item_sk#12, i_product_name#16] -Input [5]: [item_sk#7, rnk#11, item_sk#12, i_item_sk#15, i_product_name#16] +(28) Project [codegen id : 11] +Output [3]: [rnk#12, item_sk#13, i_product_name#17] +Input [5]: [item_sk#8, rnk#12, item_sk#13, i_item_sk#16, i_product_name#17] -(30) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#17, i_product_name#18] +(29) ReusedExchange [Reuses operator id: 26] +Output [2]: [i_item_sk#18, i_product_name#19] -(31) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [item_sk#12] -Right keys [1]: [i_item_sk#17] +(30) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [item_sk#13] +Right keys [1]: [i_item_sk#18] Join type: Inner Join condition: None -(32) Project [codegen id : 7] -Output [3]: [rnk#11, i_product_name#16 AS best_performing#19, i_product_name#18 AS worst_performing#20] -Input [5]: [rnk#11, item_sk#12, i_product_name#16, i_item_sk#17, i_product_name#18] +(31) Project [codegen id : 11] +Output [3]: [rnk#12, i_product_name#17 AS best_performing#20, i_product_name#19 AS worst_performing#21] +Input [5]: [rnk#12, item_sk#13, i_product_name#17, i_item_sk#18, i_product_name#19] -(33) TakeOrderedAndProject -Input [3]: [rnk#11, best_performing#19, worst_performing#20] -Arguments: 100, [rnk#11 ASC NULLS FIRST], [rnk#11, best_performing#19, worst_performing#20] +(32) TakeOrderedAndProject +Input [3]: [rnk#12, best_performing#20, worst_performing#21] +Arguments: 100, [rnk#12 ASC NULLS FIRST], [rnk#12, best_performing#20, worst_performing#21] ===== Subqueries ===== -Subquery:1 Hosting operator id = 7 Hosting Expression = Subquery scalar-subquery#9, [id=#10] -* ColumnarToRow (40) -+- CometHashAggregate (39) - +- CometExchange (38) - +- CometHashAggregate (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.store_sales (34) +Subquery:1 Hosting operator id = 8 Hosting Expression = Subquery scalar-subquery#10, [id=#11] +* HashAggregate (39) ++- Exchange (38) + +- * ColumnarToRow (37) + +- CometHashAggregate (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.store_sales (33) (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_addr_sk#21, ss_store_sk#22, ss_net_profit#23, ss_sold_date_sk#24] +Output [4]: [ss_addr_sk#22, ss_store_sk#23, ss_net_profit#24, ss_sold_date_sk#25] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)] ReadSchema: struct -(35) CometFilter -Input [4]: [ss_addr_sk#21, ss_store_sk#22, ss_net_profit#23, ss_sold_date_sk#24] -Condition : ((isnotnull(ss_store_sk#22) AND (ss_store_sk#22 = 4)) AND isnull(ss_addr_sk#21)) +(34) CometFilter +Input [4]: [ss_addr_sk#22, ss_store_sk#23, ss_net_profit#24, ss_sold_date_sk#25] +Condition : ((isnotnull(ss_store_sk#23) AND (ss_store_sk#23 = 4)) AND isnull(ss_addr_sk#22)) -(36) CometProject -Input [4]: [ss_addr_sk#21, ss_store_sk#22, ss_net_profit#23, ss_sold_date_sk#24] -Arguments: [ss_store_sk#22, ss_net_profit#23], [ss_store_sk#22, ss_net_profit#23] +(35) CometProject +Input [4]: [ss_addr_sk#22, ss_store_sk#23, ss_net_profit#24, ss_sold_date_sk#25] +Arguments: [ss_store_sk#23, ss_net_profit#24], [ss_store_sk#23, ss_net_profit#24] -(37) CometHashAggregate -Input [2]: [ss_store_sk#22, ss_net_profit#23] -Keys [1]: [ss_store_sk#22] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#23))] +(36) CometHashAggregate +Input [2]: [ss_store_sk#23, ss_net_profit#24] +Keys [1]: [ss_store_sk#23] +Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#24))] -(38) CometExchange -Input [3]: [ss_store_sk#22, sum#25, count#26] -Arguments: hashpartitioning(ss_store_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(37) ColumnarToRow [codegen id : 1] +Input [3]: [ss_store_sk#23, sum#26, count#27] -(39) CometHashAggregate -Input [3]: [ss_store_sk#22, sum#25, count#26] -Keys [1]: [ss_store_sk#22] -Functions [1]: [avg(UnscaledValue(ss_net_profit#23))] +(38) Exchange +Input [3]: [ss_store_sk#23, sum#26, count#27] +Arguments: hashpartitioning(ss_store_sk#23, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(40) ColumnarToRow [codegen id : 1] -Input [1]: [rank_col#27] +(39) HashAggregate [codegen id : 2] +Input [3]: [ss_store_sk#23, sum#26, count#27] +Keys [1]: [ss_store_sk#23] +Functions [1]: [avg(UnscaledValue(ss_net_profit#24))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#24))#28] +Results [1]: [cast((avg(UnscaledValue(ss_net_profit#24))#28 / 100.0) as decimal(11,6)) AS rank_col#29] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt index bc065345c..701a90912 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] - WholeStageCodegen (7) + WholeStageCodegen (11) Project [rnk,i_product_name,i_product_name] BroadcastHashJoin [item_sk,i_item_sk] Project [rnk,item_sk,i_product_name] @@ -7,49 +7,54 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] Project [item_sk,rnk,item_sk] SortMergeJoin [rnk,rnk] InputAdapter - WholeStageCodegen (2) + WholeStageCodegen (4) Sort [rnk] Project [item_sk,rnk] Filter [rnk,item_sk] InputAdapter Window [rank_col] - WholeStageCodegen (1) - ColumnarToRow + WholeStageCodegen (3) + Sort [rank_col] InputAdapter - CometSort [rank_col] - CometExchange #1 - CometFilter [rank_col] + Exchange #1 + WholeStageCodegen (2) + Filter [rank_col] Subquery #1 - WholeStageCodegen (1) - ColumnarToRow + WholeStageCodegen (2) + HashAggregate [ss_store_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),rank_col,sum,count] InputAdapter - CometHashAggregate [ss_store_sk,sum,count] - CometExchange [ss_store_sk] #3 - CometHashAggregate [ss_store_sk,ss_net_profit] - CometProject [ss_store_sk,ss_net_profit] - CometFilter [ss_store_sk,ss_addr_sk] - CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometHashAggregate [ss_item_sk,sum,count] - CometExchange [ss_item_sk] #2 - CometHashAggregate [ss_item_sk,ss_net_profit] - CometProject [ss_item_sk,ss_net_profit] - CometFilter [ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + Exchange [ss_store_sk] #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ss_store_sk,ss_net_profit] + CometProject [ss_store_sk,ss_net_profit] + CometFilter [ss_store_sk,ss_addr_sk] + CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,sum,count] + InputAdapter + Exchange [ss_item_sk] #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ss_item_sk,ss_net_profit] + CometProject [ss_item_sk,ss_net_profit] + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] InputAdapter - WholeStageCodegen (4) + WholeStageCodegen (8) Sort [rnk] Project [item_sk,rnk] Filter [rnk,item_sk] InputAdapter Window [rank_col] - WholeStageCodegen (3) - ColumnarToRow + WholeStageCodegen (7) + Sort [rank_col] InputAdapter - CometSort [rank_col] - ReusedExchange [item_sk,rank_col] #1 + ReusedExchange [item_sk,rank_col] #1 InputAdapter BroadcastExchange #4 - WholeStageCodegen (5) + WholeStageCodegen (9) ColumnarToRow InputAdapter CometFilter [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt index 00c8f9c9b..5c1e750c6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt @@ -389,9 +389,9 @@ Input [1]: [d_date_sk#29] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] Subquery:4 Hosting operator id = 62 Hosting Expression = Subquery scalar-subquery#42, [id=#43] -* ColumnarToRow (72) -+- CometHashAggregate (71) - +- CometExchange (70) +* HashAggregate (72) ++- Exchange (71) + +- * ColumnarToRow (70) +- CometHashAggregate (69) +- CometProject (68) +- CometFilter (67) @@ -418,22 +418,24 @@ Input [1]: [(d_month_seq + 1)#49] Keys [1]: [(d_month_seq + 1)#49] Functions: [] -(70) CometExchange +(70) ColumnarToRow [codegen id : 1] Input [1]: [(d_month_seq + 1)#49] -Arguments: hashpartitioning((d_month_seq + 1)#49, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] -(71) CometHashAggregate +(71) Exchange Input [1]: [(d_month_seq + 1)#49] -Keys [1]: [(d_month_seq + 1)#49] -Functions: [] +Arguments: hashpartitioning((d_month_seq + 1)#49, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(72) ColumnarToRow [codegen id : 1] +(72) HashAggregate [codegen id : 2] Input [1]: [(d_month_seq + 1)#49] +Keys [1]: [(d_month_seq + 1)#49] +Functions: [] +Aggregate Attributes: [] +Results [1]: [(d_month_seq + 1)#49] Subquery:5 Hosting operator id = 62 Hosting Expression = Subquery scalar-subquery#44, [id=#45] -* ColumnarToRow (79) -+- CometHashAggregate (78) - +- CometExchange (77) +* HashAggregate (79) ++- Exchange (78) + +- * ColumnarToRow (77) +- CometHashAggregate (76) +- CometProject (75) +- CometFilter (74) @@ -460,16 +462,18 @@ Input [1]: [(d_month_seq + 3)#53] Keys [1]: [(d_month_seq + 3)#53] Functions: [] -(77) CometExchange +(77) ColumnarToRow [codegen id : 1] Input [1]: [(d_month_seq + 3)#53] -Arguments: hashpartitioning((d_month_seq + 3)#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] -(78) CometHashAggregate +(78) Exchange Input [1]: [(d_month_seq + 3)#53] -Keys [1]: [(d_month_seq + 3)#53] -Functions: [] +Arguments: hashpartitioning((d_month_seq + 3)#53, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(79) ColumnarToRow [codegen id : 1] +(79) HashAggregate [codegen id : 2] Input [1]: [(d_month_seq + 3)#53] +Keys [1]: [(d_month_seq + 3)#53] +Functions: [] +Aggregate Attributes: [] +Results [1]: [(d_month_seq + 3)#53] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt index 90604339a..c1690facd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt @@ -79,25 +79,29 @@ TakeOrderedAndProject [segment,num_customers,segment_base] CometProject [d_date_sk] CometFilter [d_month_seq,d_date_sk] Subquery #3 - WholeStageCodegen (1) - ColumnarToRow + WholeStageCodegen (2) + HashAggregate [(d_month_seq + 1)] InputAdapter - CometHashAggregate [(d_month_seq + 1)] - CometExchange [(d_month_seq + 1)] #9 - CometHashAggregate [(d_month_seq + 1)] - CometProject [d_month_seq] [(d_month_seq + 1)] - CometFilter [d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + Exchange [(d_month_seq + 1)] #9 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [(d_month_seq + 1)] + CometProject [d_month_seq] [(d_month_seq + 1)] + CometFilter [d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] Subquery #4 - WholeStageCodegen (1) - ColumnarToRow + WholeStageCodegen (2) + HashAggregate [(d_month_seq + 3)] InputAdapter - CometHashAggregate [(d_month_seq + 3)] - CometExchange [(d_month_seq + 3)] #10 - CometHashAggregate [(d_month_seq + 3)] - CometProject [d_month_seq] [(d_month_seq + 3)] - CometFilter [d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + Exchange [(d_month_seq + 3)] #10 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [(d_month_seq + 3)] + CometProject [d_month_seq] [(d_month_seq + 3)] + CometFilter [d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #11 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt index 5f34aa3df..cc4720a19 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt @@ -31,10 +31,10 @@ TakeOrderedAndProject (39) : +- CometFilter (20) : +- CometScan parquet spark_catalog.default.item (19) +- BroadcastExchange (29) - +- * ColumnarToRow (28) - +- CometFilter (27) - +- CometHashAggregate (26) - +- CometExchange (25) + +- * Filter (28) + +- * HashAggregate (27) + +- Exchange (26) + +- * ColumnarToRow (25) +- CometHashAggregate (24) +- CometFilter (23) +- CometScan parquet spark_catalog.default.item (22) @@ -51,7 +51,7 @@ ReadSchema: struct Input [2]: [ca_address_sk#1, ca_state#2] Condition : isnotnull(ca_address_sk#1) -(3) ColumnarToRow [codegen id : 6] +(3) ColumnarToRow [codegen id : 7] Input [2]: [ca_address_sk#1, ca_state#2] (unknown) Scan parquet spark_catalog.default.customer @@ -72,13 +72,13 @@ Input [2]: [c_customer_sk#3, c_current_addr_sk#4] Input [2]: [c_customer_sk#3, c_current_addr_sk#4] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 6] +(8) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ca_address_sk#1] Right keys [1]: [c_current_addr_sk#4] Join type: Inner Join condition: None -(9) Project [codegen id : 6] +(9) Project [codegen id : 7] Output [2]: [ca_state#2, c_customer_sk#3] Input [4]: [ca_address_sk#1, ca_state#2, c_customer_sk#3, c_current_addr_sk#4] @@ -101,26 +101,26 @@ Input [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] Input [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=2] -(14) BroadcastHashJoin [codegen id : 6] +(14) BroadcastHashJoin [codegen id : 7] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#6] Join type: Inner Join condition: None -(15) Project [codegen id : 6] +(15) Project [codegen id : 7] Output [3]: [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7] Input [5]: [ca_state#2, c_customer_sk#3, ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] (16) ReusedExchange [Reuses operator id: 44] Output [1]: [d_date_sk#9] -(17) BroadcastHashJoin [codegen id : 6] +(17) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#9] Join type: Inner Join condition: None -(18) Project [codegen id : 6] +(18) Project [codegen id : 7] Output [2]: [ca_state#2, ss_item_sk#5] Input [4]: [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7, d_date_sk#9] @@ -135,7 +135,7 @@ ReadSchema: struct Input [3]: [i_item_sk#10, i_current_price#11, i_category#12] Condition : ((isnotnull(i_current_price#11) AND isnotnull(i_category#12)) AND isnotnull(i_item_sk#10)) -(21) ColumnarToRow [codegen id : 5] +(21) ColumnarToRow [codegen id : 6] Input [3]: [i_item_sk#10, i_current_price#11, i_category#12] (unknown) Scan parquet spark_catalog.default.item @@ -154,75 +154,77 @@ Input [2]: [i_current_price#13, i_category#14] Keys [1]: [i_category#14] Functions [1]: [partial_avg(UnscaledValue(i_current_price#13))] -(25) CometExchange +(25) ColumnarToRow [codegen id : 4] Input [3]: [i_category#14, sum#15, count#16] -Arguments: hashpartitioning(i_category#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(26) CometHashAggregate +(26) Exchange +Input [3]: [i_category#14, sum#15, count#16] +Arguments: hashpartitioning(i_category#14, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(27) HashAggregate [codegen id : 5] Input [3]: [i_category#14, sum#15, count#16] Keys [1]: [i_category#14] Functions [1]: [avg(UnscaledValue(i_current_price#13))] +Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#13))#17] +Results [2]: [cast((avg(UnscaledValue(i_current_price#13))#17 / 100.0) as decimal(11,6)) AS avg(i_current_price)#18, i_category#14] -(27) CometFilter -Input [2]: [avg(i_current_price)#17, i_category#14] -Condition : isnotnull(avg(i_current_price)#17) - -(28) ColumnarToRow [codegen id : 4] -Input [2]: [avg(i_current_price)#17, i_category#14] +(28) Filter [codegen id : 5] +Input [2]: [avg(i_current_price)#18, i_category#14] +Condition : isnotnull(avg(i_current_price)#18) (29) BroadcastExchange -Input [2]: [avg(i_current_price)#17, i_category#14] +Input [2]: [avg(i_current_price)#18, i_category#14] Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=4] -(30) BroadcastHashJoin [codegen id : 5] +(30) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_category#12] Right keys [1]: [i_category#14] Join type: Inner -Join condition: (cast(i_current_price#11 as decimal(14,7)) > (1.2 * avg(i_current_price)#17)) +Join condition: (cast(i_current_price#11 as decimal(14,7)) > (1.2 * avg(i_current_price)#18)) -(31) Project [codegen id : 5] +(31) Project [codegen id : 6] Output [1]: [i_item_sk#10] -Input [5]: [i_item_sk#10, i_current_price#11, i_category#12, avg(i_current_price)#17, i_category#14] +Input [5]: [i_item_sk#10, i_current_price#11, i_category#12, avg(i_current_price)#18, i_category#14] (32) BroadcastExchange Input [1]: [i_item_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(33) BroadcastHashJoin [codegen id : 6] +(33) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ss_item_sk#5] Right keys [1]: [i_item_sk#10] Join type: Inner Join condition: None -(34) Project [codegen id : 6] +(34) Project [codegen id : 7] Output [1]: [ca_state#2] Input [3]: [ca_state#2, ss_item_sk#5, i_item_sk#10] -(35) HashAggregate [codegen id : 6] +(35) HashAggregate [codegen id : 7] Input [1]: [ca_state#2] Keys [1]: [ca_state#2] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#18] -Results [2]: [ca_state#2, count#19] +Aggregate Attributes [1]: [count#19] +Results [2]: [ca_state#2, count#20] (36) Exchange -Input [2]: [ca_state#2, count#19] +Input [2]: [ca_state#2, count#20] Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(37) HashAggregate [codegen id : 7] -Input [2]: [ca_state#2, count#19] +(37) HashAggregate [codegen id : 8] +Input [2]: [ca_state#2, count#20] Keys [1]: [ca_state#2] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#20] -Results [2]: [ca_state#2 AS state#21, count(1)#20 AS cnt#22] +Aggregate Attributes [1]: [count(1)#21] +Results [2]: [ca_state#2 AS state#22, count(1)#21 AS cnt#23] -(38) Filter [codegen id : 7] -Input [2]: [state#21, cnt#22] -Condition : (cnt#22 >= 10) +(38) Filter [codegen id : 8] +Input [2]: [state#22, cnt#23] +Condition : (cnt#23 >= 10) (39) TakeOrderedAndProject -Input [2]: [state#21, cnt#22] -Arguments: 100, [cnt#22 ASC NULLS FIRST], [state#21, cnt#22] +Input [2]: [state#22, cnt#23] +Arguments: 100, [cnt#23 ASC NULLS FIRST], [state#22, cnt#23] ===== Subqueries ===== @@ -235,18 +237,18 @@ BroadcastExchange (44) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#9, d_month_seq#23] +Output [2]: [d_date_sk#9, d_month_seq#24] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct (41) CometFilter -Input [2]: [d_date_sk#9, d_month_seq#23] -Condition : ((isnotnull(d_month_seq#23) AND (d_month_seq#23 = Subquery scalar-subquery#24, [id=#25])) AND isnotnull(d_date_sk#9)) +Input [2]: [d_date_sk#9, d_month_seq#24] +Condition : ((isnotnull(d_month_seq#24) AND (d_month_seq#24 = Subquery scalar-subquery#25, [id=#26])) AND isnotnull(d_date_sk#9)) (42) CometProject -Input [2]: [d_date_sk#9, d_month_seq#23] +Input [2]: [d_date_sk#9, d_month_seq#24] Arguments: [d_date_sk#9], [d_date_sk#9] (43) ColumnarToRow [codegen id : 1] @@ -256,10 +258,10 @@ Input [1]: [d_date_sk#9] Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 41 Hosting Expression = Subquery scalar-subquery#24, [id=#25] -* ColumnarToRow (51) -+- CometHashAggregate (50) - +- CometExchange (49) +Subquery:2 Hosting operator id = 41 Hosting Expression = Subquery scalar-subquery#25, [id=#26] +* HashAggregate (51) ++- Exchange (50) + +- * ColumnarToRow (49) +- CometHashAggregate (48) +- CometProject (47) +- CometFilter (46) @@ -267,35 +269,37 @@ Subquery:2 Hosting operator id = 41 Hosting Expression = Subquery scalar-subquer (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#26, d_year#27, d_moy#28] +Output [3]: [d_month_seq#27, d_year#28, 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,1)] ReadSchema: struct (46) CometFilter -Input [3]: [d_month_seq#26, d_year#27, d_moy#28] -Condition : (((isnotnull(d_year#27) AND isnotnull(d_moy#28)) AND (d_year#27 = 2000)) AND (d_moy#28 = 1)) +Input [3]: [d_month_seq#27, d_year#28, d_moy#29] +Condition : (((isnotnull(d_year#28) AND isnotnull(d_moy#29)) AND (d_year#28 = 2000)) AND (d_moy#29 = 1)) (47) CometProject -Input [3]: [d_month_seq#26, d_year#27, d_moy#28] -Arguments: [d_month_seq#26], [d_month_seq#26] +Input [3]: [d_month_seq#27, d_year#28, d_moy#29] +Arguments: [d_month_seq#27], [d_month_seq#27] (48) CometHashAggregate -Input [1]: [d_month_seq#26] -Keys [1]: [d_month_seq#26] +Input [1]: [d_month_seq#27] +Keys [1]: [d_month_seq#27] Functions: [] -(49) CometExchange -Input [1]: [d_month_seq#26] -Arguments: hashpartitioning(d_month_seq#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +(49) ColumnarToRow [codegen id : 1] +Input [1]: [d_month_seq#27] -(50) CometHashAggregate -Input [1]: [d_month_seq#26] -Keys [1]: [d_month_seq#26] -Functions: [] +(50) Exchange +Input [1]: [d_month_seq#27] +Arguments: hashpartitioning(d_month_seq#27, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(51) ColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#26] +(51) HashAggregate [codegen id : 2] +Input [1]: [d_month_seq#27] +Keys [1]: [d_month_seq#27] +Functions: [] +Aggregate Attributes: [] +Results [1]: [d_month_seq#27] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt index 24a40f804..fc6d5d012 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [cnt,state] - WholeStageCodegen (7) + WholeStageCodegen (8) Filter [cnt] HashAggregate [ca_state,count] [count(1),state,cnt,count] InputAdapter Exchange [ca_state] #1 - WholeStageCodegen (6) + WholeStageCodegen (7) HashAggregate [ca_state] [count,count] Project [ca_state] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -40,21 +40,23 @@ TakeOrderedAndProject [cnt,state] CometProject [d_date_sk] CometFilter [d_month_seq,d_date_sk] Subquery #2 - WholeStageCodegen (1) - ColumnarToRow + WholeStageCodegen (2) + HashAggregate [d_month_seq] InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + Exchange [d_month_seq] #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #6 - WholeStageCodegen (5) + WholeStageCodegen (6) Project [i_item_sk] BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] ColumnarToRow @@ -63,12 +65,14 @@ TakeOrderedAndProject [cnt,state] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] InputAdapter BroadcastExchange #7 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [avg(i_current_price)] - CometHashAggregate [i_category,sum,count] - CometExchange [i_category] #8 - CometHashAggregate [i_category,i_current_price] - CometFilter [i_category] - CometScan parquet spark_catalog.default.item [i_current_price,i_category] + WholeStageCodegen (5) + Filter [avg(i_current_price)] + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count] + InputAdapter + Exchange [i_category] #8 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometHashAggregate [i_category,i_current_price] + CometFilter [i_category] + CometScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt index c21eda6be..a6f957162 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt @@ -1,183 +1,185 @@ == Physical Plan == -* Sort (179) -+- Exchange (178) - +- * Project (177) - +- * SortMergeJoin Inner (176) - :- * Sort (108) - : +- Exchange (107) - : +- * HashAggregate (106) - : +- * HashAggregate (105) - : +- * Project (104) - : +- * BroadcastHashJoin Inner BuildRight (103) - : :- * Project (97) - : : +- * BroadcastHashJoin Inner BuildRight (96) - : : :- * Project (94) - : : : +- * BroadcastHashJoin Inner BuildRight (93) - : : : :- * Project (88) - : : : : +- * BroadcastHashJoin Inner BuildRight (87) - : : : : :- * Project (85) - : : : : : +- * BroadcastHashJoin Inner BuildRight (84) - : : : : : :- * Project (79) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (78) - : : : : : : :- * Project (76) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (75) - : : : : : : : :- * Project (70) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (69) - : : : : : : : : :- * Project (64) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (63) - : : : : : : : : : :- * Project (61) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (60) - : : : : : : : : : : :- * Project (55) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : : : : : : : : : :- * Project (52) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (51) - : : : : : : : : : : : : :- * Project (46) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (45) - : : : : : : : : : : : : : :- * Project (40) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (39) - : : : : : : : : : : : : : : :- * Project (34) - : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : : : : : : : : : : : : : :- * ColumnarToRow (31) - : : : : : : : : : : : : : : : : +- CometProject (30) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (29) - : : : : : : : : : : : : : : : : :- CometSort (10) - : : : : : : : : : : : : : : : : : +- CometExchange (9) - : : : : : : : : : : : : : : : : : +- CometProject (8) - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) - : : : : : : : : : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- CometProject (6) - : : : : : : : : : : : : : : : : : +- CometFilter (5) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : : : : : : : : : : : : : : : +- CometSort (28) - : : : : : : : : : : : : : : : : +- CometProject (27) - : : : : : : : : : : : : : : : : +- CometFilter (26) - : : : : : : : : : : : : : : : : +- CometHashAggregate (25) - : : : : : : : : : : : : : : : : +- CometExchange (24) - : : : : : : : : : : : : : : : : +- CometHashAggregate (23) - : : : : : : : : : : : : : : : : +- CometProject (22) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (21) - : : : : : : : : : : : : : : : : :- CometSort (15) - : : : : : : : : : : : : : : : : : +- CometExchange (14) - : : : : : : : : : : : : : : : : : +- CometProject (13) - : : : : : : : : : : : : : : : : : +- CometFilter (12) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (11) - : : : : : : : : : : : : : : : : +- CometSort (20) - : : : : : : : : : : : : : : : : +- CometExchange (19) - : : : : : : : : : : : : : : : : +- CometProject (18) - : : : : : : : : : : : : : : : : +- CometFilter (17) - : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (16) - : : : : : : : : : : : : : : : +- ReusedExchange (32) - : : : : : : : : : : : : : : +- BroadcastExchange (38) - : : : : : : : : : : : : : : +- * ColumnarToRow (37) - : : : : : : : : : : : : : : +- CometFilter (36) - : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (35) - : : : : : : : : : : : : : +- BroadcastExchange (44) - : : : : : : : : : : : : : +- * ColumnarToRow (43) - : : : : : : : : : : : : : +- CometFilter (42) - : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (41) - : : : : : : : : : : : : +- BroadcastExchange (50) - : : : : : : : : : : : : +- * ColumnarToRow (49) - : : : : : : : : : : : : +- CometFilter (48) - : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (47) - : : : : : : : : : : : +- ReusedExchange (53) - : : : : : : : : : : +- BroadcastExchange (59) - : : : : : : : : : : +- * ColumnarToRow (58) - : : : : : : : : : : +- CometFilter (57) - : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (56) - : : : : : : : : : +- ReusedExchange (62) - : : : : : : : : +- BroadcastExchange (68) - : : : : : : : : +- * ColumnarToRow (67) - : : : : : : : : +- CometFilter (66) - : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (65) - : : : : : : : +- BroadcastExchange (74) - : : : : : : : +- * ColumnarToRow (73) - : : : : : : : +- CometFilter (72) - : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (71) - : : : : : : +- ReusedExchange (77) - : : : : : +- BroadcastExchange (83) - : : : : : +- * ColumnarToRow (82) - : : : : : +- CometFilter (81) - : : : : : +- CometScan parquet spark_catalog.default.customer_address (80) - : : : : +- ReusedExchange (86) - : : : +- BroadcastExchange (92) - : : : +- * ColumnarToRow (91) - : : : +- CometFilter (90) - : : : +- CometScan parquet spark_catalog.default.income_band (89) - : : +- ReusedExchange (95) - : +- BroadcastExchange (102) - : +- * ColumnarToRow (101) - : +- CometProject (100) - : +- CometFilter (99) - : +- CometScan parquet spark_catalog.default.item (98) - +- * Sort (175) - +- Exchange (174) - +- * HashAggregate (173) - +- * HashAggregate (172) - +- * Project (171) - +- * BroadcastHashJoin Inner BuildRight (170) - :- * Project (168) - : +- * BroadcastHashJoin Inner BuildRight (167) - : :- * Project (165) - : : +- * BroadcastHashJoin Inner BuildRight (164) - : : :- * Project (162) - : : : +- * BroadcastHashJoin Inner BuildRight (161) - : : : :- * Project (159) - : : : : +- * BroadcastHashJoin Inner BuildRight (158) - : : : : :- * Project (156) - : : : : : +- * BroadcastHashJoin Inner BuildRight (155) - : : : : : :- * Project (153) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (152) - : : : : : : :- * Project (150) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (149) - : : : : : : : :- * Project (147) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (146) - : : : : : : : : :- * Project (144) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (143) - : : : : : : : : : :- * Project (141) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (140) - : : : : : : : : : : :- * Project (138) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (137) - : : : : : : : : : : : :- * Project (135) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (134) - : : : : : : : : : : : : :- * Project (132) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (131) - : : : : : : : : : : : : : :- * Project (129) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (128) - : : : : : : : : : : : : : : :- * ColumnarToRow (126) - : : : : : : : : : : : : : : : +- CometProject (125) - : : : : : : : : : : : : : : : +- CometSortMergeJoin (124) - : : : : : : : : : : : : : : : :- CometSort (118) - : : : : : : : : : : : : : : : : +- CometExchange (117) - : : : : : : : : : : : : : : : : +- CometProject (116) - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (115) - : : : : : : : : : : : : : : : : :- CometBroadcastExchange (111) - : : : : : : : : : : : : : : : : : +- CometFilter (110) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (109) - : : : : : : : : : : : : : : : : +- CometProject (114) - : : : : : : : : : : : : : : : : +- CometFilter (113) - : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (112) - : : : : : : : : : : : : : : : +- CometSort (123) - : : : : : : : : : : : : : : : +- CometProject (122) - : : : : : : : : : : : : : : : +- CometFilter (121) - : : : : : : : : : : : : : : : +- CometHashAggregate (120) - : : : : : : : : : : : : : : : +- ReusedExchange (119) - : : : : : : : : : : : : : : +- ReusedExchange (127) - : : : : : : : : : : : : : +- ReusedExchange (130) - : : : : : : : : : : : : +- ReusedExchange (133) - : : : : : : : : : : : +- ReusedExchange (136) - : : : : : : : : : : +- ReusedExchange (139) - : : : : : : : : : +- ReusedExchange (142) - : : : : : : : : +- ReusedExchange (145) - : : : : : : : +- ReusedExchange (148) - : : : : : : +- ReusedExchange (151) - : : : : : +- ReusedExchange (154) - : : : : +- ReusedExchange (157) - : : : +- ReusedExchange (160) - : : +- ReusedExchange (163) - : +- ReusedExchange (166) - +- ReusedExchange (169) +* Sort (181) ++- Exchange (180) + +- * Project (179) + +- * SortMergeJoin Inner (178) + :- * Sort (110) + : +- Exchange (109) + : +- * HashAggregate (108) + : +- * HashAggregate (107) + : +- * Project (106) + : +- * BroadcastHashJoin Inner BuildRight (105) + : :- * Project (99) + : : +- * BroadcastHashJoin Inner BuildRight (98) + : : :- * Project (96) + : : : +- * BroadcastHashJoin Inner BuildRight (95) + : : : :- * Project (90) + : : : : +- * BroadcastHashJoin Inner BuildRight (89) + : : : : :- * Project (87) + : : : : : +- * BroadcastHashJoin Inner BuildRight (86) + : : : : : :- * Project (81) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (80) + : : : : : : :- * Project (78) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (77) + : : : : : : : :- * Project (72) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (71) + : : : : : : : : :- * Project (66) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (65) + : : : : : : : : : :- * Project (63) + : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (62) + : : : : : : : : : : :- * Project (57) + : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (56) + : : : : : : : : : : : :- * Project (54) + : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (53) + : : : : : : : : : : : : :- * Project (48) + : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (47) + : : : : : : : : : : : : : :- * Project (42) + : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (41) + : : : : : : : : : : : : : : :- * Project (36) + : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (35) + : : : : : : : : : : : : : : : :- * Project (33) + : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (32) + : : : : : : : : : : : : : : : : :- * Sort (11) + : : : : : : : : : : : : : : : : : +- Exchange (10) + : : : : : : : : : : : : : : : : : +- * ColumnarToRow (9) + : : : : : : : : : : : : : : : : : +- CometProject (8) + : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) + : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) + : : : : : : : : : : : : : : : : : : +- CometFilter (2) + : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : : : : : : : : : : : : +- CometProject (6) + : : : : : : : : : : : : : : : : : +- CometFilter (5) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : : : : : : : : : : : : : : : +- * Sort (31) + : : : : : : : : : : : : : : : : +- * Project (30) + : : : : : : : : : : : : : : : : +- * Filter (29) + : : : : : : : : : : : : : : : : +- * HashAggregate (28) + : : : : : : : : : : : : : : : : +- Exchange (27) + : : : : : : : : : : : : : : : : +- * HashAggregate (26) + : : : : : : : : : : : : : : : : +- * Project (25) + : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (24) + : : : : : : : : : : : : : : : : :- * Sort (17) + : : : : : : : : : : : : : : : : : +- Exchange (16) + : : : : : : : : : : : : : : : : : +- * ColumnarToRow (15) + : : : : : : : : : : : : : : : : : +- CometProject (14) + : : : : : : : : : : : : : : : : : +- CometFilter (13) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (12) + : : : : : : : : : : : : : : : : +- * Sort (23) + : : : : : : : : : : : : : : : : +- Exchange (22) + : : : : : : : : : : : : : : : : +- * ColumnarToRow (21) + : : : : : : : : : : : : : : : : +- CometProject (20) + : : : : : : : : : : : : : : : : +- CometFilter (19) + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (18) + : : : : : : : : : : : : : : : +- ReusedExchange (34) + : : : : : : : : : : : : : : +- BroadcastExchange (40) + : : : : : : : : : : : : : : +- * ColumnarToRow (39) + : : : : : : : : : : : : : : +- CometFilter (38) + : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (37) + : : : : : : : : : : : : : +- BroadcastExchange (46) + : : : : : : : : : : : : : +- * ColumnarToRow (45) + : : : : : : : : : : : : : +- CometFilter (44) + : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (43) + : : : : : : : : : : : : +- BroadcastExchange (52) + : : : : : : : : : : : : +- * ColumnarToRow (51) + : : : : : : : : : : : : +- CometFilter (50) + : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (49) + : : : : : : : : : : : +- ReusedExchange (55) + : : : : : : : : : : +- BroadcastExchange (61) + : : : : : : : : : : +- * ColumnarToRow (60) + : : : : : : : : : : +- CometFilter (59) + : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (58) + : : : : : : : : : +- ReusedExchange (64) + : : : : : : : : +- BroadcastExchange (70) + : : : : : : : : +- * ColumnarToRow (69) + : : : : : : : : +- CometFilter (68) + : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (67) + : : : : : : : +- BroadcastExchange (76) + : : : : : : : +- * ColumnarToRow (75) + : : : : : : : +- CometFilter (74) + : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (73) + : : : : : : +- ReusedExchange (79) + : : : : : +- BroadcastExchange (85) + : : : : : +- * ColumnarToRow (84) + : : : : : +- CometFilter (83) + : : : : : +- CometScan parquet spark_catalog.default.customer_address (82) + : : : : +- ReusedExchange (88) + : : : +- BroadcastExchange (94) + : : : +- * ColumnarToRow (93) + : : : +- CometFilter (92) + : : : +- CometScan parquet spark_catalog.default.income_band (91) + : : +- ReusedExchange (97) + : +- BroadcastExchange (104) + : +- * ColumnarToRow (103) + : +- CometProject (102) + : +- CometFilter (101) + : +- CometScan parquet spark_catalog.default.item (100) + +- * Sort (177) + +- Exchange (176) + +- * HashAggregate (175) + +- * HashAggregate (174) + +- * Project (173) + +- * BroadcastHashJoin Inner BuildRight (172) + :- * Project (170) + : +- * BroadcastHashJoin Inner BuildRight (169) + : :- * Project (167) + : : +- * BroadcastHashJoin Inner BuildRight (166) + : : :- * Project (164) + : : : +- * BroadcastHashJoin Inner BuildRight (163) + : : : :- * Project (161) + : : : : +- * BroadcastHashJoin Inner BuildRight (160) + : : : : :- * Project (158) + : : : : : +- * BroadcastHashJoin Inner BuildRight (157) + : : : : : :- * Project (155) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (154) + : : : : : : :- * Project (152) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (151) + : : : : : : : :- * Project (149) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (148) + : : : : : : : : :- * Project (146) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (145) + : : : : : : : : : :- * Project (143) + : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (142) + : : : : : : : : : : :- * Project (140) + : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (139) + : : : : : : : : : : : :- * Project (137) + : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (136) + : : : : : : : : : : : : :- * Project (134) + : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (133) + : : : : : : : : : : : : : :- * Project (131) + : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (130) + : : : : : : : : : : : : : : :- * Project (128) + : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (127) + : : : : : : : : : : : : : : : :- * Sort (121) + : : : : : : : : : : : : : : : : +- Exchange (120) + : : : : : : : : : : : : : : : : +- * ColumnarToRow (119) + : : : : : : : : : : : : : : : : +- CometProject (118) + : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (117) + : : : : : : : : : : : : : : : : :- CometBroadcastExchange (113) + : : : : : : : : : : : : : : : : : +- CometFilter (112) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (111) + : : : : : : : : : : : : : : : : +- CometProject (116) + : : : : : : : : : : : : : : : : +- CometFilter (115) + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (114) + : : : : : : : : : : : : : : : +- * Sort (126) + : : : : : : : : : : : : : : : +- * Project (125) + : : : : : : : : : : : : : : : +- * Filter (124) + : : : : : : : : : : : : : : : +- * HashAggregate (123) + : : : : : : : : : : : : : : : +- ReusedExchange (122) + : : : : : : : : : : : : : : +- ReusedExchange (129) + : : : : : : : : : : : : : +- ReusedExchange (132) + : : : : : : : : : : : : +- ReusedExchange (135) + : : : : : : : : : : : +- ReusedExchange (138) + : : : : : : : : : : +- ReusedExchange (141) + : : : : : : : : : +- ReusedExchange (144) + : : : : : : : : +- ReusedExchange (147) + : : : : : : : +- ReusedExchange (150) + : : : : : : +- ReusedExchange (153) + : : : : : +- ReusedExchange (156) + : : : : +- ReusedExchange (159) + : : : +- ReusedExchange (162) + : : +- ReusedExchange (165) + : +- ReusedExchange (168) + +- ReusedExchange (171) (unknown) Scan parquet spark_catalog.default.store_sales @@ -220,13 +222,16 @@ Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number# Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -(9) CometExchange +(9) ColumnarToRow [codegen id : 1] Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(10) CometSort +(10) Exchange Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1 ASC NULLS FIRST] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(11) Sort [codegen id : 2] +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.catalog_sales Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] @@ -235,21 +240,24 @@ Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_order_number)] ReadSchema: struct -(12) CometFilter +(13) CometFilter Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) -(13) CometProject +(14) CometProject Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -(14) CometExchange +(15) ColumnarToRow [codegen id : 3] Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(15) CometSort +(16) Exchange Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST] +Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(17) Sort [codegen id : 4] +Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] +Arguments: [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.catalog_returns Output [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] @@ -258,790 +266,799 @@ Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(17) CometFilter +(19) CometFilter Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) -(18) CometProject +(20) CometProject Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -(19) CometExchange +(21) ColumnarToRow [codegen id : 5] Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(20) CometSort +(22) Exchange Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST] +Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) CometSortMergeJoin -Left output [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Right output [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number#22], Inner +(23) Sort [codegen id : 6] +Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +Arguments: [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST], false, 0 -(22) CometProject +(24) SortMergeJoin [codegen id : 7] +Left keys [2]: [cs_item_sk#17, cs_order_number#18] +Right keys [2]: [cr_item_sk#21, cr_order_number#22] +Join type: Inner +Join condition: None + +(25) Project [codegen id : 7] +Output [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -(23) CometHashAggregate +(26) HashAggregate [codegen id : 7] Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Keys [1]: [cs_item_sk#17] Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] +Aggregate Attributes [3]: [sum#27, sum#28, isEmpty#29] +Results [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] -(24) CometExchange -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(27) Exchange +Input [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] +Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(25) CometHashAggregate -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] +(28) HashAggregate [codegen id : 8] +Input [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] Keys [1]: [cs_item_sk#17] Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#19))#33, sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))#34] +Results [3]: [cs_item_sk#17, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#19))#33,17,2) AS sale#35, sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))#34 AS refund#36] -(26) CometFilter -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) +(29) Filter [codegen id : 8] +Input [3]: [cs_item_sk#17, sale#35, refund#36] +Condition : ((isnotnull(sale#35) AND isnotnull(refund#36)) AND (cast(sale#35 as decimal(21,2)) > (2 * refund#36))) -(27) CometProject -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Arguments: [cs_item_sk#17], [cs_item_sk#17] +(30) Project [codegen id : 8] +Output [1]: [cs_item_sk#17] +Input [3]: [cs_item_sk#17, sale#35, refund#36] -(28) CometSort +(31) Sort [codegen id : 8] Input [1]: [cs_item_sk#17] -Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] +Arguments: [cs_item_sk#17 ASC NULLS FIRST], false, 0 -(29) CometSortMergeJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [1]: [cs_item_sk#17] -Arguments: [ss_item_sk#1], [cs_item_sk#17], Inner +(32) SortMergeJoin [codegen id : 24] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [cs_item_sk#17] +Join type: Inner +Join condition: None -(30) CometProject +(33) Project [codegen id : 24] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#17] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(31) ColumnarToRow [codegen id : 16] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -(32) ReusedExchange [Reuses operator id: 183] -Output [2]: [d_date_sk#32, d_year#33] +(34) ReusedExchange [Reuses operator id: 185] +Output [2]: [d_date_sk#37, d_year#38] -(33) BroadcastHashJoin [codegen id : 16] +(35) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#32] +Right keys [1]: [d_date_sk#37] Join type: Inner Join condition: None -(34) Project [codegen id : 16] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#32, d_year#33] +(36) Project [codegen id : 24] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38] +Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#37, d_year#38] (unknown) Scan parquet spark_catalog.default.store -Output [3]: [s_store_sk#34, s_store_name#35, s_zip#36] +Output [3]: [s_store_sk#39, s_store_name#40, s_zip#41] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_zip)] ReadSchema: struct -(36) CometFilter -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Condition : ((isnotnull(s_store_sk#34) AND isnotnull(s_store_name#35)) AND isnotnull(s_zip#36)) +(38) CometFilter +Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] +Condition : ((isnotnull(s_store_sk#39) AND isnotnull(s_store_name#40)) AND isnotnull(s_zip#41)) -(37) ColumnarToRow [codegen id : 2] -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] +(39) ColumnarToRow [codegen id : 10] +Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] -(38) BroadcastExchange -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] +(40) BroadcastExchange +Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(39) BroadcastHashJoin [codegen id : 16] +(41) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ss_store_sk#6] -Right keys [1]: [s_store_sk#34] +Right keys [1]: [s_store_sk#39] Join type: Inner Join condition: None -(40) Project [codegen id : 16] -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_sk#34, s_store_name#35, s_zip#36] +(42) Project [codegen id : 24] +Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41] +Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_sk#39, s_store_name#40, s_zip#41] (unknown) Scan parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +Output [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(42) CometFilter -Input [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] -Condition : (((((isnotnull(c_customer_sk#37) AND isnotnull(c_first_sales_date_sk#42)) AND isnotnull(c_first_shipto_date_sk#41)) AND isnotnull(c_current_cdemo_sk#38)) AND isnotnull(c_current_hdemo_sk#39)) AND isnotnull(c_current_addr_sk#40)) +(44) CometFilter +Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] +Condition : (((((isnotnull(c_customer_sk#42) AND isnotnull(c_first_sales_date_sk#47)) AND isnotnull(c_first_shipto_date_sk#46)) AND isnotnull(c_current_cdemo_sk#43)) AND isnotnull(c_current_hdemo_sk#44)) AND isnotnull(c_current_addr_sk#45)) -(43) ColumnarToRow [codegen id : 3] -Input [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +(45) ColumnarToRow [codegen id : 11] +Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] -(44) BroadcastExchange -Input [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +(46) BroadcastExchange +Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(45) BroadcastHashJoin [codegen id : 16] +(47) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#37] +Right keys [1]: [c_customer_sk#42] Join type: Inner Join condition: None -(46) Project [codegen id : 16] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +(48) Project [codegen id : 24] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] +Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_year#44] +Output [2]: [d_date_sk#48, d_year#49] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct -(48) CometFilter -Input [2]: [d_date_sk#43, d_year#44] -Condition : isnotnull(d_date_sk#43) +(50) CometFilter +Input [2]: [d_date_sk#48, d_year#49] +Condition : isnotnull(d_date_sk#48) -(49) ColumnarToRow [codegen id : 4] -Input [2]: [d_date_sk#43, d_year#44] +(51) ColumnarToRow [codegen id : 12] +Input [2]: [d_date_sk#48, d_year#49] -(50) BroadcastExchange -Input [2]: [d_date_sk#43, d_year#44] +(52) BroadcastExchange +Input [2]: [d_date_sk#48, d_year#49] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(51) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [c_first_sales_date_sk#42] -Right keys [1]: [d_date_sk#43] +(53) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [c_first_sales_date_sk#47] +Right keys [1]: [d_date_sk#48] Join type: Inner Join condition: None -(52) Project [codegen id : 16] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, d_year#44] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42, d_date_sk#43, d_year#44] +(54) Project [codegen id : 24] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, d_year#49] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47, d_date_sk#48, d_year#49] -(53) ReusedExchange [Reuses operator id: 50] -Output [2]: [d_date_sk#45, d_year#46] +(55) ReusedExchange [Reuses operator id: 52] +Output [2]: [d_date_sk#50, d_year#51] -(54) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [c_first_shipto_date_sk#41] -Right keys [1]: [d_date_sk#45] +(56) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [c_first_shipto_date_sk#46] +Right keys [1]: [d_date_sk#50] Join type: Inner Join condition: None -(55) Project [codegen id : 16] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, d_year#44, d_date_sk#45, d_year#46] +(57) Project [codegen id : 24] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, d_year#49, d_date_sk#50, d_year#51] (unknown) Scan parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#47, cd_marital_status#48] +Output [2]: [cd_demo_sk#52, cd_marital_status#53] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status)] ReadSchema: struct -(57) CometFilter -Input [2]: [cd_demo_sk#47, cd_marital_status#48] -Condition : (isnotnull(cd_demo_sk#47) AND isnotnull(cd_marital_status#48)) +(59) CometFilter +Input [2]: [cd_demo_sk#52, cd_marital_status#53] +Condition : (isnotnull(cd_demo_sk#52) AND isnotnull(cd_marital_status#53)) -(58) ColumnarToRow [codegen id : 6] -Input [2]: [cd_demo_sk#47, cd_marital_status#48] +(60) ColumnarToRow [codegen id : 14] +Input [2]: [cd_demo_sk#52, cd_marital_status#53] -(59) BroadcastExchange -Input [2]: [cd_demo_sk#47, cd_marital_status#48] +(61) BroadcastExchange +Input [2]: [cd_demo_sk#52, cd_marital_status#53] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(60) BroadcastHashJoin [codegen id : 16] +(62) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ss_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#47] +Right keys [1]: [cd_demo_sk#52] Join type: Inner Join condition: None -(61) Project [codegen id : 16] -Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_marital_status#48] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_demo_sk#47, cd_marital_status#48] +(63) Project [codegen id : 24] +Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_marital_status#53] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_demo_sk#52, cd_marital_status#53] -(62) ReusedExchange [Reuses operator id: 59] -Output [2]: [cd_demo_sk#49, cd_marital_status#50] +(64) ReusedExchange [Reuses operator id: 61] +Output [2]: [cd_demo_sk#54, cd_marital_status#55] -(63) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [c_current_cdemo_sk#38] -Right keys [1]: [cd_demo_sk#49] +(65) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [c_current_cdemo_sk#43] +Right keys [1]: [cd_demo_sk#54] Join type: Inner -Join condition: NOT (cd_marital_status#48 = cd_marital_status#50) +Join condition: NOT (cd_marital_status#53 = cd_marital_status#55) -(64) Project [codegen id : 16] -Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_marital_status#48, cd_demo_sk#49, cd_marital_status#50] +(66) Project [codegen id : 24] +Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] +Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_marital_status#53, cd_demo_sk#54, cd_marital_status#55] (unknown) Scan parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#51] +Output [1]: [p_promo_sk#56] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_promo_sk)] ReadSchema: struct -(66) CometFilter -Input [1]: [p_promo_sk#51] -Condition : isnotnull(p_promo_sk#51) +(68) CometFilter +Input [1]: [p_promo_sk#56] +Condition : isnotnull(p_promo_sk#56) -(67) ColumnarToRow [codegen id : 8] -Input [1]: [p_promo_sk#51] +(69) ColumnarToRow [codegen id : 16] +Input [1]: [p_promo_sk#56] -(68) BroadcastExchange -Input [1]: [p_promo_sk#51] +(70) BroadcastExchange +Input [1]: [p_promo_sk#56] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(69) BroadcastHashJoin [codegen id : 16] +(71) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ss_promo_sk#7] -Right keys [1]: [p_promo_sk#51] +Right keys [1]: [p_promo_sk#56] Join type: Inner Join condition: None -(70) Project [codegen id : 16] -Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, p_promo_sk#51] +(72) Project [codegen id : 24] +Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, p_promo_sk#56] (unknown) Scan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#52, hd_income_band_sk#53] +Output [2]: [hd_demo_sk#57, hd_income_band_sk#58] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] ReadSchema: struct -(72) CometFilter -Input [2]: [hd_demo_sk#52, hd_income_band_sk#53] -Condition : (isnotnull(hd_demo_sk#52) AND isnotnull(hd_income_band_sk#53)) +(74) CometFilter +Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] +Condition : (isnotnull(hd_demo_sk#57) AND isnotnull(hd_income_band_sk#58)) -(73) ColumnarToRow [codegen id : 9] -Input [2]: [hd_demo_sk#52, hd_income_band_sk#53] +(75) ColumnarToRow [codegen id : 17] +Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] -(74) BroadcastExchange -Input [2]: [hd_demo_sk#52, hd_income_band_sk#53] +(76) BroadcastExchange +Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -(75) BroadcastHashJoin [codegen id : 16] +(77) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ss_hdemo_sk#4] -Right keys [1]: [hd_demo_sk#52] +Right keys [1]: [hd_demo_sk#57] Join type: Inner Join condition: None -(76) Project [codegen id : 16] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_demo_sk#52, hd_income_band_sk#53] +(78) Project [codegen id : 24] +Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58] +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_demo_sk#57, hd_income_band_sk#58] -(77) ReusedExchange [Reuses operator id: 74] -Output [2]: [hd_demo_sk#54, hd_income_band_sk#55] +(79) ReusedExchange [Reuses operator id: 76] +Output [2]: [hd_demo_sk#59, hd_income_band_sk#60] -(78) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [c_current_hdemo_sk#39] -Right keys [1]: [hd_demo_sk#54] +(80) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [c_current_hdemo_sk#44] +Right keys [1]: [hd_demo_sk#59] Join type: Inner Join condition: None -(79) Project [codegen id : 16] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55] -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_demo_sk#54, hd_income_band_sk#55] +(81) Project [codegen id : 24] +Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60] +Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_demo_sk#59, hd_income_band_sk#60] (unknown) Scan parquet spark_catalog.default.customer_address -Output [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +Output [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(81) CometFilter -Input [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] -Condition : isnotnull(ca_address_sk#56) +(83) CometFilter +Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Condition : isnotnull(ca_address_sk#61) -(82) ColumnarToRow [codegen id : 11] -Input [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +(84) ColumnarToRow [codegen id : 19] +Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -(83) BroadcastExchange -Input [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +(85) BroadcastExchange +Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=11] -(84) BroadcastHashJoin [codegen id : 16] +(86) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ss_addr_sk#5] -Right keys [1]: [ca_address_sk#56] +Right keys [1]: [ca_address_sk#61] Join type: Inner Join condition: None -(85) Project [codegen id : 16] -Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +(87) Project [codegen id : 24] +Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -(86) ReusedExchange [Reuses operator id: 83] -Output [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +(88) ReusedExchange [Reuses operator id: 85] +Output [5]: [ca_address_sk#66, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] -(87) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [c_current_addr_sk#40] -Right keys [1]: [ca_address_sk#61] +(89) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [c_current_addr_sk#45] +Right keys [1]: [ca_address_sk#66] Join type: Inner Join condition: None -(88) Project [codegen id : 16] -Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +(90) Project [codegen id : 24] +Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] +Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_address_sk#66, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] (unknown) Scan parquet spark_catalog.default.income_band -Output [1]: [ib_income_band_sk#66] +Output [1]: [ib_income_band_sk#71] Batched: true Location [not included in comparison]/{warehouse_dir}/income_band] PushedFilters: [IsNotNull(ib_income_band_sk)] ReadSchema: struct -(90) CometFilter -Input [1]: [ib_income_band_sk#66] -Condition : isnotnull(ib_income_band_sk#66) +(92) CometFilter +Input [1]: [ib_income_band_sk#71] +Condition : isnotnull(ib_income_band_sk#71) -(91) ColumnarToRow [codegen id : 13] -Input [1]: [ib_income_band_sk#66] +(93) ColumnarToRow [codegen id : 21] +Input [1]: [ib_income_band_sk#71] -(92) BroadcastExchange -Input [1]: [ib_income_band_sk#66] +(94) BroadcastExchange +Input [1]: [ib_income_band_sk#71] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -(93) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [hd_income_band_sk#53] -Right keys [1]: [ib_income_band_sk#66] +(95) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [hd_income_band_sk#58] +Right keys [1]: [ib_income_band_sk#71] Join type: Inner Join condition: None -(94) Project [codegen id : 16] -Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ib_income_band_sk#66] +(96) Project [codegen id : 24] +Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] +Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, ib_income_band_sk#71] -(95) ReusedExchange [Reuses operator id: 92] -Output [1]: [ib_income_band_sk#67] +(97) ReusedExchange [Reuses operator id: 94] +Output [1]: [ib_income_band_sk#72] -(96) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [hd_income_band_sk#55] -Right keys [1]: [ib_income_band_sk#67] +(98) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [hd_income_band_sk#60] +Right keys [1]: [ib_income_band_sk#72] Join type: Inner Join condition: None -(97) Project [codegen id : 16] -Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ib_income_band_sk#67] +(99) Project [codegen id : 24] +Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, ib_income_band_sk#72] (unknown) Scan parquet spark_catalog.default.item -Output [4]: [i_item_sk#68, i_current_price#69, i_color#70, i_product_name#71] +Output [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), In(i_color, [burlywood ,floral ,indian ,medium ,purple ,spring ]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] ReadSchema: struct -(99) CometFilter -Input [4]: [i_item_sk#68, i_current_price#69, i_color#70, i_product_name#71] -Condition : ((((((isnotnull(i_current_price#69) AND i_color#70 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#69 >= 64.00)) AND (i_current_price#69 <= 74.00)) AND (i_current_price#69 >= 65.00)) AND (i_current_price#69 <= 79.00)) AND isnotnull(i_item_sk#68)) +(101) CometFilter +Input [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] +Condition : ((((((isnotnull(i_current_price#74) AND i_color#75 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#74 >= 64.00)) AND (i_current_price#74 <= 74.00)) AND (i_current_price#74 >= 65.00)) AND (i_current_price#74 <= 79.00)) AND isnotnull(i_item_sk#73)) -(100) CometProject -Input [4]: [i_item_sk#68, i_current_price#69, i_color#70, i_product_name#71] -Arguments: [i_item_sk#68, i_product_name#71], [i_item_sk#68, i_product_name#71] +(102) CometProject +Input [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] +Arguments: [i_item_sk#73, i_product_name#76], [i_item_sk#73, i_product_name#76] -(101) ColumnarToRow [codegen id : 15] -Input [2]: [i_item_sk#68, i_product_name#71] +(103) ColumnarToRow [codegen id : 23] +Input [2]: [i_item_sk#73, i_product_name#76] -(102) BroadcastExchange -Input [2]: [i_item_sk#68, i_product_name#71] +(104) BroadcastExchange +Input [2]: [i_item_sk#73, i_product_name#76] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -(103) BroadcastHashJoin [codegen id : 16] +(105) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#68] +Right keys [1]: [i_item_sk#73] Join type: Inner Join condition: None -(104) Project [codegen id : 16] -Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#44, d_year#46, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, i_item_sk#68, i_product_name#71] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, i_item_sk#68, i_product_name#71] +(106) Project [codegen id : 24] +Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, d_year#49, d_year#51, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] -(105) HashAggregate [codegen id : 16] -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#44, d_year#46, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, i_item_sk#68, i_product_name#71] -Keys [15]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46] +(107) HashAggregate [codegen id : 24] +Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, d_year#49, d_year#51, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] +Keys [15]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51] Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count#72, sum#73, sum#74, sum#75] -Results [19]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46, count#76, sum#77, sum#78, sum#79] +Aggregate Attributes [4]: [count#77, sum#78, sum#79, sum#80] +Results [19]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51, count#81, sum#82, sum#83, sum#84] -(106) HashAggregate [codegen id : 16] -Input [19]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46, count#76, sum#77, sum#78, sum#79] -Keys [15]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46] +(108) HashAggregate [codegen id : 24] +Input [19]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51, count#81, sum#82, sum#83, sum#84] +Keys [15]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51] Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count(1)#80, sum(UnscaledValue(ss_wholesale_cost#9))#81, sum(UnscaledValue(ss_list_price#10))#82, sum(UnscaledValue(ss_coupon_amt#11))#83] -Results [17]: [i_product_name#71 AS product_name#84, i_item_sk#68 AS item_sk#85, s_store_name#35 AS store_name#86, s_zip#36 AS store_zip#87, ca_street_number#57 AS b_street_number#88, ca_street_name#58 AS b_streen_name#89, ca_city#59 AS b_city#90, ca_zip#60 AS b_zip#91, ca_street_number#62 AS c_street_number#92, ca_street_name#63 AS c_street_name#93, ca_city#64 AS c_city#94, ca_zip#65 AS c_zip#95, d_year#33 AS syear#96, count(1)#80 AS cnt#97, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#81,17,2) AS s1#98, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#82,17,2) AS s2#99, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#83,17,2) AS s3#100] +Aggregate Attributes [4]: [count(1)#85, sum(UnscaledValue(ss_wholesale_cost#9))#86, sum(UnscaledValue(ss_list_price#10))#87, sum(UnscaledValue(ss_coupon_amt#11))#88] +Results [17]: [i_product_name#76 AS product_name#89, i_item_sk#73 AS item_sk#90, s_store_name#40 AS store_name#91, s_zip#41 AS store_zip#92, ca_street_number#62 AS b_street_number#93, ca_street_name#63 AS b_streen_name#94, ca_city#64 AS b_city#95, ca_zip#65 AS b_zip#96, ca_street_number#67 AS c_street_number#97, ca_street_name#68 AS c_street_name#98, ca_city#69 AS c_city#99, ca_zip#70 AS c_zip#100, d_year#38 AS syear#101, count(1)#85 AS cnt#102, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#86,17,2) AS s1#103, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#87,17,2) AS s2#104, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#88,17,2) AS s3#105] -(107) Exchange -Input [17]: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100] -Arguments: hashpartitioning(item_sk#85, store_name#86, store_zip#87, 5), ENSURE_REQUIREMENTS, [plan_id=14] +(109) Exchange +Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] +Arguments: hashpartitioning(item_sk#90, store_name#91, store_zip#92, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(108) Sort [codegen id : 17] -Input [17]: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100] -Arguments: [item_sk#85 ASC NULLS FIRST, store_name#86 ASC NULLS FIRST, store_zip#87 ASC NULLS FIRST], false, 0 +(110) Sort [codegen id : 25] +Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] +Arguments: [item_sk#90 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, store_zip#92 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] +Output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#112), dynamicpruningexpression(ss_sold_date_sk#112 IN dynamicpruning#113)] +PartitionFilters: [isnotnull(ss_sold_date_sk#117), dynamicpruningexpression(ss_sold_date_sk#117 IN dynamicpruning#118)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct -(110) CometFilter -Input [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -Condition : (((((((isnotnull(ss_item_sk#101) AND isnotnull(ss_ticket_number#108)) AND isnotnull(ss_store_sk#106)) AND isnotnull(ss_customer_sk#102)) AND isnotnull(ss_cdemo_sk#103)) AND isnotnull(ss_promo_sk#107)) AND isnotnull(ss_hdemo_sk#104)) AND isnotnull(ss_addr_sk#105)) +(112) CometFilter +Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Condition : (((((((isnotnull(ss_item_sk#106) AND isnotnull(ss_ticket_number#113)) AND isnotnull(ss_store_sk#111)) AND isnotnull(ss_customer_sk#107)) AND isnotnull(ss_cdemo_sk#108)) AND isnotnull(ss_promo_sk#112)) AND isnotnull(ss_hdemo_sk#109)) AND isnotnull(ss_addr_sk#110)) -(111) CometBroadcastExchange -Input [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -Arguments: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] +(113) CometBroadcastExchange +Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Arguments: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] (unknown) Scan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#114, sr_ticket_number#115, sr_returned_date_sk#116] +Output [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] ReadSchema: struct -(113) CometFilter -Input [3]: [sr_item_sk#114, sr_ticket_number#115, sr_returned_date_sk#116] -Condition : (isnotnull(sr_item_sk#114) AND isnotnull(sr_ticket_number#115)) - -(114) CometProject -Input [3]: [sr_item_sk#114, sr_ticket_number#115, sr_returned_date_sk#116] -Arguments: [sr_item_sk#114, sr_ticket_number#115], [sr_item_sk#114, sr_ticket_number#115] - -(115) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -Right output [2]: [sr_item_sk#114, sr_ticket_number#115] -Arguments: [ss_item_sk#101, ss_ticket_number#108], [sr_item_sk#114, sr_ticket_number#115], Inner +(115) CometFilter +Input [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] +Condition : (isnotnull(sr_item_sk#119) AND isnotnull(sr_ticket_number#120)) (116) CometProject -Input [14]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112, sr_item_sk#114, sr_ticket_number#115] -Arguments: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112], [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] - -(117) CometExchange -Input [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -Arguments: hashpartitioning(ss_item_sk#101, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] - -(118) CometSort -Input [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -Arguments: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112], [ss_item_sk#101 ASC NULLS FIRST] - -(119) ReusedExchange [Reuses operator id: 24] -Output [4]: [cs_item_sk#117, sum#118, sum#119, isEmpty#120] - -(120) CometHashAggregate -Input [4]: [cs_item_sk#117, sum#118, sum#119, isEmpty#120] -Keys [1]: [cs_item_sk#117] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#121)), sum(((cr_refunded_cash#122 + cr_reversed_charge#123) + cr_store_credit#124))] - -(121) CometFilter -Input [3]: [cs_item_sk#117, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) - -(122) CometProject -Input [3]: [cs_item_sk#117, sale#30, refund#31] -Arguments: [cs_item_sk#117], [cs_item_sk#117] - -(123) CometSort -Input [1]: [cs_item_sk#117] -Arguments: [cs_item_sk#117], [cs_item_sk#117 ASC NULLS FIRST] - -(124) CometSortMergeJoin -Left output [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -Right output [1]: [cs_item_sk#117] -Arguments: [ss_item_sk#101], [cs_item_sk#117], Inner - -(125) CometProject -Input [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112, cs_item_sk#117] -Arguments: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112], [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] +Input [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] +Arguments: [sr_item_sk#119, sr_ticket_number#120], [sr_item_sk#119, sr_ticket_number#120] + +(117) CometBroadcastHashJoin +Left output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Right output [2]: [sr_item_sk#119, sr_ticket_number#120] +Arguments: [ss_item_sk#106, ss_ticket_number#113], [sr_item_sk#119, sr_ticket_number#120], Inner + +(118) CometProject +Input [14]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, sr_item_sk#119, sr_ticket_number#120] +Arguments: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117], [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] + +(119) ColumnarToRow [codegen id : 26] +Input [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] + +(120) Exchange +Input [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Arguments: hashpartitioning(ss_item_sk#106, 5), ENSURE_REQUIREMENTS, [plan_id=15] + +(121) Sort [codegen id : 27] +Input [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Arguments: [ss_item_sk#106 ASC NULLS FIRST], false, 0 + +(122) ReusedExchange [Reuses operator id: 27] +Output [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] + +(123) HashAggregate [codegen id : 33] +Input [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] +Keys [1]: [cs_item_sk#122] +Functions [2]: [sum(UnscaledValue(cs_ext_list_price#126)), sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#126))#33, sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))#34] +Results [3]: [cs_item_sk#122, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#126))#33,17,2) AS sale#35, sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))#34 AS refund#36] + +(124) Filter [codegen id : 33] +Input [3]: [cs_item_sk#122, sale#35, refund#36] +Condition : ((isnotnull(sale#35) AND isnotnull(refund#36)) AND (cast(sale#35 as decimal(21,2)) > (2 * refund#36))) + +(125) Project [codegen id : 33] +Output [1]: [cs_item_sk#122] +Input [3]: [cs_item_sk#122, sale#35, refund#36] + +(126) Sort [codegen id : 33] +Input [1]: [cs_item_sk#122] +Arguments: [cs_item_sk#122 ASC NULLS FIRST], false, 0 + +(127) SortMergeJoin [codegen id : 49] +Left keys [1]: [ss_item_sk#106] +Right keys [1]: [cs_item_sk#122] +Join type: Inner +Join condition: None -(126) ColumnarToRow [codegen id : 33] -Input [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] +(128) Project [codegen id : 49] +Output [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, cs_item_sk#122] -(127) ReusedExchange [Reuses operator id: 187] -Output [2]: [d_date_sk#125, d_year#126] +(129) ReusedExchange [Reuses operator id: 189] +Output [2]: [d_date_sk#130, d_year#131] -(128) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_sold_date_sk#112] -Right keys [1]: [d_date_sk#125] +(130) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_sold_date_sk#117] +Right keys [1]: [d_date_sk#130] Join type: Inner Join condition: None -(129) Project [codegen id : 33] -Output [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126] -Input [13]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112, d_date_sk#125, d_year#126] +(131) Project [codegen id : 49] +Output [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131] +Input [13]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, d_date_sk#130, d_year#131] -(130) ReusedExchange [Reuses operator id: 38] -Output [3]: [s_store_sk#127, s_store_name#128, s_zip#129] +(132) ReusedExchange [Reuses operator id: 40] +Output [3]: [s_store_sk#132, s_store_name#133, s_zip#134] -(131) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_store_sk#106] -Right keys [1]: [s_store_sk#127] +(133) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_store_sk#111] +Right keys [1]: [s_store_sk#132] Join type: Inner Join condition: None -(132) Project [codegen id : 33] -Output [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129] -Input [14]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_sk#127, s_store_name#128, s_zip#129] +(134) Project [codegen id : 49] +Output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134] +Input [14]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_sk#132, s_store_name#133, s_zip#134] -(133) ReusedExchange [Reuses operator id: 44] -Output [6]: [c_customer_sk#130, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, c_first_sales_date_sk#135] +(135) ReusedExchange [Reuses operator id: 46] +Output [6]: [c_customer_sk#135, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, c_first_sales_date_sk#140] -(134) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_customer_sk#102] -Right keys [1]: [c_customer_sk#130] +(136) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_customer_sk#107] +Right keys [1]: [c_customer_sk#135] Join type: Inner Join condition: None -(135) Project [codegen id : 33] -Output [16]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, c_first_sales_date_sk#135] -Input [18]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_customer_sk#130, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, c_first_sales_date_sk#135] +(137) Project [codegen id : 49] +Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, c_first_sales_date_sk#140] +Input [18]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_customer_sk#135, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, c_first_sales_date_sk#140] -(136) ReusedExchange [Reuses operator id: 50] -Output [2]: [d_date_sk#136, d_year#137] +(138) ReusedExchange [Reuses operator id: 52] +Output [2]: [d_date_sk#141, d_year#142] -(137) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [c_first_sales_date_sk#135] -Right keys [1]: [d_date_sk#136] +(139) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [c_first_sales_date_sk#140] +Right keys [1]: [d_date_sk#141] Join type: Inner Join condition: None -(138) Project [codegen id : 33] -Output [16]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, d_year#137] -Input [18]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, c_first_sales_date_sk#135, d_date_sk#136, d_year#137] +(140) Project [codegen id : 49] +Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, d_year#142] +Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, c_first_sales_date_sk#140, d_date_sk#141, d_year#142] -(139) ReusedExchange [Reuses operator id: 50] -Output [2]: [d_date_sk#138, d_year#139] +(141) ReusedExchange [Reuses operator id: 52] +Output [2]: [d_date_sk#143, d_year#144] -(140) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [c_first_shipto_date_sk#134] -Right keys [1]: [d_date_sk#138] +(142) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [c_first_shipto_date_sk#139] +Right keys [1]: [d_date_sk#143] Join type: Inner Join condition: None -(141) Project [codegen id : 33] -Output [16]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139] -Input [18]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, d_year#137, d_date_sk#138, d_year#139] +(143) Project [codegen id : 49] +Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144] +Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, d_year#142, d_date_sk#143, d_year#144] -(142) ReusedExchange [Reuses operator id: 59] -Output [2]: [cd_demo_sk#140, cd_marital_status#141] +(144) ReusedExchange [Reuses operator id: 61] +Output [2]: [cd_demo_sk#145, cd_marital_status#146] -(143) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_cdemo_sk#103] -Right keys [1]: [cd_demo_sk#140] +(145) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_cdemo_sk#108] +Right keys [1]: [cd_demo_sk#145] Join type: Inner Join condition: None -(144) Project [codegen id : 33] -Output [16]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, cd_marital_status#141] -Input [18]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, cd_demo_sk#140, cd_marital_status#141] +(146) Project [codegen id : 49] +Output [16]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, cd_marital_status#146] +Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, cd_demo_sk#145, cd_marital_status#146] -(145) ReusedExchange [Reuses operator id: 59] -Output [2]: [cd_demo_sk#142, cd_marital_status#143] +(147) ReusedExchange [Reuses operator id: 61] +Output [2]: [cd_demo_sk#147, cd_marital_status#148] -(146) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [c_current_cdemo_sk#131] -Right keys [1]: [cd_demo_sk#142] +(148) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [c_current_cdemo_sk#136] +Right keys [1]: [cd_demo_sk#147] Join type: Inner -Join condition: NOT (cd_marital_status#141 = cd_marital_status#143) +Join condition: NOT (cd_marital_status#146 = cd_marital_status#148) -(147) Project [codegen id : 33] -Output [14]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139] -Input [18]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, cd_marital_status#141, cd_demo_sk#142, cd_marital_status#143] +(149) Project [codegen id : 49] +Output [14]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144] +Input [18]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, cd_marital_status#146, cd_demo_sk#147, cd_marital_status#148] -(148) ReusedExchange [Reuses operator id: 68] -Output [1]: [p_promo_sk#144] +(150) ReusedExchange [Reuses operator id: 70] +Output [1]: [p_promo_sk#149] -(149) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_promo_sk#107] -Right keys [1]: [p_promo_sk#144] +(151) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_promo_sk#112] +Right keys [1]: [p_promo_sk#149] Join type: Inner Join condition: None -(150) Project [codegen id : 33] -Output [13]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139] -Input [15]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, p_promo_sk#144] +(152) Project [codegen id : 49] +Output [13]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144] +Input [15]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, p_promo_sk#149] -(151) ReusedExchange [Reuses operator id: 74] -Output [2]: [hd_demo_sk#145, hd_income_band_sk#146] +(153) ReusedExchange [Reuses operator id: 76] +Output [2]: [hd_demo_sk#150, hd_income_band_sk#151] -(152) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_hdemo_sk#104] -Right keys [1]: [hd_demo_sk#145] +(154) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_hdemo_sk#109] +Right keys [1]: [hd_demo_sk#150] Join type: Inner Join condition: None -(153) Project [codegen id : 33] -Output [13]: [ss_item_sk#101, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146] -Input [15]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, hd_demo_sk#145, hd_income_band_sk#146] +(155) Project [codegen id : 49] +Output [13]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151] +Input [15]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, hd_demo_sk#150, hd_income_band_sk#151] -(154) ReusedExchange [Reuses operator id: 74] -Output [2]: [hd_demo_sk#147, hd_income_band_sk#148] +(156) ReusedExchange [Reuses operator id: 76] +Output [2]: [hd_demo_sk#152, hd_income_band_sk#153] -(155) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [c_current_hdemo_sk#132] -Right keys [1]: [hd_demo_sk#147] +(157) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [c_current_hdemo_sk#137] +Right keys [1]: [hd_demo_sk#152] Join type: Inner Join condition: None -(156) Project [codegen id : 33] -Output [13]: [ss_item_sk#101, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148] -Input [15]: [ss_item_sk#101, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_demo_sk#147, hd_income_band_sk#148] +(158) Project [codegen id : 49] +Output [13]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153] +Input [15]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_demo_sk#152, hd_income_band_sk#153] -(157) ReusedExchange [Reuses operator id: 83] -Output [5]: [ca_address_sk#149, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153] +(159) ReusedExchange [Reuses operator id: 85] +Output [5]: [ca_address_sk#154, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] -(158) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_addr_sk#105] -Right keys [1]: [ca_address_sk#149] +(160) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_addr_sk#110] +Right keys [1]: [ca_address_sk#154] Join type: Inner Join condition: None -(159) Project [codegen id : 33] -Output [16]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153] -Input [18]: [ss_item_sk#101, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_address_sk#149, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153] +(161) Project [codegen id : 49] +Output [16]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] +Input [18]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_address_sk#154, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] -(160) ReusedExchange [Reuses operator id: 83] -Output [5]: [ca_address_sk#154, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] +(162) ReusedExchange [Reuses operator id: 85] +Output [5]: [ca_address_sk#159, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] -(161) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [c_current_addr_sk#133] -Right keys [1]: [ca_address_sk#154] +(163) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [c_current_addr_sk#138] +Right keys [1]: [ca_address_sk#159] Join type: Inner Join condition: None -(162) Project [codegen id : 33] -Output [19]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] -Input [21]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_address_sk#154, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] +(164) Project [codegen id : 49] +Output [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] +Input [21]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_address_sk#159, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] -(163) ReusedExchange [Reuses operator id: 92] -Output [1]: [ib_income_band_sk#159] +(165) ReusedExchange [Reuses operator id: 94] +Output [1]: [ib_income_band_sk#164] -(164) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [hd_income_band_sk#146] -Right keys [1]: [ib_income_band_sk#159] +(166) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [hd_income_band_sk#151] +Right keys [1]: [ib_income_band_sk#164] Join type: Inner Join condition: None -(165) Project [codegen id : 33] -Output [18]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] -Input [20]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ib_income_band_sk#159] +(167) Project [codegen id : 49] +Output [18]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] +Input [20]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, ib_income_band_sk#164] -(166) ReusedExchange [Reuses operator id: 92] -Output [1]: [ib_income_band_sk#160] +(168) ReusedExchange [Reuses operator id: 94] +Output [1]: [ib_income_band_sk#165] -(167) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [hd_income_band_sk#148] -Right keys [1]: [ib_income_band_sk#160] +(169) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [hd_income_band_sk#153] +Right keys [1]: [ib_income_band_sk#165] Join type: Inner Join condition: None -(168) Project [codegen id : 33] -Output [17]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] -Input [19]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ib_income_band_sk#160] +(170) Project [codegen id : 49] +Output [17]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] +Input [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, ib_income_band_sk#165] -(169) ReusedExchange [Reuses operator id: 102] -Output [2]: [i_item_sk#161, i_product_name#162] +(171) ReusedExchange [Reuses operator id: 104] +Output [2]: [i_item_sk#166, i_product_name#167] -(170) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_item_sk#101] -Right keys [1]: [i_item_sk#161] +(172) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_item_sk#106] +Right keys [1]: [i_item_sk#166] Join type: Inner Join condition: None -(171) Project [codegen id : 33] -Output [18]: [ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, d_year#137, d_year#139, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, i_item_sk#161, i_product_name#162] -Input [19]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, i_item_sk#161, i_product_name#162] - -(172) HashAggregate [codegen id : 33] -Input [18]: [ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, d_year#137, d_year#139, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, i_item_sk#161, i_product_name#162] -Keys [15]: [i_product_name#162, i_item_sk#161, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, d_year#126, d_year#137, d_year#139] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#109)), partial_sum(UnscaledValue(ss_list_price#110)), partial_sum(UnscaledValue(ss_coupon_amt#111))] -Aggregate Attributes [4]: [count#72, sum#163, sum#164, sum#165] -Results [19]: [i_product_name#162, i_item_sk#161, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, d_year#126, d_year#137, d_year#139, count#76, sum#166, sum#167, sum#168] - -(173) HashAggregate [codegen id : 33] -Input [19]: [i_product_name#162, i_item_sk#161, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, d_year#126, d_year#137, d_year#139, count#76, sum#166, sum#167, sum#168] -Keys [15]: [i_product_name#162, i_item_sk#161, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, d_year#126, d_year#137, d_year#139] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#109)), sum(UnscaledValue(ss_list_price#110)), sum(UnscaledValue(ss_coupon_amt#111))] -Aggregate Attributes [4]: [count(1)#80, sum(UnscaledValue(ss_wholesale_cost#109))#81, sum(UnscaledValue(ss_list_price#110))#82, sum(UnscaledValue(ss_coupon_amt#111))#83] -Results [8]: [i_item_sk#161 AS item_sk#169, s_store_name#128 AS store_name#170, s_zip#129 AS store_zip#171, d_year#126 AS syear#172, count(1)#80 AS cnt#173, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#109))#81,17,2) AS s1#174, MakeDecimal(sum(UnscaledValue(ss_list_price#110))#82,17,2) AS s2#175, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#111))#83,17,2) AS s3#176] - -(174) Exchange -Input [8]: [item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176] -Arguments: hashpartitioning(item_sk#169, store_name#170, store_zip#171, 5), ENSURE_REQUIREMENTS, [plan_id=16] - -(175) Sort [codegen id : 34] -Input [8]: [item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176] -Arguments: [item_sk#169 ASC NULLS FIRST, store_name#170 ASC NULLS FIRST, store_zip#171 ASC NULLS FIRST], false, 0 - -(176) SortMergeJoin [codegen id : 35] -Left keys [3]: [item_sk#85, store_name#86, store_zip#87] -Right keys [3]: [item_sk#169, store_name#170, store_zip#171] +(173) Project [codegen id : 49] +Output [18]: [ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, d_year#142, d_year#144, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, i_item_sk#166, i_product_name#167] +Input [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, i_item_sk#166, i_product_name#167] + +(174) HashAggregate [codegen id : 49] +Input [18]: [ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, d_year#142, d_year#144, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, i_item_sk#166, i_product_name#167] +Keys [15]: [i_product_name#167, i_item_sk#166, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, d_year#131, d_year#142, d_year#144] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#114)), partial_sum(UnscaledValue(ss_list_price#115)), partial_sum(UnscaledValue(ss_coupon_amt#116))] +Aggregate Attributes [4]: [count#77, sum#168, sum#169, sum#170] +Results [19]: [i_product_name#167, i_item_sk#166, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, d_year#131, d_year#142, d_year#144, count#81, sum#171, sum#172, sum#173] + +(175) HashAggregate [codegen id : 49] +Input [19]: [i_product_name#167, i_item_sk#166, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, d_year#131, d_year#142, d_year#144, count#81, sum#171, sum#172, sum#173] +Keys [15]: [i_product_name#167, i_item_sk#166, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, d_year#131, d_year#142, d_year#144] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#114)), sum(UnscaledValue(ss_list_price#115)), sum(UnscaledValue(ss_coupon_amt#116))] +Aggregate Attributes [4]: [count(1)#85, sum(UnscaledValue(ss_wholesale_cost#114))#86, sum(UnscaledValue(ss_list_price#115))#87, sum(UnscaledValue(ss_coupon_amt#116))#88] +Results [8]: [i_item_sk#166 AS item_sk#174, s_store_name#133 AS store_name#175, s_zip#134 AS store_zip#176, d_year#131 AS syear#177, count(1)#85 AS cnt#178, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#114))#86,17,2) AS s1#179, MakeDecimal(sum(UnscaledValue(ss_list_price#115))#87,17,2) AS s2#180, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#116))#88,17,2) AS s3#181] + +(176) Exchange +Input [8]: [item_sk#174, store_name#175, store_zip#176, syear#177, cnt#178, s1#179, s2#180, s3#181] +Arguments: hashpartitioning(item_sk#174, store_name#175, store_zip#176, 5), ENSURE_REQUIREMENTS, [plan_id=16] + +(177) Sort [codegen id : 50] +Input [8]: [item_sk#174, store_name#175, store_zip#176, syear#177, cnt#178, s1#179, s2#180, s3#181] +Arguments: [item_sk#174 ASC NULLS FIRST, store_name#175 ASC NULLS FIRST, store_zip#176 ASC NULLS FIRST], false, 0 + +(178) SortMergeJoin [codegen id : 51] +Left keys [3]: [item_sk#90, store_name#91, store_zip#92] +Right keys [3]: [item_sk#174, store_name#175, store_zip#176] Join type: Inner -Join condition: (cnt#173 <= cnt#97) +Join condition: (cnt#178 <= cnt#102) -(177) Project [codegen id : 35] -Output [21]: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173] -Input [25]: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176] +(179) Project [codegen id : 51] +Output [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#179, s2#180, s3#181, syear#177, cnt#178] +Input [25]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, item_sk#174, store_name#175, store_zip#176, syear#177, cnt#178, s1#179, s2#180, s3#181] -(178) Exchange -Input [21]: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173] -Arguments: rangepartitioning(product_name#84 ASC NULLS FIRST, store_name#86 ASC NULLS FIRST, cnt#173 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=17] +(180) Exchange +Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#179, s2#180, s3#181, syear#177, cnt#178] +Arguments: rangepartitioning(product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#178 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=17] -(179) Sort [codegen id : 36] -Input [21]: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173] -Arguments: [product_name#84 ASC NULLS FIRST, store_name#86 ASC NULLS FIRST, cnt#173 ASC NULLS FIRST], true, 0 +(181) Sort [codegen id : 52] +Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#179, s2#180, s3#181, syear#177, cnt#178] +Arguments: [product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#178 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (183) -+- * ColumnarToRow (182) - +- CometFilter (181) - +- CometScan parquet spark_catalog.default.date_dim (180) +BroadcastExchange (185) ++- * ColumnarToRow (184) + +- CometFilter (183) + +- CometScan parquet spark_catalog.default.date_dim (182) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] +Output [2]: [d_date_sk#37, d_year#38] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct -(181) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) +(183) CometFilter +Input [2]: [d_date_sk#37, d_year#38] +Condition : ((isnotnull(d_year#38) AND (d_year#38 = 1999)) AND isnotnull(d_date_sk#37)) -(182) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#32, d_year#33] +(184) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#37, d_year#38] -(183) BroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] +(185) BroadcastExchange +Input [2]: [d_date_sk#37, d_year#38] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] -Subquery:2 Hosting operator id = 109 Hosting Expression = ss_sold_date_sk#112 IN dynamicpruning#113 -BroadcastExchange (187) -+- * ColumnarToRow (186) - +- CometFilter (185) - +- CometScan parquet spark_catalog.default.date_dim (184) +Subquery:2 Hosting operator id = 111 Hosting Expression = ss_sold_date_sk#117 IN dynamicpruning#118 +BroadcastExchange (189) ++- * ColumnarToRow (188) + +- CometFilter (187) + +- CometScan parquet spark_catalog.default.date_dim (186) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#125, d_year#126] +Output [2]: [d_date_sk#130, d_year#131] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(185) CometFilter -Input [2]: [d_date_sk#125, d_year#126] -Condition : ((isnotnull(d_year#126) AND (d_year#126 = 2000)) AND isnotnull(d_date_sk#125)) +(187) CometFilter +Input [2]: [d_date_sk#130, d_year#131] +Condition : ((isnotnull(d_year#131) AND (d_year#131 = 2000)) AND isnotnull(d_date_sk#130)) -(186) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#125, d_year#126] +(188) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#130, d_year#131] -(187) BroadcastExchange -Input [2]: [d_date_sk#125, d_year#126] +(189) BroadcastExchange +Input [2]: [d_date_sk#130, d_year#131] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=19] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt index eda7db4e5..d972e0082 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt @@ -1,16 +1,16 @@ -WholeStageCodegen (36) +WholeStageCodegen (52) Sort [product_name,store_name,cnt] InputAdapter Exchange [product_name,store_name,cnt] #1 - WholeStageCodegen (35) + WholeStageCodegen (51) Project [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] SortMergeJoin [item_sk,store_name,store_zip,item_sk,store_name,store_zip,cnt,cnt] InputAdapter - WholeStageCodegen (17) + WholeStageCodegen (25) Sort [item_sk,store_name,store_zip] InputAdapter Exchange [item_sk,store_name,store_zip] #2 - WholeStageCodegen (16) + WholeStageCodegen (24) HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] @@ -43,64 +43,84 @@ WholeStageCodegen (36) BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometSortMergeJoin [ss_item_sk,cs_item_sk] - CometSort [ss_item_sk] - CometExchange [ss_item_sk] #3 - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #4 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [sale,refund] - CometHashAggregate [cs_item_sk,sum,sum,isEmpty] - CometExchange [cs_item_sk] #6 - CometHashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number] - CometExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SortMergeJoin [ss_item_sk,cs_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ss_item_sk] + InputAdapter + Exchange [ss_item_sk] #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + CometBroadcastExchange #4 + CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + WholeStageCodegen (8) + Sort [cs_item_sk] + Project [cs_item_sk] + Filter [sale,refund] + HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] + InputAdapter + Exchange [cs_item_sk] #6 + WholeStageCodegen (7) + HashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [sum,sum,isEmpty,sum,sum,isEmpty] + Project [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] + InputAdapter + WholeStageCodegen (4) + Sort [cs_item_sk,cs_order_number] + InputAdapter + Exchange [cs_item_sk,cs_order_number] #7 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] + CometFilter [cs_item_sk,cs_order_number] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + InputAdapter + WholeStageCodegen (6) + Sort [cr_item_sk,cr_order_number] + InputAdapter + Exchange [cr_item_sk,cr_order_number] #8 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometFilter [cr_item_sk,cr_order_number] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #5 InputAdapter BroadcastExchange #9 - WholeStageCodegen (2) + WholeStageCodegen (10) ColumnarToRow InputAdapter CometFilter [s_store_sk,s_store_name,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] InputAdapter BroadcastExchange #10 - WholeStageCodegen (3) + WholeStageCodegen (11) ColumnarToRow InputAdapter CometFilter [c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] InputAdapter BroadcastExchange #11 - WholeStageCodegen (4) + WholeStageCodegen (12) ColumnarToRow InputAdapter CometFilter [d_date_sk] @@ -109,7 +129,7 @@ WholeStageCodegen (36) ReusedExchange [d_date_sk,d_year] #11 InputAdapter BroadcastExchange #12 - WholeStageCodegen (6) + WholeStageCodegen (14) ColumnarToRow InputAdapter CometFilter [cd_demo_sk,cd_marital_status] @@ -118,14 +138,14 @@ WholeStageCodegen (36) ReusedExchange [cd_demo_sk,cd_marital_status] #12 InputAdapter BroadcastExchange #13 - WholeStageCodegen (8) + WholeStageCodegen (16) ColumnarToRow InputAdapter CometFilter [p_promo_sk] CometScan parquet spark_catalog.default.promotion [p_promo_sk] InputAdapter BroadcastExchange #14 - WholeStageCodegen (9) + WholeStageCodegen (17) ColumnarToRow InputAdapter CometFilter [hd_demo_sk,hd_income_band_sk] @@ -134,7 +154,7 @@ WholeStageCodegen (36) ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 InputAdapter BroadcastExchange #15 - WholeStageCodegen (11) + WholeStageCodegen (19) ColumnarToRow InputAdapter CometFilter [ca_address_sk] @@ -143,7 +163,7 @@ WholeStageCodegen (36) ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 InputAdapter BroadcastExchange #16 - WholeStageCodegen (13) + WholeStageCodegen (21) ColumnarToRow InputAdapter CometFilter [ib_income_band_sk] @@ -152,18 +172,18 @@ WholeStageCodegen (36) ReusedExchange [ib_income_band_sk] #16 InputAdapter BroadcastExchange #17 - WholeStageCodegen (15) + WholeStageCodegen (23) ColumnarToRow InputAdapter CometProject [i_item_sk,i_product_name] CometFilter [i_current_price,i_color,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] InputAdapter - WholeStageCodegen (34) + WholeStageCodegen (50) Sort [item_sk,store_name,store_zip] InputAdapter Exchange [item_sk,store_name,store_zip] #18 - WholeStageCodegen (33) + WholeStageCodegen (49) HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] @@ -196,32 +216,39 @@ WholeStageCodegen (36) BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometSortMergeJoin [ss_item_sk,cs_item_sk] - CometSort [ss_item_sk] - CometExchange [ss_item_sk] #19 - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #20 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #21 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [sale,refund] - CometHashAggregate [cs_item_sk,sum,sum,isEmpty] - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SortMergeJoin [ss_item_sk,cs_item_sk] + InputAdapter + WholeStageCodegen (27) + Sort [ss_item_sk] + InputAdapter + Exchange [ss_item_sk] #19 + WholeStageCodegen (26) + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + CometBroadcastExchange #20 + CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #21 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + WholeStageCodegen (33) + Sort [cs_item_sk] + Project [cs_item_sk] + Filter [sale,refund] + HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] + InputAdapter + ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 InputAdapter ReusedExchange [d_date_sk,d_year] #21 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt index c0d1f949a..e55a0ca10 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt @@ -63,9 +63,9 @@ TakeOrderedAndProject (70) : +- * ColumnarToRow (53) : +- CometFilter (52) : +- CometScan parquet spark_catalog.default.promotion (51) - +- * ColumnarToRow (64) - +- CometSort (63) - +- CometExchange (62) + +- * Sort (64) + +- Exchange (63) + +- * ColumnarToRow (62) +- CometProject (61) +- CometFilter (60) +- CometScan parquet spark_catalog.default.catalog_returns (59) @@ -355,28 +355,28 @@ Condition : (isnotnull(cr_item_sk#30) AND isnotnull(cr_order_number#31)) Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30, cr_order_number#31] -(62) CometExchange +(62) ColumnarToRow [codegen id : 12] Input [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: hashpartitioning(cr_item_sk#30, cr_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] -(63) CometSort +(63) Exchange Input [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30 ASC NULLS FIRST, cr_order_number#31 ASC NULLS FIRST] +Arguments: hashpartitioning(cr_item_sk#30, cr_order_number#31, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(64) ColumnarToRow [codegen id : 12] +(64) Sort [codegen id : 13] Input [2]: [cr_item_sk#30, cr_order_number#31] +Arguments: [cr_item_sk#30 ASC NULLS FIRST, cr_order_number#31 ASC NULLS FIRST], false, 0 -(65) SortMergeJoin [codegen id : 13] +(65) SortMergeJoin [codegen id : 14] Left keys [2]: [cs_item_sk#4, cs_order_number#6] Right keys [2]: [cr_item_sk#30, cr_order_number#31] Join type: LeftOuter Join condition: None -(66) Project [codegen id : 13] +(66) Project [codegen id : 14] Output [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#30, cr_order_number#31] -(67) HashAggregate [codegen id : 13] +(67) HashAggregate [codegen id : 14] Input [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] Functions [1]: [partial_count(1)] @@ -387,7 +387,7 @@ Results [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(69) HashAggregate [codegen id : 14] +(69) HashAggregate [codegen id : 15] Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] Functions [1]: [count(1)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt index 5eb8ea527..17fc9dee7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt @@ -1,9 +1,9 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo] - WholeStageCodegen (14) + WholeStageCodegen (15) HashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] [count(1),no_promo,promo,total_cnt,count] InputAdapter Exchange [i_item_desc,w_warehouse_name,d_week_seq] #1 - WholeStageCodegen (13) + WholeStageCodegen (14) HashAggregate [i_item_desc,w_warehouse_name,d_week_seq] [count,count] Project [w_warehouse_name,i_item_desc,d_week_seq] SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] @@ -104,11 +104,13 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom CometFilter [p_promo_sk] CometScan parquet spark_catalog.default.promotion [p_promo_sk] InputAdapter - WholeStageCodegen (12) - ColumnarToRow + WholeStageCodegen (13) + Sort [cr_item_sk,cr_order_number] InputAdapter - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] + Exchange [cr_item_sk,cr_order_number] #12 + WholeStageCodegen (12) + ColumnarToRow + InputAdapter + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt index 3a29db88e..0ff759f36 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -TakeOrderedAndProject (132) -+- * Project (131) - +- * SortMergeJoin Inner (130) +TakeOrderedAndProject (129) ++- * Project (128) + +- * SortMergeJoin Inner (127) :- * Sort (71) : +- Exchange (70) : +- * Filter (69) @@ -29,9 +29,9 @@ TakeOrderedAndProject (132) : : : : +- CometFilter (5) : : : : +- CometScan parquet spark_catalog.default.item (4) : : : +- ReusedExchange (11) - : : +- * ColumnarToRow (21) - : : +- CometSort (20) - : : +- CometExchange (19) + : : +- * Sort (21) + : : +- Exchange (20) + : : +- * ColumnarToRow (19) : : +- CometProject (18) : : +- CometFilter (17) : : +- CometScan parquet spark_catalog.default.catalog_returns (16) @@ -48,9 +48,9 @@ TakeOrderedAndProject (132) : : : : : +- CometScan parquet spark_catalog.default.store_sales (24) : : : : +- ReusedExchange (27) : : : +- ReusedExchange (30) - : : +- * ColumnarToRow (40) - : : +- CometSort (39) - : : +- CometExchange (38) + : : +- * Sort (40) + : : +- Exchange (39) + : : +- * ColumnarToRow (38) : : +- CometProject (37) : : +- CometFilter (36) : : +- CometScan parquet spark_catalog.default.store_returns (35) @@ -67,24 +67,24 @@ TakeOrderedAndProject (132) : : : : +- CometScan parquet spark_catalog.default.web_sales (43) : : : +- ReusedExchange (46) : : +- ReusedExchange (49) - : +- * ColumnarToRow (59) - : +- CometSort (58) - : +- CometExchange (57) + : +- * Sort (59) + : +- Exchange (58) + : +- * ColumnarToRow (57) : +- CometProject (56) : +- CometFilter (55) : +- CometScan parquet spark_catalog.default.web_returns (54) - +- * Sort (129) - +- Exchange (128) - +- * Filter (127) - +- * HashAggregate (126) - +- Exchange (125) - +- * HashAggregate (124) - +- * HashAggregate (123) - +- Exchange (122) - +- * HashAggregate (121) - +- Union (120) - :- * Project (87) - : +- * SortMergeJoin LeftOuter (86) + +- * Sort (126) + +- Exchange (125) + +- * Filter (124) + +- * HashAggregate (123) + +- Exchange (122) + +- * HashAggregate (121) + +- * HashAggregate (120) + +- Exchange (119) + +- * HashAggregate (118) + +- Union (117) + :- * Project (86) + : +- * SortMergeJoin LeftOuter (85) : :- * Sort (82) : : +- Exchange (81) : : +- * Project (80) @@ -96,41 +96,38 @@ TakeOrderedAndProject (132) : : : : +- CometScan parquet spark_catalog.default.catalog_sales (72) : : : +- ReusedExchange (75) : : +- ReusedExchange (78) - : +- * ColumnarToRow (85) - : +- CometSort (84) - : +- ReusedExchange (83) - :- * Project (103) - : +- * SortMergeJoin LeftOuter (102) - : :- * Sort (98) - : : +- Exchange (97) - : : +- * Project (96) - : : +- * BroadcastHashJoin Inner BuildRight (95) - : : :- * Project (93) - : : : +- * BroadcastHashJoin Inner BuildRight (92) - : : : :- * ColumnarToRow (90) - : : : : +- CometFilter (89) - : : : : +- CometScan parquet spark_catalog.default.store_sales (88) - : : : +- ReusedExchange (91) - : : +- ReusedExchange (94) - : +- * ColumnarToRow (101) - : +- CometSort (100) - : +- ReusedExchange (99) - +- * Project (119) - +- * SortMergeJoin LeftOuter (118) - :- * Sort (114) - : +- Exchange (113) - : +- * Project (112) - : +- * BroadcastHashJoin Inner BuildRight (111) - : :- * Project (109) - : : +- * BroadcastHashJoin Inner BuildRight (108) - : : :- * ColumnarToRow (106) - : : : +- CometFilter (105) - : : : +- CometScan parquet spark_catalog.default.web_sales (104) - : : +- ReusedExchange (107) - : +- ReusedExchange (110) - +- * ColumnarToRow (117) - +- CometSort (116) - +- ReusedExchange (115) + : +- * Sort (84) + : +- ReusedExchange (83) + :- * Project (101) + : +- * SortMergeJoin LeftOuter (100) + : :- * Sort (97) + : : +- Exchange (96) + : : +- * Project (95) + : : +- * BroadcastHashJoin Inner BuildRight (94) + : : :- * Project (92) + : : : +- * BroadcastHashJoin Inner BuildRight (91) + : : : :- * ColumnarToRow (89) + : : : : +- CometFilter (88) + : : : : +- CometScan parquet spark_catalog.default.store_sales (87) + : : : +- ReusedExchange (90) + : : +- ReusedExchange (93) + : +- * Sort (99) + : +- ReusedExchange (98) + +- * Project (116) + +- * SortMergeJoin LeftOuter (115) + :- * Sort (112) + : +- Exchange (111) + : +- * Project (110) + : +- * BroadcastHashJoin Inner BuildRight (109) + : :- * Project (107) + : : +- * BroadcastHashJoin Inner BuildRight (106) + : : :- * ColumnarToRow (104) + : : : +- CometFilter (103) + : : : +- CometScan parquet spark_catalog.default.web_sales (102) + : : +- ReusedExchange (105) + : +- ReusedExchange (108) + +- * Sort (114) + +- ReusedExchange (113) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -180,7 +177,7 @@ Join condition: None Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -(11) ReusedExchange [Reuses operator id: 136] +(11) ReusedExchange [Reuses operator id: 133] Output [2]: [d_date_sk#13, d_year#14] (12) BroadcastHashJoin [codegen id : 3] @@ -216,24 +213,24 @@ Condition : (isnotnull(cr_order_number#16) AND isnotnull(cr_item_sk#15)) Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -(19) CometExchange +(19) ColumnarToRow [codegen id : 5] Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(20) CometSort +(20) Exchange Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST] +Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) ColumnarToRow [codegen id : 5] +(21) Sort [codegen id : 6] Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] +Arguments: [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST], false, 0 -(22) SortMergeJoin [codegen id : 6] +(22) SortMergeJoin [codegen id : 7] Left keys [2]: [cs_order_number#2, cs_item_sk#1] Right keys [2]: [cr_order_number#16, cr_item_sk#15] Join type: LeftOuter Join condition: None -(23) Project [codegen id : 6] +(23) Project [codegen id : 7] Output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, (cs_ext_sales_price#4 - coalesce(cr_return_amount#18, 0.00)) AS sales_amt#21] Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14, cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] @@ -249,32 +246,32 @@ ReadSchema: struct -(89) CometFilter +(88) CometFilter Input [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] Condition : isnotnull(ss_item_sk#87) -(90) ColumnarToRow [codegen id : 31] +(89) ColumnarToRow [codegen id : 35] Input [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] -(91) ReusedExchange [Reuses operator id: 8] +(90) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] -(92) BroadcastHashJoin [codegen id : 31] +(91) BroadcastHashJoin [codegen id : 35] Left keys [1]: [ss_item_sk#87] Right keys [1]: [i_item_sk#93] Join type: Inner Join condition: None -(93) Project [codegen id : 31] +(92) Project [codegen id : 35] Output [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] Input [10]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] -(94) ReusedExchange [Reuses operator id: 140] +(93) ReusedExchange [Reuses operator id: 137] Output [2]: [d_date_sk#98, d_year#99] -(95) BroadcastHashJoin [codegen id : 31] +(94) BroadcastHashJoin [codegen id : 35] Left keys [1]: [ss_sold_date_sk#91] Right keys [1]: [d_date_sk#98] Join type: Inner Join condition: None -(96) Project [codegen id : 31] +(95) Project [codegen id : 35] Output [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] Input [11]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_date_sk#98, d_year#99] -(97) Exchange +(96) Exchange Input [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] Arguments: hashpartitioning(ss_ticket_number#88, ss_item_sk#87, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(98) Sort [codegen id : 32] +(97) Sort [codegen id : 36] Input [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] Arguments: [ss_ticket_number#88 ASC NULLS FIRST, ss_item_sk#87 ASC NULLS FIRST], false, 0 -(99) ReusedExchange [Reuses operator id: 38] +(98) ReusedExchange [Reuses operator id: 39] Output [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] -(100) CometSort +(99) Sort [codegen id : 38] Input [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] -Arguments: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103], [sr_ticket_number#101 ASC NULLS FIRST, sr_item_sk#100 ASC NULLS FIRST] +Arguments: [sr_ticket_number#101 ASC NULLS FIRST, sr_item_sk#100 ASC NULLS FIRST], false, 0 -(101) ColumnarToRow [codegen id : 33] -Input [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] - -(102) SortMergeJoin [codegen id : 34] +(100) SortMergeJoin [codegen id : 39] Left keys [2]: [ss_ticket_number#88, ss_item_sk#87] Right keys [2]: [sr_ticket_number#101, sr_item_sk#100] Join type: LeftOuter Join condition: None -(103) Project [codegen id : 34] +(101) Project [codegen id : 39] Output [7]: [d_year#99, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, (ss_quantity#89 - coalesce(sr_return_quantity#102, 0)) AS sales_cnt#40, (ss_ext_sales_price#90 - coalesce(sr_return_amt#103, 0.00)) AS sales_amt#41] Input [13]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99, sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] @@ -603,138 +594,135 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#108), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(105) CometFilter +(103) CometFilter Input [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] Condition : isnotnull(ws_item_sk#104) -(106) ColumnarToRow [codegen id : 37] +(104) ColumnarToRow [codegen id : 42] Input [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] -(107) ReusedExchange [Reuses operator id: 8] +(105) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -(108) BroadcastHashJoin [codegen id : 37] +(106) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ws_item_sk#104] Right keys [1]: [i_item_sk#110] Join type: Inner Join condition: None -(109) Project [codegen id : 37] +(107) Project [codegen id : 42] Output [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] Input [10]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -(110) ReusedExchange [Reuses operator id: 140] +(108) ReusedExchange [Reuses operator id: 137] Output [2]: [d_date_sk#115, d_year#116] -(111) BroadcastHashJoin [codegen id : 37] +(109) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ws_sold_date_sk#108] Right keys [1]: [d_date_sk#115] Join type: Inner Join condition: None -(112) Project [codegen id : 37] +(110) Project [codegen id : 42] Output [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] Input [11]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_date_sk#115, d_year#116] -(113) Exchange +(111) Exchange Input [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] Arguments: hashpartitioning(ws_order_number#105, ws_item_sk#104, 5), ENSURE_REQUIREMENTS, [plan_id=13] -(114) Sort [codegen id : 38] +(112) Sort [codegen id : 43] Input [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] Arguments: [ws_order_number#105 ASC NULLS FIRST, ws_item_sk#104 ASC NULLS FIRST], false, 0 -(115) ReusedExchange [Reuses operator id: 57] +(113) ReusedExchange [Reuses operator id: 58] Output [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] -(116) CometSort -Input [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] -Arguments: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120], [wr_order_number#118 ASC NULLS FIRST, wr_item_sk#117 ASC NULLS FIRST] - -(117) ColumnarToRow [codegen id : 39] +(114) Sort [codegen id : 45] Input [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] +Arguments: [wr_order_number#118 ASC NULLS FIRST, wr_item_sk#117 ASC NULLS FIRST], false, 0 -(118) SortMergeJoin [codegen id : 40] +(115) SortMergeJoin [codegen id : 46] Left keys [2]: [ws_order_number#105, ws_item_sk#104] Right keys [2]: [wr_order_number#118, wr_item_sk#117] Join type: LeftOuter Join condition: None -(119) Project [codegen id : 40] +(116) Project [codegen id : 46] Output [7]: [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, (ws_quantity#106 - coalesce(wr_return_quantity#119, 0)) AS sales_cnt#60, (ws_ext_sales_price#107 - coalesce(wr_return_amt#120, 0.00)) AS sales_amt#61] Input [13]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116, wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] -(120) Union +(117) Union -(121) HashAggregate [codegen id : 41] +(118) HashAggregate [codegen id : 47] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] Functions: [] Aggregate Attributes: [] Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] -(122) Exchange +(119) Exchange Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] Arguments: hashpartitioning(d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(123) HashAggregate [codegen id : 42] +(120) HashAggregate [codegen id : 48] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] Functions: [] Aggregate Attributes: [] Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] -(124) HashAggregate [codegen id : 42] +(121) HashAggregate [codegen id : 48] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] Keys [5]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] Aggregate Attributes [2]: [sum#62, sum#121] Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#64, sum#122] -(125) Exchange +(122) Exchange Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#64, sum#122] Arguments: hashpartitioning(d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, 5), ENSURE_REQUIREMENTS, [plan_id=15] -(126) HashAggregate [codegen id : 43] +(123) HashAggregate [codegen id : 49] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#64, sum#122] Keys [5]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] Aggregate Attributes [2]: [sum(sales_cnt#20)#66, sum(UnscaledValue(sales_amt#21))#67] Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum(sales_cnt#20)#66 AS sales_cnt#123, MakeDecimal(sum(UnscaledValue(sales_amt#21))#67,18,2) AS sales_amt#124] -(127) Filter [codegen id : 43] +(124) Filter [codegen id : 49] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#123, sales_amt#124] Condition : isnotnull(sales_cnt#123) -(128) Exchange +(125) Exchange Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#123, sales_amt#124] Arguments: hashpartitioning(i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, 5), ENSURE_REQUIREMENTS, [plan_id=16] -(129) Sort [codegen id : 44] +(126) Sort [codegen id : 50] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#123, sales_amt#124] Arguments: [i_brand_id#77 ASC NULLS FIRST, i_class_id#78 ASC NULLS FIRST, i_category_id#79 ASC NULLS FIRST, i_manufact_id#80 ASC NULLS FIRST], false, 0 -(130) SortMergeJoin [codegen id : 45] +(127) SortMergeJoin [codegen id : 51] Left keys [4]: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Right keys [4]: [i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] Join type: Inner Join condition: ((cast(sales_cnt#68 as decimal(17,2)) / cast(sales_cnt#123 as decimal(17,2))) < 0.90000000000000000000) -(131) Project [codegen id : 45] +(128) Project [codegen id : 51] Output [10]: [d_year#82 AS prev_year#125, d_year#14 AS year#126, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#123 AS prev_yr_cnt#127, sales_cnt#68 AS curr_yr_cnt#128, (sales_cnt#68 - sales_cnt#123) AS sales_cnt_diff#129, (sales_amt#69 - sales_amt#124) AS sales_amt_diff#130] Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69, d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#123, sales_amt#124] -(132) TakeOrderedAndProject +(129) TakeOrderedAndProject Input [10]: [prev_year#125, year#126, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#127, curr_yr_cnt#128, sales_cnt_diff#129, sales_amt_diff#130] Arguments: 100, [sales_cnt_diff#129 ASC NULLS FIRST], [prev_year#125, year#126, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#127, curr_yr_cnt#128, sales_cnt_diff#129, sales_amt_diff#130] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (136) -+- * ColumnarToRow (135) - +- CometFilter (134) - +- CometScan parquet spark_catalog.default.date_dim (133) +BroadcastExchange (133) ++- * ColumnarToRow (132) + +- CometFilter (131) + +- CometScan parquet spark_catalog.default.date_dim (130) (unknown) Scan parquet spark_catalog.default.date_dim @@ -744,14 +732,14 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(134) CometFilter +(131) CometFilter Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(135) ColumnarToRow [codegen id : 1] +(132) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] -(136) BroadcastExchange +(133) BroadcastExchange Input [2]: [d_date_sk#13, d_year#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] @@ -760,10 +748,10 @@ Subquery:2 Hosting operator id = 24 Hosting Expression = ss_sold_date_sk#26 IN d Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 Subquery:4 Hosting operator id = 72 Hosting Expression = cs_sold_date_sk#74 IN dynamicpruning#75 -BroadcastExchange (140) -+- * ColumnarToRow (139) - +- CometFilter (138) - +- CometScan parquet spark_catalog.default.date_dim (137) +BroadcastExchange (137) ++- * ColumnarToRow (136) + +- CometFilter (135) + +- CometScan parquet spark_catalog.default.date_dim (134) (unknown) Scan parquet spark_catalog.default.date_dim @@ -773,19 +761,19 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(138) CometFilter +(135) CometFilter Input [2]: [d_date_sk#81, d_year#82] Condition : ((isnotnull(d_year#82) AND (d_year#82 = 2001)) AND isnotnull(d_date_sk#81)) -(139) ColumnarToRow [codegen id : 1] +(136) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#81, d_year#82] -(140) BroadcastExchange +(137) BroadcastExchange Input [2]: [d_date_sk#81, d_year#82] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] -Subquery:5 Hosting operator id = 88 Hosting Expression = ss_sold_date_sk#91 IN dynamicpruning#75 +Subquery:5 Hosting operator id = 87 Hosting Expression = ss_sold_date_sk#91 IN dynamicpruning#75 -Subquery:6 Hosting operator id = 104 Hosting Expression = ws_sold_date_sk#108 IN dynamicpruning#75 +Subquery:6 Hosting operator id = 102 Hosting Expression = ws_sold_date_sk#108 IN dynamicpruning#75 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt index 837398c8f..44bcabcdb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt @@ -1,27 +1,27 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_amt_diff] - WholeStageCodegen (45) + WholeStageCodegen (51) Project [d_year,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt,sales_amt,sales_amt] SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_manufact_id,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt] InputAdapter - WholeStageCodegen (22) + WholeStageCodegen (25) Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - WholeStageCodegen (21) + WholeStageCodegen (24) Filter [sales_cnt] HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(sales_cnt),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] InputAdapter Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - WholeStageCodegen (20) + WholeStageCodegen (23) HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 - WholeStageCodegen (19) + WholeStageCodegen (22) HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Union - WholeStageCodegen (6) + WholeStageCodegen (7) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] InputAdapter @@ -56,23 +56,25 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat InputAdapter ReusedExchange [d_date_sk,d_year] #5 InputAdapter - WholeStageCodegen (5) - ColumnarToRow + WholeStageCodegen (6) + Sort [cr_order_number,cr_item_sk] InputAdapter - CometSort [cr_order_number,cr_item_sk] - CometExchange [cr_order_number,cr_item_sk] #7 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_order_number,cr_item_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - WholeStageCodegen (12) + Exchange [cr_order_number,cr_item_sk] #7 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_order_number,cr_item_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + WholeStageCodegen (14) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] InputAdapter - WholeStageCodegen (10) + WholeStageCodegen (11) Sort [ss_ticket_number,ss_item_sk] InputAdapter Exchange [ss_ticket_number,ss_item_sk] #8 - WholeStageCodegen (9) + WholeStageCodegen (10) Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] @@ -87,23 +89,25 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat InputAdapter ReusedExchange [d_date_sk,d_year] #5 InputAdapter - WholeStageCodegen (11) - ColumnarToRow + WholeStageCodegen (13) + Sort [sr_ticket_number,sr_item_sk] InputAdapter - CometSort [sr_ticket_number,sr_item_sk] - CometExchange [sr_ticket_number,sr_item_sk] #9 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_ticket_number,sr_item_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - WholeStageCodegen (18) + Exchange [sr_ticket_number,sr_item_sk] #9 + WholeStageCodegen (12) + ColumnarToRow + InputAdapter + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_ticket_number,sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + WholeStageCodegen (21) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] InputAdapter - WholeStageCodegen (16) + WholeStageCodegen (18) Sort [ws_order_number,ws_item_sk] InputAdapter Exchange [ws_order_number,ws_item_sk] #10 - WholeStageCodegen (15) + WholeStageCodegen (17) Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] @@ -118,42 +122,44 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat InputAdapter ReusedExchange [d_date_sk,d_year] #5 InputAdapter - WholeStageCodegen (17) - ColumnarToRow + WholeStageCodegen (20) + Sort [wr_order_number,wr_item_sk] InputAdapter - CometSort [wr_order_number,wr_item_sk] - CometExchange [wr_order_number,wr_item_sk] #11 - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_order_number,wr_item_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + Exchange [wr_order_number,wr_item_sk] #11 + WholeStageCodegen (19) + ColumnarToRow + InputAdapter + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_order_number,wr_item_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] InputAdapter - WholeStageCodegen (44) + WholeStageCodegen (50) Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 - WholeStageCodegen (43) + WholeStageCodegen (49) Filter [sales_cnt] HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(sales_cnt),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] InputAdapter Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 - WholeStageCodegen (42) + WholeStageCodegen (48) HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #14 - WholeStageCodegen (41) + WholeStageCodegen (47) HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Union - WholeStageCodegen (28) + WholeStageCodegen (32) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] InputAdapter - WholeStageCodegen (26) + WholeStageCodegen (29) Sort [cs_order_number,cs_item_sk] InputAdapter Exchange [cs_order_number,cs_item_sk] #15 - WholeStageCodegen (25) + WholeStageCodegen (28) Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] @@ -174,20 +180,19 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat InputAdapter ReusedExchange [d_date_sk,d_year] #16 InputAdapter - WholeStageCodegen (27) - ColumnarToRow + WholeStageCodegen (31) + Sort [cr_order_number,cr_item_sk] InputAdapter - CometSort [cr_order_number,cr_item_sk] - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 - WholeStageCodegen (34) + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 + WholeStageCodegen (39) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] InputAdapter - WholeStageCodegen (32) + WholeStageCodegen (36) Sort [ss_ticket_number,ss_item_sk] InputAdapter Exchange [ss_ticket_number,ss_item_sk] #17 - WholeStageCodegen (31) + WholeStageCodegen (35) Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] @@ -202,20 +207,19 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat InputAdapter ReusedExchange [d_date_sk,d_year] #16 InputAdapter - WholeStageCodegen (33) - ColumnarToRow + WholeStageCodegen (38) + Sort [sr_ticket_number,sr_item_sk] InputAdapter - CometSort [sr_ticket_number,sr_item_sk] - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 - WholeStageCodegen (40) + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 + WholeStageCodegen (46) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] InputAdapter - WholeStageCodegen (38) + WholeStageCodegen (43) Sort [ws_order_number,ws_item_sk] InputAdapter Exchange [ws_order_number,ws_item_sk] #18 - WholeStageCodegen (37) + WholeStageCodegen (42) Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] @@ -230,8 +234,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat InputAdapter ReusedExchange [d_date_sk,d_year] #16 InputAdapter - WholeStageCodegen (39) - ColumnarToRow + WholeStageCodegen (45) + Sort [wr_order_number,wr_item_sk] InputAdapter - CometSort [wr_order_number,wr_item_sk] - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 + ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt index caab92168..d9e4b80d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt @@ -1,71 +1,74 @@ == Physical Plan == -TakeOrderedAndProject (67) -+- * Project (66) - +- * SortMergeJoin Inner (65) - :- * Project (43) - : +- * SortMergeJoin Inner (42) - : :- * Sort (20) - : : +- * HashAggregate (19) - : : +- Exchange (18) - : : +- * HashAggregate (17) - : : +- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * ColumnarToRow (13) - : : : +- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometSortMergeJoin (10) - : : : :- CometSort (4) - : : : : +- CometExchange (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- CometSort (9) - : : : +- CometExchange (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan parquet spark_catalog.default.store_returns (5) - : : +- ReusedExchange (14) - : +- * Sort (41) - : +- * Filter (40) - : +- * HashAggregate (39) - : +- Exchange (38) - : +- * HashAggregate (37) - : +- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * ColumnarToRow (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometSortMergeJoin (30) - : : :- CometSort (24) - : : : +- CometExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan parquet spark_catalog.default.web_sales (21) - : : +- CometSort (29) - : : +- CometExchange (28) - : : +- CometProject (27) - : : +- CometFilter (26) - : : +- CometScan parquet spark_catalog.default.web_returns (25) - : +- ReusedExchange (34) - +- * Sort (64) - +- * Filter (63) - +- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- * Project (59) - +- * BroadcastHashJoin Inner BuildRight (58) - :- * ColumnarToRow (56) - : +- CometProject (55) - : +- CometFilter (54) - : +- CometSortMergeJoin (53) - : :- CometSort (47) - : : +- CometExchange (46) - : : +- CometFilter (45) - : : +- CometScan parquet spark_catalog.default.catalog_sales (44) - : +- CometSort (52) - : +- CometExchange (51) - : +- CometProject (50) - : +- CometFilter (49) - : +- CometScan parquet spark_catalog.default.catalog_returns (48) - +- ReusedExchange (57) +TakeOrderedAndProject (70) ++- * Project (69) + +- * SortMergeJoin Inner (68) + :- * Project (45) + : +- * SortMergeJoin Inner (44) + : :- * Sort (21) + : : +- * HashAggregate (20) + : : +- Exchange (19) + : : +- * HashAggregate (18) + : : +- * Project (17) + : : +- * BroadcastHashJoin Inner BuildRight (16) + : : :- * Project (14) + : : : +- * Filter (13) + : : : +- * SortMergeJoin LeftOuter (12) + : : : :- * Sort (5) + : : : : +- Exchange (4) + : : : : +- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- * Sort (11) + : : : +- Exchange (10) + : : : +- * ColumnarToRow (9) + : : : +- CometProject (8) + : : : +- CometFilter (7) + : : : +- CometScan parquet spark_catalog.default.store_returns (6) + : : +- ReusedExchange (15) + : +- * Sort (43) + : +- * Filter (42) + : +- * HashAggregate (41) + : +- Exchange (40) + : +- * HashAggregate (39) + : +- * Project (38) + : +- * BroadcastHashJoin Inner BuildRight (37) + : :- * Project (35) + : : +- * Filter (34) + : : +- * SortMergeJoin LeftOuter (33) + : : :- * Sort (26) + : : : +- Exchange (25) + : : : +- * ColumnarToRow (24) + : : : +- CometFilter (23) + : : : +- CometScan parquet spark_catalog.default.web_sales (22) + : : +- * Sort (32) + : : +- Exchange (31) + : : +- * ColumnarToRow (30) + : : +- CometProject (29) + : : +- CometFilter (28) + : : +- CometScan parquet spark_catalog.default.web_returns (27) + : +- ReusedExchange (36) + +- * Sort (67) + +- * Filter (66) + +- * HashAggregate (65) + +- Exchange (64) + +- * HashAggregate (63) + +- * Project (62) + +- * BroadcastHashJoin Inner BuildRight (61) + :- * Project (59) + : +- * Filter (58) + : +- * SortMergeJoin LeftOuter (57) + : :- * Sort (50) + : : +- Exchange (49) + : : +- * ColumnarToRow (48) + : : +- CometFilter (47) + : : +- CometScan parquet spark_catalog.default.catalog_sales (46) + : +- * Sort (56) + : +- Exchange (55) + : +- * ColumnarToRow (54) + : +- CometProject (53) + : +- CometFilter (52) + : +- CometScan parquet spark_catalog.default.catalog_returns (51) + +- ReusedExchange (60) (unknown) Scan parquet spark_catalog.default.store_sales @@ -80,13 +83,16 @@ ReadSchema: struct -(6) CometFilter +(7) CometFilter Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] Condition : (isnotnull(sr_ticket_number#10) AND isnotnull(sr_item_sk#9)) -(7) CometProject +(8) CometProject Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_item_sk#9, sr_ticket_number#10] -(8) CometExchange +(9) ColumnarToRow [codegen id : 3] Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(9) CometSort +(10) Exchange Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST] +Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(10) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_ticket_number#3, ss_item_sk#1], [sr_ticket_number#10, sr_item_sk#9], LeftOuter +(11) Sort [codegen id : 4] +Input [2]: [sr_item_sk#9, sr_ticket_number#10] +Arguments: [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST], false, 0 + +(12) SortMergeJoin [codegen id : 6] +Left keys [2]: [ss_ticket_number#3, ss_item_sk#1] +Right keys [2]: [sr_ticket_number#10, sr_item_sk#9] +Join type: LeftOuter +Join condition: None -(11) CometFilter +(13) Filter [codegen id : 6] Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] Condition : isnull(sr_ticket_number#10) -(12) CometProject +(14) Project [codegen id : 6] +Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -(13) ColumnarToRow [codegen id : 2] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] - -(14) ReusedExchange [Reuses operator id: 71] +(15) ReusedExchange [Reuses operator id: 74] Output [2]: [d_date_sk#12, d_year#13] -(15) BroadcastHashJoin [codegen id : 2] +(16) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#12] Join type: Inner Join condition: None -(16) Project [codegen id : 2] +(17) Project [codegen id : 6] Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] -(17) HashAggregate [codegen id : 2] +(18) HashAggregate [codegen id : 6] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesale_cost#5)), partial_sum(UnscaledValue(ss_sales_price#6))] Aggregate Attributes [3]: [sum#14, sum#15, sum#16] Results [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] -(18) Exchange +(19) Exchange Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(19) HashAggregate [codegen id : 3] +(20) HashAggregate [codegen id : 7] Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] Aggregate Attributes [3]: [sum(ss_quantity#4)#20, sum(UnscaledValue(ss_wholesale_cost#5))#21, sum(UnscaledValue(ss_sales_price#6))#22] Results [6]: [d_year#13 AS ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, sum(ss_quantity#4)#20 AS ss_qty#24, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#5))#21,17,2) AS ss_wc#25, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#22,17,2) AS ss_sp#26] -(20) Sort [codegen id : 3] +(21) Sort [codegen id : 7] Input [6]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] Arguments: [ss_sold_year#23 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], false, 0 @@ -170,17 +177,20 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(22) CometFilter +(23) CometFilter Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] Condition : (isnotnull(ws_item_sk#27) AND isnotnull(ws_bill_customer_sk#28)) -(23) CometExchange +(24) ColumnarToRow [codegen id : 8] +Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] + +(25) Exchange Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Arguments: hashpartitioning(ws_order_number#29, ws_item_sk#27, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +Arguments: hashpartitioning(ws_order_number#29, ws_item_sk#27, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(24) CometSort +(26) Sort [codegen id : 9] Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Arguments: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33], [ws_order_number#29 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST] +Arguments: [ws_order_number#29 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.web_returns Output [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] @@ -189,84 +199,85 @@ Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct -(26) CometFilter +(28) CometFilter Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] Condition : (isnotnull(wr_order_number#36) AND isnotnull(wr_item_sk#35)) -(27) CometProject +(29) CometProject Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] Arguments: [wr_item_sk#35, wr_order_number#36], [wr_item_sk#35, wr_order_number#36] -(28) CometExchange +(30) ColumnarToRow [codegen id : 10] +Input [2]: [wr_item_sk#35, wr_order_number#36] + +(31) Exchange Input [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: hashpartitioning(wr_order_number#36, wr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +Arguments: hashpartitioning(wr_order_number#36, wr_item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(29) CometSort +(32) Sort [codegen id : 11] Input [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: [wr_item_sk#35, wr_order_number#36], [wr_order_number#36 ASC NULLS FIRST, wr_item_sk#35 ASC NULLS FIRST] +Arguments: [wr_order_number#36 ASC NULLS FIRST, wr_item_sk#35 ASC NULLS FIRST], false, 0 -(30) CometSortMergeJoin -Left output [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Right output [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: [ws_order_number#29, ws_item_sk#27], [wr_order_number#36, wr_item_sk#35], LeftOuter +(33) SortMergeJoin [codegen id : 13] +Left keys [2]: [ws_order_number#29, ws_item_sk#27] +Right keys [2]: [wr_order_number#36, wr_item_sk#35] +Join type: LeftOuter +Join condition: None -(31) CometFilter +(34) Filter [codegen id : 13] Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] Condition : isnull(wr_order_number#36) -(32) CometProject +(35) Project [codegen id : 13] +Output [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] -Arguments: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33], [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] - -(33) ColumnarToRow [codegen id : 5] -Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -(34) ReusedExchange [Reuses operator id: 71] +(36) ReusedExchange [Reuses operator id: 74] Output [2]: [d_date_sk#38, d_year#39] -(35) BroadcastHashJoin [codegen id : 5] +(37) BroadcastHashJoin [codegen id : 13] Left keys [1]: [ws_sold_date_sk#33] Right keys [1]: [d_date_sk#38] Join type: Inner Join condition: None -(36) Project [codegen id : 5] +(38) Project [codegen id : 13] Output [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] Input [8]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, d_date_sk#38, d_year#39] -(37) HashAggregate [codegen id : 5] +(39) HashAggregate [codegen id : 13] Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] Functions [3]: [partial_sum(ws_quantity#30), partial_sum(UnscaledValue(ws_wholesale_cost#31)), partial_sum(UnscaledValue(ws_sales_price#32))] Aggregate Attributes [3]: [sum#40, sum#41, sum#42] Results [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] -(38) Exchange +(40) Exchange Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] Arguments: hashpartitioning(d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(39) HashAggregate [codegen id : 6] +(41) HashAggregate [codegen id : 14] Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] Functions [3]: [sum(ws_quantity#30), sum(UnscaledValue(ws_wholesale_cost#31)), sum(UnscaledValue(ws_sales_price#32))] Aggregate Attributes [3]: [sum(ws_quantity#30)#46, sum(UnscaledValue(ws_wholesale_cost#31))#47, sum(UnscaledValue(ws_sales_price#32))#48] Results [6]: [d_year#39 AS ws_sold_year#49, ws_item_sk#27, ws_bill_customer_sk#28 AS ws_customer_sk#50, sum(ws_quantity#30)#46 AS ws_qty#51, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#31))#47,17,2) AS ws_wc#52, MakeDecimal(sum(UnscaledValue(ws_sales_price#32))#48,17,2) AS ws_sp#53] -(40) Filter [codegen id : 6] +(42) Filter [codegen id : 14] Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] Condition : (coalesce(ws_qty#51, 0) > 0) -(41) Sort [codegen id : 6] +(43) Sort [codegen id : 14] Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] Arguments: [ws_sold_year#49 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST, ws_customer_sk#50 ASC NULLS FIRST], false, 0 -(42) SortMergeJoin [codegen id : 7] +(44) SortMergeJoin [codegen id : 15] Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] Right keys [3]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50] Join type: Inner Join condition: None -(43) Project [codegen id : 7] +(45) Project [codegen id : 15] Output [9]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53] Input [12]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] @@ -278,17 +289,20 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(45) CometFilter +(47) CometFilter Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] Condition : (isnotnull(cs_item_sk#55) AND isnotnull(cs_bill_customer_sk#54)) -(46) CometExchange +(48) ColumnarToRow [codegen id : 16] +Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] + +(49) Exchange Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Arguments: hashpartitioning(cs_order_number#56, cs_item_sk#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +Arguments: hashpartitioning(cs_order_number#56, cs_item_sk#55, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(47) CometSort +(50) Sort [codegen id : 17] Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Arguments: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60], [cs_order_number#56 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST] +Arguments: [cs_order_number#56 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.catalog_returns Output [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] @@ -297,98 +311,99 @@ Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(49) CometFilter +(52) CometFilter Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] Condition : (isnotnull(cr_order_number#63) AND isnotnull(cr_item_sk#62)) -(50) CometProject +(53) CometProject Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] Arguments: [cr_item_sk#62, cr_order_number#63], [cr_item_sk#62, cr_order_number#63] -(51) CometExchange +(54) ColumnarToRow [codegen id : 18] Input [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: hashpartitioning(cr_order_number#63, cr_item_sk#62, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] -(52) CometSort +(55) Exchange Input [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: [cr_item_sk#62, cr_order_number#63], [cr_order_number#63 ASC NULLS FIRST, cr_item_sk#62 ASC NULLS FIRST] +Arguments: hashpartitioning(cr_order_number#63, cr_item_sk#62, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(53) CometSortMergeJoin -Left output [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Right output [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: [cs_order_number#56, cs_item_sk#55], [cr_order_number#63, cr_item_sk#62], LeftOuter +(56) Sort [codegen id : 19] +Input [2]: [cr_item_sk#62, cr_order_number#63] +Arguments: [cr_order_number#63 ASC NULLS FIRST, cr_item_sk#62 ASC NULLS FIRST], false, 0 + +(57) SortMergeJoin [codegen id : 21] +Left keys [2]: [cs_order_number#56, cs_item_sk#55] +Right keys [2]: [cr_order_number#63, cr_item_sk#62] +Join type: LeftOuter +Join condition: None -(54) CometFilter +(58) Filter [codegen id : 21] Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] Condition : isnull(cr_order_number#63) -(55) CometProject +(59) Project [codegen id : 21] +Output [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] -Arguments: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60], [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] - -(56) ColumnarToRow [codegen id : 9] -Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -(57) ReusedExchange [Reuses operator id: 71] +(60) ReusedExchange [Reuses operator id: 74] Output [2]: [d_date_sk#65, d_year#66] -(58) BroadcastHashJoin [codegen id : 9] +(61) BroadcastHashJoin [codegen id : 21] Left keys [1]: [cs_sold_date_sk#60] Right keys [1]: [d_date_sk#65] Join type: Inner Join condition: None -(59) Project [codegen id : 9] +(62) Project [codegen id : 21] Output [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] Input [8]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, d_date_sk#65, d_year#66] -(60) HashAggregate [codegen id : 9] +(63) HashAggregate [codegen id : 21] Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] Functions [3]: [partial_sum(cs_quantity#57), partial_sum(UnscaledValue(cs_wholesale_cost#58)), partial_sum(UnscaledValue(cs_sales_price#59))] Aggregate Attributes [3]: [sum#67, sum#68, sum#69] Results [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] -(61) Exchange +(64) Exchange Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] Arguments: hashpartitioning(d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(62) HashAggregate [codegen id : 10] +(65) HashAggregate [codegen id : 22] Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] Functions [3]: [sum(cs_quantity#57), sum(UnscaledValue(cs_wholesale_cost#58)), sum(UnscaledValue(cs_sales_price#59))] Aggregate Attributes [3]: [sum(cs_quantity#57)#73, sum(UnscaledValue(cs_wholesale_cost#58))#74, sum(UnscaledValue(cs_sales_price#59))#75] Results [6]: [d_year#66 AS cs_sold_year#76, cs_item_sk#55, cs_bill_customer_sk#54 AS cs_customer_sk#77, sum(cs_quantity#57)#73 AS cs_qty#78, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#58))#74,17,2) AS cs_wc#79, MakeDecimal(sum(UnscaledValue(cs_sales_price#59))#75,17,2) AS cs_sp#80] -(63) Filter [codegen id : 10] +(66) Filter [codegen id : 22] Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] Condition : (coalesce(cs_qty#78, 0) > 0) -(64) Sort [codegen id : 10] +(67) Sort [codegen id : 22] Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] Arguments: [cs_sold_year#76 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST, cs_customer_sk#77 ASC NULLS FIRST], false, 0 -(65) SortMergeJoin [codegen id : 11] +(68) SortMergeJoin [codegen id : 23] Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] Right keys [3]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77] Join type: Inner Join condition: None -(66) Project [codegen id : 11] +(69) Project [codegen id : 23] Output [12]: [round((cast(ss_qty#24 as double) / cast(coalesce((ws_qty#51 + cs_qty#78), 1) as double)), 2) AS ratio#81, ss_qty#24 AS store_qty#82, ss_wc#25 AS store_wholesale_cost#83, ss_sp#26 AS store_sales_price#84, (coalesce(ws_qty#51, 0) + coalesce(cs_qty#78, 0)) AS other_chan_qty#85, (coalesce(ws_wc#52, 0.00) + coalesce(cs_wc#79, 0.00)) AS other_chan_wholesale_cost#86, (coalesce(ws_sp#53, 0.00) + coalesce(cs_sp#80, 0.00)) AS other_chan_sales_price#87, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, cs_qty#78] Input [15]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53, cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] -(67) TakeOrderedAndProject +(70) TakeOrderedAndProject Input [12]: [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, cs_qty#78] Arguments: 100, [ratio#81 ASC NULLS FIRST, ss_qty#24 DESC NULLS LAST, ss_wc#25 DESC NULLS LAST, ss_sp#26 DESC NULLS LAST, other_chan_qty#85 ASC NULLS FIRST, other_chan_wholesale_cost#86 ASC NULLS FIRST, other_chan_sales_price#87 ASC NULLS FIRST, round((cast(ss_qty#24 as double) / cast(coalesce((ws_qty#51 + cs_qty#78), 1) as double)), 2) ASC NULLS FIRST], [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (71) -+- * ColumnarToRow (70) - +- CometFilter (69) - +- CometScan parquet spark_catalog.default.date_dim (68) +BroadcastExchange (74) ++- * ColumnarToRow (73) + +- CometFilter (72) + +- CometScan parquet spark_catalog.default.date_dim (71) (unknown) Scan parquet spark_catalog.default.date_dim @@ -398,19 +413,19 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(69) CometFilter +(72) CometFilter Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(70) ColumnarToRow [codegen id : 1] +(73) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] -(71) BroadcastExchange +(74) BroadcastExchange Input [2]: [d_date_sk#12, d_year#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -Subquery:2 Hosting operator id = 21 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt index 7b351afcd..280687e30 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt @@ -1,97 +1,127 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ws_qty,cs_qty,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (11) + WholeStageCodegen (23) Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp] SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,cs_sold_year,cs_item_sk,cs_customer_sk] InputAdapter - WholeStageCodegen (7) + WholeStageCodegen (15) Project [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ws_sold_year,ws_item_sk,ws_customer_sk] InputAdapter - WholeStageCodegen (3) + WholeStageCodegen (7) Sort [ss_sold_year,ss_item_sk,ss_customer_sk] HashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] [sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price)),ss_sold_year,ss_qty,ss_wc,ss_sp,sum,sum,sum] InputAdapter Exchange [d_year,ss_item_sk,ss_customer_sk] #1 - WholeStageCodegen (2) + WholeStageCodegen (6) HashAggregate [d_year,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] [sum,sum,sum,sum,sum,sum] Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometFilter [sr_ticket_number] - CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometSort [ss_ticket_number,ss_item_sk] - CometExchange [ss_ticket_number,ss_item_sk] #2 - CometFilter [ss_item_sk,ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [sr_ticket_number,sr_item_sk] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + Filter [sr_ticket_number] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ss_ticket_number,ss_item_sk] + InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + WholeStageCodegen (4) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + Exchange [sr_ticket_number,sr_item_sk] #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_ticket_number,sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 InputAdapter - WholeStageCodegen (6) + WholeStageCodegen (14) Sort [ws_sold_year,ws_item_sk,ws_customer_sk] Filter [ws_qty] HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] [sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price)),ws_sold_year,ws_customer_sk,ws_qty,ws_wc,ws_sp,sum,sum,sum] InputAdapter Exchange [d_year,ws_item_sk,ws_bill_customer_sk] #5 - WholeStageCodegen (5) + WholeStageCodegen (13) HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] [sum,sum,sum,sum,sum,sum] Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometFilter [wr_order_number] - CometSortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - CometSort [ws_order_number,ws_item_sk] - CometExchange [ws_order_number,ws_item_sk] #6 - CometFilter [ws_item_sk,ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [wr_order_number,wr_item_sk] - CometExchange [wr_order_number,wr_item_sk] #7 - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_order_number,wr_item_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] + Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + Filter [wr_order_number] + SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + InputAdapter + WholeStageCodegen (9) + Sort [ws_order_number,ws_item_sk] + InputAdapter + Exchange [ws_order_number,ws_item_sk] #6 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk,ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + WholeStageCodegen (11) + Sort [wr_order_number,wr_item_sk] + InputAdapter + Exchange [wr_order_number,wr_item_sk] #7 + WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometProject [wr_item_sk,wr_order_number] + CometFilter [wr_order_number,wr_item_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 InputAdapter - WholeStageCodegen (10) + WholeStageCodegen (22) Sort [cs_sold_year,cs_item_sk,cs_customer_sk] Filter [cs_qty] HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] [sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price)),cs_sold_year,cs_customer_sk,cs_qty,cs_wc,cs_sp,sum,sum,sum] InputAdapter Exchange [d_year,cs_item_sk,cs_bill_customer_sk] #8 - WholeStageCodegen (9) + WholeStageCodegen (21) HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] [sum,sum,sum,sum,sum,sum] Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometFilter [cr_order_number] - CometSortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometSort [cs_order_number,cs_item_sk] - CometExchange [cs_order_number,cs_item_sk] #9 - CometFilter [cs_item_sk,cs_bill_customer_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [cr_order_number,cr_item_sk] - CometExchange [cr_order_number,cr_item_sk] #10 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_order_number,cr_item_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + Filter [cr_order_number] + SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + InputAdapter + WholeStageCodegen (17) + Sort [cs_order_number,cs_item_sk] + InputAdapter + Exchange [cs_order_number,cs_item_sk] #9 + WholeStageCodegen (16) + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk,cs_bill_customer_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + WholeStageCodegen (19) + Sort [cr_order_number,cr_item_sk] + InputAdapter + Exchange [cr_order_number,cr_item_sk] #10 + WholeStageCodegen (18) + ColumnarToRow + InputAdapter + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_order_number,cr_item_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt index dfd123a1d..338f86087 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt @@ -1,108 +1,111 @@ == Physical Plan == -TakeOrderedAndProject (104) -+- * HashAggregate (103) - +- Exchange (102) - +- * HashAggregate (101) - +- * Expand (100) - +- Union (99) - :- * HashAggregate (38) - : +- Exchange (37) - : +- * HashAggregate (36) - : +- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (28) - : : +- * BroadcastHashJoin Inner BuildRight (27) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Project (15) - : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : :- * ColumnarToRow (12) - : : : : : +- CometProject (11) - : : : : : +- CometSortMergeJoin (10) - : : : : : :- CometSort (4) - : : : : : : +- CometExchange (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : +- CometSort (9) - : : : : : +- CometExchange (8) - : : : : : +- CometProject (7) - : : : : : +- CometFilter (6) - : : : : : +- CometScan parquet spark_catalog.default.store_returns (5) - : : : : +- ReusedExchange (13) - : : : +- BroadcastExchange (19) - : : : +- * ColumnarToRow (18) - : : : +- CometFilter (17) - : : : +- CometScan parquet spark_catalog.default.store (16) - : : +- BroadcastExchange (26) - : : +- * ColumnarToRow (25) - : : +- CometProject (24) - : : +- CometFilter (23) - : : +- CometScan parquet spark_catalog.default.item (22) - : +- BroadcastExchange (33) - : +- * ColumnarToRow (32) - : +- CometProject (31) - : +- CometFilter (30) - : +- CometScan parquet spark_catalog.default.promotion (29) - :- * HashAggregate (68) - : +- Exchange (67) - : +- * HashAggregate (66) - : +- * Project (65) - : +- * BroadcastHashJoin Inner BuildRight (64) - : :- * Project (62) - : : +- * BroadcastHashJoin Inner BuildRight (61) - : : :- * Project (59) - : : : +- * BroadcastHashJoin Inner BuildRight (58) - : : : :- * Project (53) - : : : : +- * BroadcastHashJoin Inner BuildRight (52) - : : : : :- * ColumnarToRow (50) - : : : : : +- CometProject (49) - : : : : : +- CometSortMergeJoin (48) - : : : : : :- CometSort (42) - : : : : : : +- CometExchange (41) - : : : : : : +- CometFilter (40) - : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (39) - : : : : : +- CometSort (47) - : : : : : +- CometExchange (46) - : : : : : +- CometProject (45) - : : : : : +- CometFilter (44) - : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (43) - : : : : +- ReusedExchange (51) - : : : +- BroadcastExchange (57) - : : : +- * ColumnarToRow (56) - : : : +- CometFilter (55) - : : : +- CometScan parquet spark_catalog.default.catalog_page (54) - : : +- ReusedExchange (60) - : +- ReusedExchange (63) - +- * HashAggregate (98) - +- Exchange (97) - +- * HashAggregate (96) - +- * Project (95) - +- * BroadcastHashJoin Inner BuildRight (94) - :- * Project (92) - : +- * BroadcastHashJoin Inner BuildRight (91) - : :- * Project (89) - : : +- * BroadcastHashJoin Inner BuildRight (88) - : : :- * Project (83) - : : : +- * BroadcastHashJoin Inner BuildRight (82) - : : : :- * ColumnarToRow (80) - : : : : +- CometProject (79) - : : : : +- CometSortMergeJoin (78) - : : : : :- CometSort (72) - : : : : : +- CometExchange (71) - : : : : : +- CometFilter (70) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (69) - : : : : +- CometSort (77) - : : : : +- CometExchange (76) - : : : : +- CometProject (75) - : : : : +- CometFilter (74) - : : : : +- CometScan parquet spark_catalog.default.web_returns (73) - : : : +- ReusedExchange (81) - : : +- BroadcastExchange (87) - : : +- * ColumnarToRow (86) - : : +- CometFilter (85) - : : +- CometScan parquet spark_catalog.default.web_site (84) - : +- ReusedExchange (90) - +- ReusedExchange (93) +TakeOrderedAndProject (107) ++- * HashAggregate (106) + +- Exchange (105) + +- * HashAggregate (104) + +- * Expand (103) + +- Union (102) + :- * HashAggregate (39) + : +- Exchange (38) + : +- * HashAggregate (37) + : +- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * Project (29) + : : +- * BroadcastHashJoin Inner BuildRight (28) + : : :- * Project (22) + : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : :- * Project (16) + : : : : +- * BroadcastHashJoin Inner BuildRight (15) + : : : : :- * Project (13) + : : : : : +- * SortMergeJoin LeftOuter (12) + : : : : : :- * Sort (5) + : : : : : : +- Exchange (4) + : : : : : : +- * ColumnarToRow (3) + : : : : : : +- CometFilter (2) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : +- * Sort (11) + : : : : : +- Exchange (10) + : : : : : +- * ColumnarToRow (9) + : : : : : +- CometProject (8) + : : : : : +- CometFilter (7) + : : : : : +- CometScan parquet spark_catalog.default.store_returns (6) + : : : : +- ReusedExchange (14) + : : : +- BroadcastExchange (20) + : : : +- * ColumnarToRow (19) + : : : +- CometFilter (18) + : : : +- CometScan parquet spark_catalog.default.store (17) + : : +- BroadcastExchange (27) + : : +- * ColumnarToRow (26) + : : +- CometProject (25) + : : +- CometFilter (24) + : : +- CometScan parquet spark_catalog.default.item (23) + : +- BroadcastExchange (34) + : +- * ColumnarToRow (33) + : +- CometProject (32) + : +- CometFilter (31) + : +- CometScan parquet spark_catalog.default.promotion (30) + :- * HashAggregate (70) + : +- Exchange (69) + : +- * HashAggregate (68) + : +- * Project (67) + : +- * BroadcastHashJoin Inner BuildRight (66) + : :- * Project (64) + : : +- * BroadcastHashJoin Inner BuildRight (63) + : : :- * Project (61) + : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : :- * Project (55) + : : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : : :- * Project (52) + : : : : : +- * SortMergeJoin LeftOuter (51) + : : : : : :- * Sort (44) + : : : : : : +- Exchange (43) + : : : : : : +- * ColumnarToRow (42) + : : : : : : +- CometFilter (41) + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (40) + : : : : : +- * Sort (50) + : : : : : +- Exchange (49) + : : : : : +- * ColumnarToRow (48) + : : : : : +- CometProject (47) + : : : : : +- CometFilter (46) + : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (45) + : : : : +- ReusedExchange (53) + : : : +- BroadcastExchange (59) + : : : +- * ColumnarToRow (58) + : : : +- CometFilter (57) + : : : +- CometScan parquet spark_catalog.default.catalog_page (56) + : : +- ReusedExchange (62) + : +- ReusedExchange (65) + +- * HashAggregate (101) + +- Exchange (100) + +- * HashAggregate (99) + +- * Project (98) + +- * BroadcastHashJoin Inner BuildRight (97) + :- * Project (95) + : +- * BroadcastHashJoin Inner BuildRight (94) + : :- * Project (92) + : : +- * BroadcastHashJoin Inner BuildRight (91) + : : :- * Project (86) + : : : +- * BroadcastHashJoin Inner BuildRight (85) + : : : :- * Project (83) + : : : : +- * SortMergeJoin LeftOuter (82) + : : : : :- * Sort (75) + : : : : : +- Exchange (74) + : : : : : +- * ColumnarToRow (73) + : : : : : +- CometFilter (72) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (71) + : : : : +- * Sort (81) + : : : : +- Exchange (80) + : : : : +- * ColumnarToRow (79) + : : : : +- CometProject (78) + : : : : +- CometFilter (77) + : : : : +- CometScan parquet spark_catalog.default.web_returns (76) + : : : +- ReusedExchange (84) + : : +- BroadcastExchange (90) + : : +- * ColumnarToRow (89) + : : +- CometFilter (88) + : : +- CometScan parquet spark_catalog.default.web_site (87) + : +- ReusedExchange (93) + +- ReusedExchange (96) (unknown) Scan parquet spark_catalog.default.store_sales @@ -117,13 +120,16 @@ ReadSchema: struct -(6) CometFilter +(7) CometFilter Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] Condition : (isnotnull(sr_item_sk#9) AND isnotnull(sr_ticket_number#10)) -(7) CometProject +(8) CometProject Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -(8) CometExchange +(9) ColumnarToRow [codegen id : 3] Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(9) CometSort +(10) Exchange Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST] +Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(10) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Right output [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_ticket_number#4], [sr_item_sk#9, sr_ticket_number#10], LeftOuter +(11) Sort [codegen id : 4] +Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] +Arguments: [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST], false, 0 -(11) CometProject -Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] +(12) SortMergeJoin [codegen id : 9] +Left keys [2]: [ss_item_sk#1, ss_ticket_number#4] +Right keys [2]: [sr_item_sk#9, sr_ticket_number#10] +Join type: LeftOuter +Join condition: None -(12) ColumnarToRow [codegen id : 5] -Input [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] +(13) Project [codegen id : 9] +Output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] +Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -(13) ReusedExchange [Reuses operator id: 109] +(14) ReusedExchange [Reuses operator id: 112] Output [1]: [d_date_sk#14] -(14) BroadcastHashJoin [codegen id : 5] +(15) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(15) Project [codegen id : 5] +(16) Project [codegen id : 9] Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#14] @@ -180,24 +187,24 @@ Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(17) CometFilter +(18) CometFilter Input [2]: [s_store_sk#15, s_store_id#16] Condition : isnotnull(s_store_sk#15) -(18) ColumnarToRow [codegen id : 2] +(19) ColumnarToRow [codegen id : 6] Input [2]: [s_store_sk#15, s_store_id#16] -(19) BroadcastExchange +(20) BroadcastExchange Input [2]: [s_store_sk#15, s_store_id#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(20) BroadcastHashJoin [codegen id : 5] +(21) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#2] Right keys [1]: [s_store_sk#15] Join type: Inner Join condition: None -(21) Project [codegen id : 5] +(22) Project [codegen id : 9] Output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#15, s_store_id#16] @@ -208,28 +215,28 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), GreaterThan(i_current_price,50.00), IsNotNull(i_item_sk)] ReadSchema: struct -(23) CometFilter +(24) CometFilter Input [2]: [i_item_sk#17, i_current_price#18] Condition : ((isnotnull(i_current_price#18) AND (i_current_price#18 > 50.00)) AND isnotnull(i_item_sk#17)) -(24) CometProject +(25) CometProject Input [2]: [i_item_sk#17, i_current_price#18] Arguments: [i_item_sk#17], [i_item_sk#17] -(25) ColumnarToRow [codegen id : 3] +(26) ColumnarToRow [codegen id : 7] Input [1]: [i_item_sk#17] -(26) BroadcastExchange +(27) BroadcastExchange Input [1]: [i_item_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(27) BroadcastHashJoin [codegen id : 5] +(28) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#17] Join type: Inner Join condition: None -(28) Project [codegen id : 5] +(29) Project [codegen id : 9] Output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16, i_item_sk#17] @@ -240,43 +247,43 @@ Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_channel_tv), EqualTo(p_channel_tv,N), IsNotNull(p_promo_sk)] ReadSchema: struct -(30) CometFilter +(31) CometFilter Input [2]: [p_promo_sk#19, p_channel_tv#20] Condition : ((isnotnull(p_channel_tv#20) AND (p_channel_tv#20 = N)) AND isnotnull(p_promo_sk#19)) -(31) CometProject +(32) CometProject Input [2]: [p_promo_sk#19, p_channel_tv#20] Arguments: [p_promo_sk#19], [p_promo_sk#19] -(32) ColumnarToRow [codegen id : 4] +(33) ColumnarToRow [codegen id : 8] Input [1]: [p_promo_sk#19] -(33) BroadcastExchange +(34) BroadcastExchange Input [1]: [p_promo_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(34) BroadcastHashJoin [codegen id : 5] +(35) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_promo_sk#3] Right keys [1]: [p_promo_sk#19] Join type: Inner Join condition: None -(35) Project [codegen id : 5] +(36) Project [codegen id : 9] Output [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16, p_promo_sk#19] -(36) HashAggregate [codegen id : 5] +(37) HashAggregate [codegen id : 9] Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] Keys [1]: [s_store_id#16] Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] Aggregate Attributes [5]: [sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] Results [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] -(37) Exchange +(38) Exchange Input [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] Arguments: hashpartitioning(s_store_id#16, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(38) HashAggregate [codegen id : 6] +(39) HashAggregate [codegen id : 10] Input [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] Keys [1]: [s_store_id#16] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] @@ -291,17 +298,20 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#45), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] ReadSchema: struct -(40) CometFilter +(41) CometFilter Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] Condition : ((isnotnull(cs_catalog_page_sk#39) AND isnotnull(cs_item_sk#40)) AND isnotnull(cs_promo_sk#41)) -(41) CometExchange +(42) ColumnarToRow [codegen id : 11] +Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] + +(43) Exchange Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Arguments: hashpartitioning(cs_item_sk#40, cs_order_number#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +Arguments: hashpartitioning(cs_item_sk#40, cs_order_number#42, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(42) CometSort +(44) Sort [codegen id : 12] Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Arguments: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45], [cs_item_sk#40 ASC NULLS FIRST, cs_order_number#42 ASC NULLS FIRST] +Arguments: [cs_item_sk#40 ASC NULLS FIRST, cs_order_number#42 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.catalog_returns Output [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] @@ -310,44 +320,45 @@ Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(44) CometFilter +(46) CometFilter Input [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] Condition : (isnotnull(cr_item_sk#47) AND isnotnull(cr_order_number#48)) -(45) CometProject +(47) CometProject Input [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] Arguments: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50], [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -(46) CometExchange +(48) ColumnarToRow [codegen id : 13] +Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] + +(49) Exchange Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -Arguments: hashpartitioning(cr_item_sk#47, cr_order_number#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +Arguments: hashpartitioning(cr_item_sk#47, cr_order_number#48, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(47) CometSort +(50) Sort [codegen id : 14] Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -Arguments: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50], [cr_item_sk#47 ASC NULLS FIRST, cr_order_number#48 ASC NULLS FIRST] +Arguments: [cr_item_sk#47 ASC NULLS FIRST, cr_order_number#48 ASC NULLS FIRST], false, 0 -(48) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Right output [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -Arguments: [cs_item_sk#40, cs_order_number#42], [cr_item_sk#47, cr_order_number#48], LeftOuter +(51) SortMergeJoin [codegen id : 19] +Left keys [2]: [cs_item_sk#40, cs_order_number#42] +Right keys [2]: [cr_item_sk#47, cr_order_number#48] +Join type: LeftOuter +Join condition: None -(49) CometProject +(52) Project [codegen id : 19] +Output [8]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50] Input [11]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -Arguments: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50], [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50] - -(50) ColumnarToRow [codegen id : 11] -Input [8]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50] -(51) ReusedExchange [Reuses operator id: 109] +(53) ReusedExchange [Reuses operator id: 112] Output [1]: [d_date_sk#52] -(52) BroadcastHashJoin [codegen id : 11] +(54) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_sold_date_sk#45] Right keys [1]: [d_date_sk#52] Join type: Inner Join condition: None -(53) Project [codegen id : 11] +(55) Project [codegen id : 19] Output [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50] Input [9]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50, d_date_sk#52] @@ -358,65 +369,65 @@ Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(55) CometFilter +(57) CometFilter Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] Condition : isnotnull(cp_catalog_page_sk#53) -(56) ColumnarToRow [codegen id : 8] +(58) ColumnarToRow [codegen id : 16] Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -(57) BroadcastExchange +(59) BroadcastExchange Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(58) BroadcastHashJoin [codegen id : 11] +(60) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_catalog_page_sk#39] Right keys [1]: [cp_catalog_page_sk#53] Join type: Inner Join condition: None -(59) Project [codegen id : 11] +(61) Project [codegen id : 19] Output [7]: [cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] Input [9]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_sk#53, cp_catalog_page_id#54] -(60) ReusedExchange [Reuses operator id: 26] +(62) ReusedExchange [Reuses operator id: 27] Output [1]: [i_item_sk#55] -(61) BroadcastHashJoin [codegen id : 11] +(63) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_item_sk#40] Right keys [1]: [i_item_sk#55] Join type: Inner Join condition: None -(62) Project [codegen id : 11] +(64) Project [codegen id : 19] Output [6]: [cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] Input [8]: [cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54, i_item_sk#55] -(63) ReusedExchange [Reuses operator id: 33] +(65) ReusedExchange [Reuses operator id: 34] Output [1]: [p_promo_sk#56] -(64) BroadcastHashJoin [codegen id : 11] +(66) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_promo_sk#41] Right keys [1]: [p_promo_sk#56] Join type: Inner Join condition: None -(65) Project [codegen id : 11] +(67) Project [codegen id : 19] Output [5]: [cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] Input [7]: [cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54, p_promo_sk#56] -(66) HashAggregate [codegen id : 11] +(68) HashAggregate [codegen id : 19] Input [5]: [cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] Keys [1]: [cp_catalog_page_id#54] Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#43)), partial_sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))] Aggregate Attributes [5]: [sum#57, sum#58, isEmpty#59, sum#60, isEmpty#61] Results [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] -(67) Exchange +(69) Exchange Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] Arguments: hashpartitioning(cp_catalog_page_id#54, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(68) HashAggregate [codegen id : 12] +(70) HashAggregate [codegen id : 20] Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] Keys [1]: [cp_catalog_page_id#54] Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#43)), sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00)), sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))] @@ -431,17 +442,20 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct -(70) CometFilter +(72) CometFilter Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] Condition : ((isnotnull(ws_web_site_sk#76) AND isnotnull(ws_item_sk#75)) AND isnotnull(ws_promo_sk#77)) -(71) CometExchange +(73) ColumnarToRow [codegen id : 21] +Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] + +(74) Exchange Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Arguments: hashpartitioning(ws_item_sk#75, ws_order_number#78, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] +Arguments: hashpartitioning(ws_item_sk#75, ws_order_number#78, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(72) CometSort +(75) Sort [codegen id : 22] Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Arguments: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81], [ws_item_sk#75 ASC NULLS FIRST, ws_order_number#78 ASC NULLS FIRST] +Arguments: [ws_item_sk#75 ASC NULLS FIRST, ws_order_number#78 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.web_returns Output [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] @@ -450,44 +464,45 @@ Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct -(74) CometFilter +(77) CometFilter Input [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] Condition : (isnotnull(wr_item_sk#83) AND isnotnull(wr_order_number#84)) -(75) CometProject +(78) CometProject Input [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] Arguments: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86], [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -(76) CometExchange +(79) ColumnarToRow [codegen id : 23] Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -Arguments: hashpartitioning(wr_item_sk#83, wr_order_number#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] -(77) CometSort +(80) Exchange Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -Arguments: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86], [wr_item_sk#83 ASC NULLS FIRST, wr_order_number#84 ASC NULLS FIRST] +Arguments: hashpartitioning(wr_item_sk#83, wr_order_number#84, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(78) CometSortMergeJoin -Left output [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Right output [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -Arguments: [ws_item_sk#75, ws_order_number#78], [wr_item_sk#83, wr_order_number#84], LeftOuter +(81) Sort [codegen id : 24] +Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] +Arguments: [wr_item_sk#83 ASC NULLS FIRST, wr_order_number#84 ASC NULLS FIRST], false, 0 -(79) CometProject -Input [11]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -Arguments: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86], [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86] +(82) SortMergeJoin [codegen id : 29] +Left keys [2]: [ws_item_sk#75, ws_order_number#78] +Right keys [2]: [wr_item_sk#83, wr_order_number#84] +Join type: LeftOuter +Join condition: None -(80) ColumnarToRow [codegen id : 17] -Input [8]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86] +(83) Project [codegen id : 29] +Output [8]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86] +Input [11]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -(81) ReusedExchange [Reuses operator id: 109] +(84) ReusedExchange [Reuses operator id: 112] Output [1]: [d_date_sk#88] -(82) BroadcastHashJoin [codegen id : 17] +(85) BroadcastHashJoin [codegen id : 29] Left keys [1]: [ws_sold_date_sk#81] Right keys [1]: [d_date_sk#88] Join type: Inner Join condition: None -(83) Project [codegen id : 17] +(86) Project [codegen id : 29] Output [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86] Input [9]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86, d_date_sk#88] @@ -498,107 +513,107 @@ Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(85) CometFilter +(88) CometFilter Input [2]: [web_site_sk#89, web_site_id#90] Condition : isnotnull(web_site_sk#89) -(86) ColumnarToRow [codegen id : 14] +(89) ColumnarToRow [codegen id : 26] Input [2]: [web_site_sk#89, web_site_id#90] -(87) BroadcastExchange +(90) BroadcastExchange Input [2]: [web_site_sk#89, web_site_id#90] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -(88) BroadcastHashJoin [codegen id : 17] +(91) BroadcastHashJoin [codegen id : 29] Left keys [1]: [ws_web_site_sk#76] Right keys [1]: [web_site_sk#89] Join type: Inner Join condition: None -(89) Project [codegen id : 17] +(92) Project [codegen id : 29] Output [7]: [ws_item_sk#75, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] Input [9]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_sk#89, web_site_id#90] -(90) ReusedExchange [Reuses operator id: 26] +(93) ReusedExchange [Reuses operator id: 27] Output [1]: [i_item_sk#91] -(91) BroadcastHashJoin [codegen id : 17] +(94) BroadcastHashJoin [codegen id : 29] Left keys [1]: [ws_item_sk#75] Right keys [1]: [i_item_sk#91] Join type: Inner Join condition: None -(92) Project [codegen id : 17] +(95) Project [codegen id : 29] Output [6]: [ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] Input [8]: [ws_item_sk#75, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90, i_item_sk#91] -(93) ReusedExchange [Reuses operator id: 33] +(96) ReusedExchange [Reuses operator id: 34] Output [1]: [p_promo_sk#92] -(94) BroadcastHashJoin [codegen id : 17] +(97) BroadcastHashJoin [codegen id : 29] Left keys [1]: [ws_promo_sk#77] Right keys [1]: [p_promo_sk#92] Join type: Inner Join condition: None -(95) Project [codegen id : 17] +(98) Project [codegen id : 29] Output [5]: [ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] Input [7]: [ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90, p_promo_sk#92] -(96) HashAggregate [codegen id : 17] +(99) HashAggregate [codegen id : 29] Input [5]: [ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] Keys [1]: [web_site_id#90] Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#79)), partial_sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))] Aggregate Attributes [5]: [sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] Results [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -(97) Exchange +(100) Exchange Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] Arguments: hashpartitioning(web_site_id#90, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(98) HashAggregate [codegen id : 18] +(101) HashAggregate [codegen id : 30] Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] Keys [1]: [web_site_id#90] Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#79)), sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00)), sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))] Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#79))#103, sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00))#104, sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))#105] Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#79))#103,17,2) AS sales#106, sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00))#104 AS returns#107, sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))#105 AS profit#108, web channel AS channel#109, concat(web_site, web_site_id#90) AS id#110] -(99) Union +(102) Union -(100) Expand [codegen id : 19] +(103) Expand [codegen id : 31] Input [5]: [sales#34, returns#35, profit#36, channel#37, id#38] Arguments: [[sales#34, returns#35, profit#36, channel#37, id#38, 0], [sales#34, returns#35, profit#36, channel#37, null, 1], [sales#34, returns#35, profit#36, null, null, 3]], [sales#34, returns#35, profit#36, channel#111, id#112, spark_grouping_id#113] -(101) HashAggregate [codegen id : 19] +(104) HashAggregate [codegen id : 31] Input [6]: [sales#34, returns#35, profit#36, channel#111, id#112, spark_grouping_id#113] Keys [3]: [channel#111, id#112, spark_grouping_id#113] Functions [3]: [partial_sum(sales#34), partial_sum(returns#35), partial_sum(profit#36)] Aggregate Attributes [6]: [sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] Results [9]: [channel#111, id#112, spark_grouping_id#113, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] -(102) Exchange +(105) Exchange Input [9]: [channel#111, id#112, spark_grouping_id#113, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] Arguments: hashpartitioning(channel#111, id#112, spark_grouping_id#113, 5), ENSURE_REQUIREMENTS, [plan_id=15] -(103) HashAggregate [codegen id : 20] +(106) HashAggregate [codegen id : 32] Input [9]: [channel#111, id#112, spark_grouping_id#113, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] Keys [3]: [channel#111, id#112, spark_grouping_id#113] Functions [3]: [sum(sales#34), sum(returns#35), sum(profit#36)] Aggregate Attributes [3]: [sum(sales#34)#126, sum(returns#35)#127, sum(profit#36)#128] Results [5]: [channel#111, id#112, sum(sales#34)#126 AS sales#129, sum(returns#35)#127 AS returns#130, sum(profit#36)#128 AS profit#131] -(104) TakeOrderedAndProject +(107) TakeOrderedAndProject Input [5]: [channel#111, id#112, sales#129, returns#130, profit#131] Arguments: 100, [channel#111 ASC NULLS FIRST, id#112 ASC NULLS FIRST], [channel#111, id#112, sales#129, returns#130, profit#131] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (109) -+- * ColumnarToRow (108) - +- CometProject (107) - +- CometFilter (106) - +- CometScan parquet spark_catalog.default.date_dim (105) +BroadcastExchange (112) ++- * ColumnarToRow (111) + +- CometProject (110) + +- CometFilter (109) + +- CometScan parquet spark_catalog.default.date_dim (108) (unknown) Scan parquet spark_catalog.default.date_dim @@ -608,23 +623,23 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-22), IsNotNull(d_date_sk)] ReadSchema: struct -(106) CometFilter +(109) CometFilter Input [2]: [d_date_sk#14, d_date#132] Condition : (((isnotnull(d_date#132) AND (d_date#132 >= 2000-08-23)) AND (d_date#132 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) -(107) CometProject +(110) CometProject Input [2]: [d_date_sk#14, d_date#132] Arguments: [d_date_sk#14], [d_date_sk#14] -(108) ColumnarToRow [codegen id : 1] +(111) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(109) BroadcastExchange +(112) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] -Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 69 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 71 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt index 8fc66cbe4..7e257bdc6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt @@ -1,18 +1,18 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (20) + WholeStageCodegen (32) HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (19) + WholeStageCodegen (31) HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] Expand [sales,returns,profit,channel,id] InputAdapter Union - WholeStageCodegen (6) + WholeStageCodegen (10) HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [s_store_id] #2 - WholeStageCodegen (5) + WholeStageCodegen (9) HashAggregate [s_store_id,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] Project [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] BroadcastHashJoin [ss_promo_sk,p_promo_sk] @@ -22,39 +22,49 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_ticket_number] - CometExchange [ss_item_sk,ss_ticket_number] #3 - CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_item_sk,sr_ticket_number] #5 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + Project [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] + SortMergeJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + InputAdapter + WholeStageCodegen (2) + Sort [ss_item_sk,ss_ticket_number] + InputAdapter + Exchange [ss_item_sk,ss_ticket_number] #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + WholeStageCodegen (4) + Sort [sr_item_sk,sr_ticket_number] + InputAdapter + Exchange [sr_item_sk,sr_ticket_number] #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometFilter [sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #6 - WholeStageCodegen (2) + WholeStageCodegen (6) ColumnarToRow InputAdapter CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #7 - WholeStageCodegen (3) + WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [i_item_sk] @@ -62,17 +72,17 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] InputAdapter BroadcastExchange #8 - WholeStageCodegen (4) + WholeStageCodegen (8) ColumnarToRow InputAdapter CometProject [p_promo_sk] CometFilter [p_channel_tv,p_promo_sk] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - WholeStageCodegen (12) + WholeStageCodegen (20) HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [cp_catalog_page_id] #9 - WholeStageCodegen (11) + WholeStageCodegen (19) HashAggregate [cp_catalog_page_id,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] Project [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] BroadcastHashJoin [cs_promo_sk,p_promo_sk] @@ -82,25 +92,35 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] BroadcastHashJoin [cs_catalog_page_sk,cp_catalog_page_sk] Project [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number] - CometExchange [cs_item_sk,cs_order_number] #10 - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #11 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + Project [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] + SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] + InputAdapter + WholeStageCodegen (12) + Sort [cs_item_sk,cs_order_number] + InputAdapter + Exchange [cs_item_sk,cs_order_number] #10 + WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + WholeStageCodegen (14) + Sort [cr_item_sk,cr_order_number] + InputAdapter + Exchange [cr_item_sk,cr_order_number] #11 + WholeStageCodegen (13) + ColumnarToRow + InputAdapter + CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometFilter [cr_item_sk,cr_order_number] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #12 - WholeStageCodegen (8) + WholeStageCodegen (16) ColumnarToRow InputAdapter CometFilter [cp_catalog_page_sk] @@ -109,11 +129,11 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedExchange [i_item_sk] #7 InputAdapter ReusedExchange [p_promo_sk] #8 - WholeStageCodegen (18) + WholeStageCodegen (30) HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [web_site_id] #13 - WholeStageCodegen (17) + WholeStageCodegen (29) HashAggregate [web_site_id,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] Project [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] BroadcastHashJoin [ws_promo_sk,p_promo_sk] @@ -123,25 +143,35 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] BroadcastHashJoin [ws_web_site_sk,web_site_sk] Project [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] - CometSort [ws_item_sk,ws_order_number] - CometExchange [ws_item_sk,ws_order_number] #14 - CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [wr_item_sk,wr_order_number] - CometExchange [wr_item_sk,wr_order_number] #15 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + Project [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] + SortMergeJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] + InputAdapter + WholeStageCodegen (22) + Sort [ws_item_sk,ws_order_number] + InputAdapter + Exchange [ws_item_sk,ws_order_number] #14 + WholeStageCodegen (21) + ColumnarToRow + InputAdapter + CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + WholeStageCodegen (24) + Sort [wr_item_sk,wr_order_number] + InputAdapter + Exchange [wr_item_sk,wr_order_number] #15 + WholeStageCodegen (23) + ColumnarToRow + InputAdapter + CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometFilter [wr_item_sk,wr_order_number] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #16 - WholeStageCodegen (14) + WholeStageCodegen (26) ColumnarToRow InputAdapter CometFilter [web_site_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt index 167181142..4fb9fc546 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt @@ -27,9 +27,9 @@ Input [1]: [r_reason_sk#1] Subquery:1 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#2, [id=#3] * Project (12) -+- * ColumnarToRow (11) - +- CometHashAggregate (10) - +- CometExchange (9) ++- * HashAggregate (11) + +- Exchange (10) + +- * ColumnarToRow (9) +- CometHashAggregate (8) +- CometProject (7) +- CometFilter (6) @@ -56,21 +56,23 @@ Input [2]: [ss_ext_discount_amt#18, ss_net_paid#19] Keys: [] Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#18)), partial_avg(UnscaledValue(ss_net_paid#19))] -(9) CometExchange +(9) ColumnarToRow [codegen id : 1] Input [5]: [count#21, sum#22, count#23, sum#24, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(10) CometHashAggregate +(10) Exchange +Input [5]: [count#21, sum#22, count#23, sum#24, count#25] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] + +(11) HashAggregate [codegen id : 2] Input [5]: [count#21, sum#22, count#23, sum#24, count#25] Keys: [] Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#18)), avg(UnscaledValue(ss_net_paid#19))] +Aggregate Attributes [3]: [count(1)#26, avg(UnscaledValue(ss_ext_discount_amt#18))#27, avg(UnscaledValue(ss_net_paid#19))#28] +Results [3]: [count(1)#26 AS count(1)#29, cast((avg(UnscaledValue(ss_ext_discount_amt#18))#27 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#30, cast((avg(UnscaledValue(ss_net_paid#19))#28 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#31] -(11) ColumnarToRow [codegen id : 1] -Input [3]: [count(1)#26, avg(ss_ext_discount_amt)#27, avg(ss_net_paid)#28] - -(12) Project [codegen id : 1] -Output [1]: [named_struct(count(1), count(1)#26, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#27, avg(ss_net_paid), avg(ss_net_paid)#28) AS mergedValue#29] -Input [3]: [count(1)#26, avg(ss_ext_discount_amt)#27, avg(ss_net_paid)#28] +(12) Project [codegen id : 2] +Output [1]: [named_struct(count(1), count(1)#29, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#30, avg(ss_net_paid), avg(ss_net_paid)#31) AS mergedValue#32] +Input [3]: [count(1)#29, avg(ss_ext_discount_amt)#30, avg(ss_net_paid)#31] Subquery:2 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] @@ -78,9 +80,9 @@ Subquery:3 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#5, [id=#6] * Project (20) -+- * ColumnarToRow (19) - +- CometHashAggregate (18) - +- CometExchange (17) ++- * HashAggregate (19) + +- Exchange (18) + +- * ColumnarToRow (17) +- CometHashAggregate (16) +- CometProject (15) +- CometFilter (14) @@ -88,40 +90,42 @@ Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] +Output [4]: [ss_quantity#33, ss_ext_discount_amt#34, ss_net_paid#35, ss_sold_date_sk#36] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct (14) CometFilter -Input [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] -Condition : ((isnotnull(ss_quantity#30) AND (ss_quantity#30 >= 21)) AND (ss_quantity#30 <= 40)) +Input [4]: [ss_quantity#33, ss_ext_discount_amt#34, ss_net_paid#35, ss_sold_date_sk#36] +Condition : ((isnotnull(ss_quantity#33) AND (ss_quantity#33 >= 21)) AND (ss_quantity#33 <= 40)) (15) CometProject -Input [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] -Arguments: [ss_ext_discount_amt#31, ss_net_paid#32], [ss_ext_discount_amt#31, ss_net_paid#32] +Input [4]: [ss_quantity#33, ss_ext_discount_amt#34, ss_net_paid#35, ss_sold_date_sk#36] +Arguments: [ss_ext_discount_amt#34, ss_net_paid#35], [ss_ext_discount_amt#34, ss_net_paid#35] (16) CometHashAggregate -Input [2]: [ss_ext_discount_amt#31, ss_net_paid#32] +Input [2]: [ss_ext_discount_amt#34, ss_net_paid#35] Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#31)), partial_avg(UnscaledValue(ss_net_paid#32))] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#34)), partial_avg(UnscaledValue(ss_net_paid#35))] -(17) CometExchange -Input [5]: [count#34, sum#35, count#36, sum#37, count#38] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(17) ColumnarToRow [codegen id : 1] +Input [5]: [count#37, sum#38, count#39, sum#40, count#41] -(18) CometHashAggregate -Input [5]: [count#34, sum#35, count#36, sum#37, count#38] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#31)), avg(UnscaledValue(ss_net_paid#32))] +(18) Exchange +Input [5]: [count#37, sum#38, count#39, sum#40, count#41] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] -(19) ColumnarToRow [codegen id : 1] -Input [3]: [count(1)#39, avg(ss_ext_discount_amt)#40, avg(ss_net_paid)#41] +(19) HashAggregate [codegen id : 2] +Input [5]: [count#37, sum#38, count#39, sum#40, count#41] +Keys: [] +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#34)), avg(UnscaledValue(ss_net_paid#35))] +Aggregate Attributes [3]: [count(1)#42, avg(UnscaledValue(ss_ext_discount_amt#34))#43, avg(UnscaledValue(ss_net_paid#35))#44] +Results [3]: [count(1)#42 AS count(1)#45, cast((avg(UnscaledValue(ss_ext_discount_amt#34))#43 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#46, cast((avg(UnscaledValue(ss_net_paid#35))#44 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#47] -(20) Project [codegen id : 1] -Output [1]: [named_struct(count(1), count(1)#39, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#40, avg(ss_net_paid), avg(ss_net_paid)#41) AS mergedValue#42] -Input [3]: [count(1)#39, avg(ss_ext_discount_amt)#40, avg(ss_net_paid)#41] +(20) Project [codegen id : 2] +Output [1]: [named_struct(count(1), count(1)#45, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#46, avg(ss_net_paid), avg(ss_net_paid)#47) AS mergedValue#48] +Input [3]: [count(1)#45, avg(ss_ext_discount_amt)#46, avg(ss_net_paid)#47] Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] @@ -129,9 +133,9 @@ Subquery:6 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#8, [id=#9] * Project (28) -+- * ColumnarToRow (27) - +- CometHashAggregate (26) - +- CometExchange (25) ++- * HashAggregate (27) + +- Exchange (26) + +- * ColumnarToRow (25) +- CometHashAggregate (24) +- CometProject (23) +- CometFilter (22) @@ -139,40 +143,42 @@ Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] +Output [4]: [ss_quantity#49, ss_ext_discount_amt#50, ss_net_paid#51, ss_sold_date_sk#52] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct (22) CometFilter -Input [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] -Condition : ((isnotnull(ss_quantity#43) AND (ss_quantity#43 >= 41)) AND (ss_quantity#43 <= 60)) +Input [4]: [ss_quantity#49, ss_ext_discount_amt#50, ss_net_paid#51, ss_sold_date_sk#52] +Condition : ((isnotnull(ss_quantity#49) AND (ss_quantity#49 >= 41)) AND (ss_quantity#49 <= 60)) (23) CometProject -Input [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] -Arguments: [ss_ext_discount_amt#44, ss_net_paid#45], [ss_ext_discount_amt#44, ss_net_paid#45] +Input [4]: [ss_quantity#49, ss_ext_discount_amt#50, ss_net_paid#51, ss_sold_date_sk#52] +Arguments: [ss_ext_discount_amt#50, ss_net_paid#51], [ss_ext_discount_amt#50, ss_net_paid#51] (24) CometHashAggregate -Input [2]: [ss_ext_discount_amt#44, ss_net_paid#45] +Input [2]: [ss_ext_discount_amt#50, ss_net_paid#51] Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#44)), partial_avg(UnscaledValue(ss_net_paid#45))] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#50)), partial_avg(UnscaledValue(ss_net_paid#51))] -(25) CometExchange -Input [5]: [count#47, sum#48, count#49, sum#50, count#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +(25) ColumnarToRow [codegen id : 1] +Input [5]: [count#53, sum#54, count#55, sum#56, count#57] -(26) CometHashAggregate -Input [5]: [count#47, sum#48, count#49, sum#50, count#51] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#44)), avg(UnscaledValue(ss_net_paid#45))] +(26) Exchange +Input [5]: [count#53, sum#54, count#55, sum#56, count#57] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] -(27) ColumnarToRow [codegen id : 1] -Input [3]: [count(1)#52, avg(ss_ext_discount_amt)#53, avg(ss_net_paid)#54] +(27) HashAggregate [codegen id : 2] +Input [5]: [count#53, sum#54, count#55, sum#56, count#57] +Keys: [] +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#50)), avg(UnscaledValue(ss_net_paid#51))] +Aggregate Attributes [3]: [count(1)#58, avg(UnscaledValue(ss_ext_discount_amt#50))#59, avg(UnscaledValue(ss_net_paid#51))#60] +Results [3]: [count(1)#58 AS count(1)#61, cast((avg(UnscaledValue(ss_ext_discount_amt#50))#59 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#62, cast((avg(UnscaledValue(ss_net_paid#51))#60 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#63] -(28) Project [codegen id : 1] -Output [1]: [named_struct(count(1), count(1)#52, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#53, avg(ss_net_paid), avg(ss_net_paid)#54) AS mergedValue#55] -Input [3]: [count(1)#52, avg(ss_ext_discount_amt)#53, avg(ss_net_paid)#54] +(28) Project [codegen id : 2] +Output [1]: [named_struct(count(1), count(1)#61, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#62, avg(ss_net_paid), avg(ss_net_paid)#63) AS mergedValue#64] +Input [3]: [count(1)#61, avg(ss_ext_discount_amt)#62, avg(ss_net_paid)#63] Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] @@ -180,9 +186,9 @@ Subquery:9 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] * Project (36) -+- * ColumnarToRow (35) - +- CometHashAggregate (34) - +- CometExchange (33) ++- * HashAggregate (35) + +- Exchange (34) + +- * ColumnarToRow (33) +- CometHashAggregate (32) +- CometProject (31) +- CometFilter (30) @@ -190,40 +196,42 @@ Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] +Output [4]: [ss_quantity#65, ss_ext_discount_amt#66, ss_net_paid#67, ss_sold_date_sk#68] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct (30) CometFilter -Input [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] -Condition : ((isnotnull(ss_quantity#56) AND (ss_quantity#56 >= 61)) AND (ss_quantity#56 <= 80)) +Input [4]: [ss_quantity#65, ss_ext_discount_amt#66, ss_net_paid#67, ss_sold_date_sk#68] +Condition : ((isnotnull(ss_quantity#65) AND (ss_quantity#65 >= 61)) AND (ss_quantity#65 <= 80)) (31) CometProject -Input [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] -Arguments: [ss_ext_discount_amt#57, ss_net_paid#58], [ss_ext_discount_amt#57, ss_net_paid#58] +Input [4]: [ss_quantity#65, ss_ext_discount_amt#66, ss_net_paid#67, ss_sold_date_sk#68] +Arguments: [ss_ext_discount_amt#66, ss_net_paid#67], [ss_ext_discount_amt#66, ss_net_paid#67] (32) CometHashAggregate -Input [2]: [ss_ext_discount_amt#57, ss_net_paid#58] +Input [2]: [ss_ext_discount_amt#66, ss_net_paid#67] Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#57)), partial_avg(UnscaledValue(ss_net_paid#58))] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#66)), partial_avg(UnscaledValue(ss_net_paid#67))] -(33) CometExchange -Input [5]: [count#60, sum#61, count#62, sum#63, count#64] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(33) ColumnarToRow [codegen id : 1] +Input [5]: [count#69, sum#70, count#71, sum#72, count#73] -(34) CometHashAggregate -Input [5]: [count#60, sum#61, count#62, sum#63, count#64] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#57)), avg(UnscaledValue(ss_net_paid#58))] +(34) Exchange +Input [5]: [count#69, sum#70, count#71, sum#72, count#73] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] -(35) ColumnarToRow [codegen id : 1] -Input [3]: [count(1)#65, avg(ss_ext_discount_amt)#66, avg(ss_net_paid)#67] +(35) HashAggregate [codegen id : 2] +Input [5]: [count#69, sum#70, count#71, sum#72, count#73] +Keys: [] +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#66)), avg(UnscaledValue(ss_net_paid#67))] +Aggregate Attributes [3]: [count(1)#74, avg(UnscaledValue(ss_ext_discount_amt#66))#75, avg(UnscaledValue(ss_net_paid#67))#76] +Results [3]: [count(1)#74 AS count(1)#77, cast((avg(UnscaledValue(ss_ext_discount_amt#66))#75 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#78, cast((avg(UnscaledValue(ss_net_paid#67))#76 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#79] -(36) Project [codegen id : 1] -Output [1]: [named_struct(count(1), count(1)#65, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#66, avg(ss_net_paid), avg(ss_net_paid)#67) AS mergedValue#68] -Input [3]: [count(1)#65, avg(ss_ext_discount_amt)#66, avg(ss_net_paid)#67] +(36) Project [codegen id : 2] +Output [1]: [named_struct(count(1), count(1)#77, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#78, avg(ss_net_paid), avg(ss_net_paid)#79) AS mergedValue#80] +Input [3]: [count(1)#77, avg(ss_ext_discount_amt)#78, avg(ss_net_paid)#79] Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] @@ -231,9 +239,9 @@ Subquery:12 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#14, [id=#15] * Project (44) -+- * ColumnarToRow (43) - +- CometHashAggregate (42) - +- CometExchange (41) ++- * HashAggregate (43) + +- Exchange (42) + +- * ColumnarToRow (41) +- CometHashAggregate (40) +- CometProject (39) +- CometFilter (38) @@ -241,40 +249,42 @@ Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (unknown) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] +Output [4]: [ss_quantity#81, ss_ext_discount_amt#82, ss_net_paid#83, ss_sold_date_sk#84] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct (38) CometFilter -Input [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] -Condition : ((isnotnull(ss_quantity#69) AND (ss_quantity#69 >= 81)) AND (ss_quantity#69 <= 100)) +Input [4]: [ss_quantity#81, ss_ext_discount_amt#82, ss_net_paid#83, ss_sold_date_sk#84] +Condition : ((isnotnull(ss_quantity#81) AND (ss_quantity#81 >= 81)) AND (ss_quantity#81 <= 100)) (39) CometProject -Input [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] -Arguments: [ss_ext_discount_amt#70, ss_net_paid#71], [ss_ext_discount_amt#70, ss_net_paid#71] +Input [4]: [ss_quantity#81, ss_ext_discount_amt#82, ss_net_paid#83, ss_sold_date_sk#84] +Arguments: [ss_ext_discount_amt#82, ss_net_paid#83], [ss_ext_discount_amt#82, ss_net_paid#83] (40) CometHashAggregate -Input [2]: [ss_ext_discount_amt#70, ss_net_paid#71] +Input [2]: [ss_ext_discount_amt#82, ss_net_paid#83] Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#70)), partial_avg(UnscaledValue(ss_net_paid#71))] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#82)), partial_avg(UnscaledValue(ss_net_paid#83))] -(41) CometExchange -Input [5]: [count#73, sum#74, count#75, sum#76, count#77] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +(41) ColumnarToRow [codegen id : 1] +Input [5]: [count#85, sum#86, count#87, sum#88, count#89] -(42) CometHashAggregate -Input [5]: [count#73, sum#74, count#75, sum#76, count#77] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#70)), avg(UnscaledValue(ss_net_paid#71))] +(42) Exchange +Input [5]: [count#85, sum#86, count#87, sum#88, count#89] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5] -(43) ColumnarToRow [codegen id : 1] -Input [3]: [count(1)#78, avg(ss_ext_discount_amt)#79, avg(ss_net_paid)#80] +(43) HashAggregate [codegen id : 2] +Input [5]: [count#85, sum#86, count#87, sum#88, count#89] +Keys: [] +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#82)), avg(UnscaledValue(ss_net_paid#83))] +Aggregate Attributes [3]: [count(1)#90, avg(UnscaledValue(ss_ext_discount_amt#82))#91, avg(UnscaledValue(ss_net_paid#83))#92] +Results [3]: [count(1)#90 AS count(1)#93, cast((avg(UnscaledValue(ss_ext_discount_amt#82))#91 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#94, cast((avg(UnscaledValue(ss_net_paid#83))#92 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#95] -(44) Project [codegen id : 1] -Output [1]: [named_struct(count(1), count(1)#78, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#79, avg(ss_net_paid), avg(ss_net_paid)#80) AS mergedValue#81] -Input [3]: [count(1)#78, avg(ss_ext_discount_amt)#79, avg(ss_net_paid)#80] +(44) Project [codegen id : 2] +Output [1]: [named_struct(count(1), count(1)#93, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#94, avg(ss_net_paid), avg(ss_net_paid)#95) AS mergedValue#96] +Input [3]: [count(1)#93, avg(ss_ext_discount_amt)#94, avg(ss_net_paid)#95] Subquery:14 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt index a1fd64ecd..c54606f6e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt @@ -1,68 +1,78 @@ WholeStageCodegen (1) Project Subquery #1 - WholeStageCodegen (1) + WholeStageCodegen (2) Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] - ColumnarToRow + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] InputAdapter - CometHashAggregate [count,sum,count,sum,count] - CometExchange #1 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + Exchange #1 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometProject [ss_ext_discount_amt,ss_net_paid] + CometFilter [ss_quantity] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #1 ReusedSubquery [mergedValue] #1 Subquery #2 - WholeStageCodegen (1) + WholeStageCodegen (2) Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] - ColumnarToRow + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] InputAdapter - CometHashAggregate [count,sum,count,sum,count] - CometExchange #2 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + Exchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometProject [ss_ext_discount_amt,ss_net_paid] + CometFilter [ss_quantity] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #2 ReusedSubquery [mergedValue] #2 Subquery #3 - WholeStageCodegen (1) + WholeStageCodegen (2) Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] - ColumnarToRow + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] InputAdapter - CometHashAggregate [count,sum,count,sum,count] - CometExchange #3 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + Exchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometProject [ss_ext_discount_amt,ss_net_paid] + CometFilter [ss_quantity] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #3 ReusedSubquery [mergedValue] #3 Subquery #4 - WholeStageCodegen (1) + WholeStageCodegen (2) Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] - ColumnarToRow + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] InputAdapter - CometHashAggregate [count,sum,count,sum,count] - CometExchange #4 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + Exchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometProject [ss_ext_discount_amt,ss_net_paid] + CometFilter [ss_quantity] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #4 ReusedSubquery [mergedValue] #4 Subquery #5 - WholeStageCodegen (1) + WholeStageCodegen (2) Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] - ColumnarToRow + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] InputAdapter - CometHashAggregate [count,sum,count,sum,count] - CometExchange #5 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + Exchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometProject [ss_ext_discount_amt,ss_net_paid] + CometFilter [ss_quantity] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #5 ReusedSubquery [mergedValue] #5 ColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt index 4b396dbc3..ee17b5291 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt @@ -1,27 +1,28 @@ == Physical Plan == -TakeOrderedAndProject (23) -+- * HashAggregate (22) - +- Exchange (21) - +- * HashAggregate (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * ColumnarToRow (12) - : +- CometProject (11) - : +- CometSortMergeJoin (10) - : :- CometSort (4) - : : +- CometExchange (3) - : : +- CometProject (2) - : : +- CometScan parquet spark_catalog.default.store_sales (1) - : +- CometSort (9) - : +- CometExchange (8) - : +- CometProject (7) - : +- CometFilter (6) - : +- CometScan parquet spark_catalog.default.store_returns (5) - +- BroadcastExchange (17) - +- * ColumnarToRow (16) - +- CometProject (15) - +- CometFilter (14) - +- CometScan parquet spark_catalog.default.reason (13) +TakeOrderedAndProject (24) ++- * HashAggregate (23) + +- Exchange (22) + +- * HashAggregate (21) + +- * Project (20) + +- * BroadcastHashJoin Inner BuildRight (19) + :- * Project (13) + : +- * SortMergeJoin Inner (12) + : :- * Sort (5) + : : +- Exchange (4) + : : +- * ColumnarToRow (3) + : : +- CometProject (2) + : : +- CometScan parquet spark_catalog.default.store_sales (1) + : +- * Sort (11) + : +- Exchange (10) + : +- * ColumnarToRow (9) + : +- CometProject (8) + : +- CometFilter (7) + : +- CometScan parquet spark_catalog.default.store_returns (6) + +- BroadcastExchange (18) + +- * ColumnarToRow (17) + +- CometProject (16) + +- CometFilter (15) + +- CometScan parquet spark_catalog.default.reason (14) (unknown) Scan parquet spark_catalog.default.store_sales @@ -34,13 +35,16 @@ ReadSchema: struct -(6) CometFilter +(7) CometFilter Input [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] Condition : ((isnotnull(sr_item_sk#7) AND isnotnull(sr_ticket_number#9)) AND isnotnull(sr_reason_sk#8)) -(7) CometProject +(8) CometProject Input [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] Arguments: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10], [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -(8) CometExchange +(9) ColumnarToRow [codegen id : 3] Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: hashpartitioning(sr_item_sk#7, sr_ticket_number#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(9) CometSort +(10) Exchange Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10], [sr_item_sk#7 ASC NULLS FIRST, sr_ticket_number#9 ASC NULLS FIRST] +Arguments: hashpartitioning(sr_item_sk#7, sr_ticket_number#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(10) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] -Right output [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [ss_item_sk#1, ss_ticket_number#3], [sr_item_sk#7, sr_ticket_number#9], Inner +(11) Sort [codegen id : 4] +Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] +Arguments: [sr_item_sk#7 ASC NULLS FIRST, sr_ticket_number#9 ASC NULLS FIRST], false, 0 -(11) CometProject -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10], [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10] +(12) SortMergeJoin [codegen id : 6] +Left keys [2]: [ss_item_sk#1, ss_ticket_number#3] +Right keys [2]: [sr_item_sk#7, sr_ticket_number#9] +Join type: Inner +Join condition: None -(12) ColumnarToRow [codegen id : 2] -Input [5]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10] +(13) Project [codegen id : 6] +Output [5]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10] +Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] (unknown) Scan parquet spark_catalog.default.reason Output [2]: [r_reason_sk#12, r_reason_desc#13] @@ -84,50 +89,50 @@ Location [not included in comparison]/{warehouse_dir}/reason] PushedFilters: [IsNotNull(r_reason_desc), EqualTo(r_reason_desc,reason 28 ), IsNotNull(r_reason_sk)] ReadSchema: struct -(14) CometFilter +(15) CometFilter Input [2]: [r_reason_sk#12, r_reason_desc#13] Condition : ((isnotnull(r_reason_desc#13) AND (r_reason_desc#13 = reason 28 )) AND isnotnull(r_reason_sk#12)) -(15) CometProject +(16) CometProject Input [2]: [r_reason_sk#12, r_reason_desc#13] Arguments: [r_reason_sk#12], [r_reason_sk#12] -(16) ColumnarToRow [codegen id : 1] +(17) ColumnarToRow [codegen id : 5] Input [1]: [r_reason_sk#12] -(17) BroadcastExchange +(18) BroadcastExchange Input [1]: [r_reason_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(18) BroadcastHashJoin [codegen id : 2] +(19) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_reason_sk#8] Right keys [1]: [r_reason_sk#12] Join type: Inner Join condition: None -(19) Project [codegen id : 2] +(20) Project [codegen id : 6] Output [2]: [ss_customer_sk#2, CASE WHEN isnotnull(sr_return_quantity#10) THEN (cast((ss_quantity#4 - sr_return_quantity#10) as decimal(10,0)) * ss_sales_price#5) ELSE (cast(ss_quantity#4 as decimal(10,0)) * ss_sales_price#5) END AS act_sales#14] Input [6]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10, r_reason_sk#12] -(20) HashAggregate [codegen id : 2] +(21) HashAggregate [codegen id : 6] Input [2]: [ss_customer_sk#2, act_sales#14] Keys [1]: [ss_customer_sk#2] Functions [1]: [partial_sum(act_sales#14)] Aggregate Attributes [2]: [sum#15, isEmpty#16] Results [3]: [ss_customer_sk#2, sum#17, isEmpty#18] -(21) Exchange +(22) Exchange Input [3]: [ss_customer_sk#2, sum#17, isEmpty#18] Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(22) HashAggregate [codegen id : 3] +(23) HashAggregate [codegen id : 7] Input [3]: [ss_customer_sk#2, sum#17, isEmpty#18] Keys [1]: [ss_customer_sk#2] Functions [1]: [sum(act_sales#14)] Aggregate Attributes [1]: [sum(act_sales#14)#19] Results [2]: [ss_customer_sk#2, sum(act_sales#14)#19 AS sumsales#20] -(23) TakeOrderedAndProject +(24) TakeOrderedAndProject Input [2]: [ss_customer_sk#2, sumsales#20] Arguments: 100, [sumsales#20 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], [ss_customer_sk#2, sumsales#20] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt index 6178d5a37..3ec7ac7b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt @@ -1,28 +1,38 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] - WholeStageCodegen (3) + WholeStageCodegen (7) HashAggregate [ss_customer_sk,sum,isEmpty] [sum(act_sales),sumsales,sum,isEmpty] InputAdapter Exchange [ss_customer_sk] #1 - WholeStageCodegen (2) + WholeStageCodegen (6) HashAggregate [ss_customer_sk,act_sales] [sum,isEmpty,sum,isEmpty] Project [ss_customer_sk,sr_return_quantity,ss_quantity,ss_sales_price] BroadcastHashJoin [sr_reason_sk,r_reason_sk] - ColumnarToRow - InputAdapter - CometProject [ss_customer_sk,ss_quantity,ss_sales_price,sr_reason_sk,sr_return_quantity] - CometSortMergeJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_ticket_number] - CometExchange [ss_item_sk,ss_ticket_number] #2 - CometProject [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_item_sk,sr_ticket_number] #3 - CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometFilter [sr_item_sk,sr_ticket_number,sr_reason_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + Project [ss_customer_sk,ss_quantity,ss_sales_price,sr_reason_sk,sr_return_quantity] + SortMergeJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + InputAdapter + WholeStageCodegen (2) + Sort [ss_item_sk,ss_ticket_number] + InputAdapter + Exchange [ss_item_sk,ss_ticket_number] #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] + InputAdapter + WholeStageCodegen (4) + Sort [sr_item_sk,sr_ticket_number] + InputAdapter + Exchange [sr_item_sk,sr_ticket_number] #3 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] + CometFilter [sr_item_sk,sr_ticket_number,sr_reason_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] InputAdapter BroadcastExchange #4 - WholeStageCodegen (1) + WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [r_reason_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt index aafe59a77..1ed6d0cb3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt @@ -13,20 +13,20 @@ : : :- * SortMergeJoin LeftAnti (19) : : : :- * Project (13) : : : : +- * SortMergeJoin LeftSemi (12) - : : : : :- * ColumnarToRow (6) - : : : : : +- CometSort (5) - : : : : : +- CometExchange (4) + : : : : :- * Sort (6) + : : : : : +- Exchange (5) + : : : : : +- * ColumnarToRow (4) : : : : : +- CometProject (3) : : : : : +- CometFilter (2) : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : : +- * ColumnarToRow (11) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) + : : : : +- * Sort (11) + : : : : +- Exchange (10) + : : : : +- * ColumnarToRow (9) : : : : +- CometProject (8) : : : : +- CometScan parquet spark_catalog.default.web_sales (7) - : : : +- * ColumnarToRow (18) - : : : +- CometSort (17) - : : : +- CometExchange (16) + : : : +- * Sort (18) + : : : +- Exchange (17) + : : : +- * ColumnarToRow (16) : : : +- CometProject (15) : : : +- CometScan parquet spark_catalog.default.web_returns (14) : : +- BroadcastExchange (24) @@ -61,16 +61,16 @@ Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -(4) CometExchange +(4) ColumnarToRow [codegen id : 1] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(5) CometSort +(5) Exchange Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_order_number#5 ASC NULLS FIRST] +Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(6) ColumnarToRow [codegen id : 1] +(6) Sort [codegen id : 2] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] +Arguments: [ws_order_number#5 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] @@ -82,24 +82,24 @@ ReadSchema: struct Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] Arguments: [ws_warehouse_sk#9, ws_order_number#10], [ws_warehouse_sk#9, ws_order_number#10] -(9) CometExchange +(9) ColumnarToRow [codegen id : 3] Input [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: hashpartitioning(ws_order_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(10) CometSort +(10) Exchange Input [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: [ws_warehouse_sk#9, ws_order_number#10], [ws_order_number#10 ASC NULLS FIRST] +Arguments: hashpartitioning(ws_order_number#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(11) ColumnarToRow [codegen id : 2] +(11) Sort [codegen id : 4] Input [2]: [ws_warehouse_sk#9, ws_order_number#10] +Arguments: [ws_order_number#10 ASC NULLS FIRST], false, 0 -(12) SortMergeJoin [codegen id : 3] +(12) SortMergeJoin [codegen id : 5] Left keys [1]: [ws_order_number#5] Right keys [1]: [ws_order_number#10] Join type: LeftSemi Join condition: NOT (ws_warehouse_sk#4 = ws_warehouse_sk#9) -(13) Project [codegen id : 3] +(13) Project [codegen id : 5] Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] @@ -113,18 +113,18 @@ ReadSchema: struct Input [2]: [wr_order_number#12, wr_returned_date_sk#13] Arguments: [wr_order_number#12], [wr_order_number#12] -(16) CometExchange +(16) ColumnarToRow [codegen id : 6] Input [1]: [wr_order_number#12] -Arguments: hashpartitioning(wr_order_number#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(17) CometSort +(17) Exchange Input [1]: [wr_order_number#12] -Arguments: [wr_order_number#12], [wr_order_number#12 ASC NULLS FIRST] +Arguments: hashpartitioning(wr_order_number#12, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(18) ColumnarToRow [codegen id : 4] +(18) Sort [codegen id : 7] Input [1]: [wr_order_number#12] +Arguments: [wr_order_number#12 ASC NULLS FIRST], false, 0 -(19) SortMergeJoin [codegen id : 8] +(19) SortMergeJoin [codegen id : 11] Left keys [1]: [ws_order_number#5] Right keys [1]: [wr_order_number#12] Join type: LeftAnti @@ -145,20 +145,20 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-01)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(23) ColumnarToRow [codegen id : 5] +(23) ColumnarToRow [codegen id : 8] Input [1]: [d_date_sk#14] (24) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(25) BroadcastHashJoin [codegen id : 8] +(25) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ws_ship_date_sk#1] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(26) Project [codegen id : 8] +(26) Project [codegen id : 11] Output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#14] @@ -177,20 +177,20 @@ Condition : ((isnotnull(ca_state#17) AND (ca_state#17 = IL)) AND isnotnull(ca_ad Input [2]: [ca_address_sk#16, ca_state#17] Arguments: [ca_address_sk#16], [ca_address_sk#16] -(30) ColumnarToRow [codegen id : 6] +(30) ColumnarToRow [codegen id : 9] Input [1]: [ca_address_sk#16] (31) BroadcastExchange Input [1]: [ca_address_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(32) BroadcastHashJoin [codegen id : 8] +(32) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ws_ship_addr_sk#2] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(33) Project [codegen id : 8] +(33) Project [codegen id : 11] Output [4]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#16] @@ -209,38 +209,38 @@ Condition : ((isnotnull(web_company_name#19) AND (web_company_name#19 = pri Input [2]: [web_site_sk#18, web_company_name#19] Arguments: [web_site_sk#18], [web_site_sk#18] -(37) ColumnarToRow [codegen id : 7] +(37) ColumnarToRow [codegen id : 10] Input [1]: [web_site_sk#18] (38) BroadcastExchange Input [1]: [web_site_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(39) BroadcastHashJoin [codegen id : 8] +(39) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ws_web_site_sk#3] Right keys [1]: [web_site_sk#18] Join type: Inner Join condition: None -(40) Project [codegen id : 8] +(40) Project [codegen id : 11] Output [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Input [5]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#18] -(41) HashAggregate [codegen id : 8] +(41) HashAggregate [codegen id : 11] Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Keys [1]: [ws_order_number#5] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#20, sum(UnscaledValue(ws_net_profit#7))#21] Results [3]: [ws_order_number#5, sum#22, sum#23] -(42) HashAggregate [codegen id : 8] +(42) HashAggregate [codegen id : 11] Input [3]: [ws_order_number#5, sum#22, sum#23] Keys [1]: [ws_order_number#5] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#20, sum(UnscaledValue(ws_net_profit#7))#21] Results [3]: [ws_order_number#5, sum#22, sum#23] -(43) HashAggregate [codegen id : 8] +(43) HashAggregate [codegen id : 11] Input [3]: [ws_order_number#5, sum#22, sum#23] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] @@ -251,7 +251,7 @@ Results [3]: [sum#22, sum#23, count#25] Input [3]: [sum#22, sum#23, count#25] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(45) HashAggregate [codegen id : 9] +(45) HashAggregate [codegen id : 12] Input [3]: [sum#22, sum#23, count#25] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt index 369065a66..34ddde768 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (9) +WholeStageCodegen (12) HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] InputAdapter Exchange #1 - WholeStageCodegen (8) + WholeStageCodegen (11) HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] @@ -14,37 +14,43 @@ WholeStageCodegen (9) BroadcastHashJoin [ws_ship_date_sk,d_date_sk] SortMergeJoin [ws_order_number,wr_order_number] InputAdapter - WholeStageCodegen (3) + WholeStageCodegen (5) Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] InputAdapter - WholeStageCodegen (1) - ColumnarToRow + WholeStageCodegen (2) + Sort [ws_order_number] InputAdapter - CometSort [ws_order_number] - CometExchange [ws_order_number] #2 - CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] - CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] + Exchange [ws_order_number] #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] InputAdapter - WholeStageCodegen (2) - ColumnarToRow + WholeStageCodegen (4) + Sort [ws_order_number] InputAdapter - CometSort [ws_order_number] - CometExchange [ws_order_number] #3 - CometProject [ws_warehouse_sk,ws_order_number] - CometScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] + Exchange [ws_order_number] #3 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [ws_warehouse_sk,ws_order_number] + CometScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] InputAdapter - WholeStageCodegen (4) - ColumnarToRow + WholeStageCodegen (7) + Sort [wr_order_number] InputAdapter - CometSort [wr_order_number] - CometExchange [wr_order_number] #4 - CometProject [wr_order_number] - CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] + Exchange [wr_order_number] #4 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometProject [wr_order_number] + CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] InputAdapter BroadcastExchange #5 - WholeStageCodegen (5) + WholeStageCodegen (8) ColumnarToRow InputAdapter CometProject [d_date_sk] @@ -52,7 +58,7 @@ WholeStageCodegen (9) CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 - WholeStageCodegen (6) + WholeStageCodegen (9) ColumnarToRow InputAdapter CometProject [ca_address_sk] @@ -60,7 +66,7 @@ WholeStageCodegen (9) CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 - WholeStageCodegen (7) + WholeStageCodegen (10) ColumnarToRow InputAdapter CometProject [web_site_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt index 375c7bf2f..d8686b2b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt @@ -1,65 +1,62 @@ == Physical Plan == -* HashAggregate (61) -+- Exchange (60) - +- * HashAggregate (59) - +- * HashAggregate (58) - +- * HashAggregate (57) - +- * Project (56) - +- * BroadcastHashJoin Inner BuildRight (55) - :- * Project (49) - : +- * BroadcastHashJoin Inner BuildRight (48) - : :- * Project (42) - : : +- * BroadcastHashJoin Inner BuildRight (41) - : : :- * SortMergeJoin LeftSemi (35) - : : : :- * SortMergeJoin LeftSemi (18) - : : : : :- * ColumnarToRow (6) - : : : : : +- CometSort (5) - : : : : : +- CometExchange (4) +* HashAggregate (58) ++- Exchange (57) + +- * HashAggregate (56) + +- * HashAggregate (55) + +- * HashAggregate (54) + +- * Project (53) + +- * BroadcastHashJoin Inner BuildRight (52) + :- * Project (46) + : +- * BroadcastHashJoin Inner BuildRight (45) + : :- * Project (39) + : : +- * BroadcastHashJoin Inner BuildRight (38) + : : :- * SortMergeJoin LeftSemi (32) + : : : :- * SortMergeJoin LeftSemi (17) + : : : : :- * Sort (6) + : : : : : +- Exchange (5) + : : : : : +- * ColumnarToRow (4) : : : : : +- CometProject (3) : : : : : +- CometFilter (2) : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : : +- * Project (17) - : : : : +- * SortMergeJoin Inner (16) - : : : : :- * ColumnarToRow (12) - : : : : : +- CometSort (11) - : : : : : +- CometExchange (10) + : : : : +- * Project (16) + : : : : +- * SortMergeJoin Inner (15) + : : : : :- * Sort (12) + : : : : : +- Exchange (11) + : : : : : +- * ColumnarToRow (10) : : : : : +- CometProject (9) : : : : : +- CometFilter (8) : : : : : +- CometScan parquet spark_catalog.default.web_sales (7) - : : : : +- * ColumnarToRow (15) - : : : : +- CometSort (14) - : : : : +- ReusedExchange (13) - : : : +- * Project (34) - : : : +- * SortMergeJoin Inner (33) - : : : :- * ColumnarToRow (24) - : : : : +- CometSort (23) - : : : : +- CometExchange (22) - : : : : +- CometProject (21) - : : : : +- CometFilter (20) - : : : : +- CometScan parquet spark_catalog.default.web_returns (19) - : : : +- * Project (32) - : : : +- * SortMergeJoin Inner (31) - : : : :- * ColumnarToRow (27) - : : : : +- CometSort (26) - : : : : +- ReusedExchange (25) - : : : +- * ColumnarToRow (30) - : : : +- CometSort (29) - : : : +- ReusedExchange (28) - : : +- BroadcastExchange (40) - : : +- * ColumnarToRow (39) - : : +- CometProject (38) - : : +- CometFilter (37) - : : +- CometScan parquet spark_catalog.default.date_dim (36) - : +- BroadcastExchange (47) - : +- * ColumnarToRow (46) - : +- CometProject (45) - : +- CometFilter (44) - : +- CometScan parquet spark_catalog.default.customer_address (43) - +- BroadcastExchange (54) - +- * ColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan parquet spark_catalog.default.web_site (50) + : : : : +- * Sort (14) + : : : : +- ReusedExchange (13) + : : : +- * Project (31) + : : : +- * SortMergeJoin Inner (30) + : : : :- * Sort (23) + : : : : +- Exchange (22) + : : : : +- * ColumnarToRow (21) + : : : : +- CometProject (20) + : : : : +- CometFilter (19) + : : : : +- CometScan parquet spark_catalog.default.web_returns (18) + : : : +- * Project (29) + : : : +- * SortMergeJoin Inner (28) + : : : :- * Sort (25) + : : : : +- ReusedExchange (24) + : : : +- * Sort (27) + : : : +- ReusedExchange (26) + : : +- BroadcastExchange (37) + : : +- * ColumnarToRow (36) + : : +- CometProject (35) + : : +- CometFilter (34) + : : +- CometScan parquet spark_catalog.default.date_dim (33) + : +- BroadcastExchange (44) + : +- * ColumnarToRow (43) + : +- CometProject (42) + : +- CometFilter (41) + : +- CometScan parquet spark_catalog.default.customer_address (40) + +- BroadcastExchange (51) + +- * ColumnarToRow (50) + +- CometProject (49) + +- CometFilter (48) + +- CometScan parquet spark_catalog.default.web_site (47) (unknown) Scan parquet spark_catalog.default.web_sales @@ -77,16 +74,16 @@ Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -(4) CometExchange +(4) ColumnarToRow [codegen id : 1] Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(5) CometSort +(5) Exchange Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_order_number#4 ASC NULLS FIRST] +Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(6) ColumnarToRow [codegen id : 1] +(6) Sort [codegen id : 2] Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] +Arguments: [ws_order_number#4 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] @@ -103,38 +100,35 @@ Condition : (isnotnull(ws_order_number#9) AND isnotnull(ws_warehouse_sk#8)) Input [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_warehouse_sk#8, ws_order_number#9] -(10) CometExchange +(10) ColumnarToRow [codegen id : 3] Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: hashpartitioning(ws_order_number#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(11) CometSort +(11) Exchange Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_order_number#9 ASC NULLS FIRST] +Arguments: hashpartitioning(ws_order_number#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(12) ColumnarToRow [codegen id : 2] +(12) Sort [codegen id : 4] Input [2]: [ws_warehouse_sk#8, ws_order_number#9] +Arguments: [ws_order_number#9 ASC NULLS FIRST], false, 0 -(13) ReusedExchange [Reuses operator id: 10] +(13) ReusedExchange [Reuses operator id: 11] Output [2]: [ws_warehouse_sk#11, ws_order_number#12] -(14) CometSort +(14) Sort [codegen id : 6] Input [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_warehouse_sk#11, ws_order_number#12], [ws_order_number#12 ASC NULLS FIRST] +Arguments: [ws_order_number#12 ASC NULLS FIRST], false, 0 -(15) ColumnarToRow [codegen id : 3] -Input [2]: [ws_warehouse_sk#11, ws_order_number#12] - -(16) SortMergeJoin [codegen id : 4] +(15) SortMergeJoin [codegen id : 7] Left keys [1]: [ws_order_number#9] Right keys [1]: [ws_order_number#12] Join type: Inner Join condition: NOT (ws_warehouse_sk#8 = ws_warehouse_sk#11) -(17) Project [codegen id : 4] +(16) Project [codegen id : 7] Output [1]: [ws_order_number#9] Input [4]: [ws_warehouse_sk#8, ws_order_number#9, ws_warehouse_sk#11, ws_order_number#12] -(18) SortMergeJoin [codegen id : 5] +(17) SortMergeJoin [codegen id : 8] Left keys [1]: [ws_order_number#4] Right keys [1]: [ws_order_number#9] Join type: LeftSemi @@ -147,66 +141,60 @@ Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number)] ReadSchema: struct -(20) CometFilter +(19) CometFilter Input [2]: [wr_order_number#13, wr_returned_date_sk#14] Condition : isnotnull(wr_order_number#13) -(21) CometProject +(20) CometProject Input [2]: [wr_order_number#13, wr_returned_date_sk#14] Arguments: [wr_order_number#13], [wr_order_number#13] -(22) CometExchange +(21) ColumnarToRow [codegen id : 9] Input [1]: [wr_order_number#13] -Arguments: hashpartitioning(wr_order_number#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(23) CometSort +(22) Exchange Input [1]: [wr_order_number#13] -Arguments: [wr_order_number#13], [wr_order_number#13 ASC NULLS FIRST] +Arguments: hashpartitioning(wr_order_number#13, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(24) ColumnarToRow [codegen id : 6] +(23) Sort [codegen id : 10] Input [1]: [wr_order_number#13] +Arguments: [wr_order_number#13 ASC NULLS FIRST], false, 0 -(25) ReusedExchange [Reuses operator id: 10] +(24) ReusedExchange [Reuses operator id: 11] Output [2]: [ws_warehouse_sk#8, ws_order_number#9] -(26) CometSort -Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_order_number#9 ASC NULLS FIRST] - -(27) ColumnarToRow [codegen id : 7] +(25) Sort [codegen id : 12] Input [2]: [ws_warehouse_sk#8, ws_order_number#9] +Arguments: [ws_order_number#9 ASC NULLS FIRST], false, 0 -(28) ReusedExchange [Reuses operator id: 10] +(26) ReusedExchange [Reuses operator id: 11] Output [2]: [ws_warehouse_sk#11, ws_order_number#12] -(29) CometSort -Input [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_warehouse_sk#11, ws_order_number#12], [ws_order_number#12 ASC NULLS FIRST] - -(30) ColumnarToRow [codegen id : 8] +(27) Sort [codegen id : 14] Input [2]: [ws_warehouse_sk#11, ws_order_number#12] +Arguments: [ws_order_number#12 ASC NULLS FIRST], false, 0 -(31) SortMergeJoin [codegen id : 9] +(28) SortMergeJoin [codegen id : 15] Left keys [1]: [ws_order_number#9] Right keys [1]: [ws_order_number#12] Join type: Inner Join condition: NOT (ws_warehouse_sk#8 = ws_warehouse_sk#11) -(32) Project [codegen id : 9] +(29) Project [codegen id : 15] Output [1]: [ws_order_number#9] Input [4]: [ws_warehouse_sk#8, ws_order_number#9, ws_warehouse_sk#11, ws_order_number#12] -(33) SortMergeJoin [codegen id : 10] +(30) SortMergeJoin [codegen id : 16] Left keys [1]: [wr_order_number#13] Right keys [1]: [ws_order_number#9] Join type: Inner Join condition: None -(34) Project [codegen id : 10] +(31) Project [codegen id : 16] Output [1]: [wr_order_number#13] Input [2]: [wr_order_number#13, ws_order_number#9] -(35) SortMergeJoin [codegen id : 14] +(32) SortMergeJoin [codegen id : 20] Left keys [1]: [ws_order_number#4] Right keys [1]: [wr_order_number#13] Join type: LeftSemi @@ -219,28 +207,28 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] ReadSchema: struct -(37) CometFilter +(34) CometFilter Input [2]: [d_date_sk#15, d_date#16] Condition : (((isnotnull(d_date#16) AND (d_date#16 >= 1999-02-01)) AND (d_date#16 <= 1999-04-02)) AND isnotnull(d_date_sk#15)) -(38) CometProject +(35) CometProject Input [2]: [d_date_sk#15, d_date#16] Arguments: [d_date_sk#15], [d_date_sk#15] -(39) ColumnarToRow [codegen id : 11] +(36) ColumnarToRow [codegen id : 17] Input [1]: [d_date_sk#15] -(40) BroadcastExchange +(37) BroadcastExchange Input [1]: [d_date_sk#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(41) BroadcastHashJoin [codegen id : 14] +(38) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ws_ship_date_sk#1] Right keys [1]: [d_date_sk#15] Join type: Inner Join condition: None -(42) Project [codegen id : 14] +(39) Project [codegen id : 20] Output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, d_date_sk#15] @@ -251,28 +239,28 @@ Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,IL), IsNotNull(ca_address_sk)] ReadSchema: struct -(44) CometFilter +(41) CometFilter Input [2]: [ca_address_sk#17, ca_state#18] Condition : ((isnotnull(ca_state#18) AND (ca_state#18 = IL)) AND isnotnull(ca_address_sk#17)) -(45) CometProject +(42) CometProject Input [2]: [ca_address_sk#17, ca_state#18] Arguments: [ca_address_sk#17], [ca_address_sk#17] -(46) ColumnarToRow [codegen id : 12] +(43) ColumnarToRow [codegen id : 18] Input [1]: [ca_address_sk#17] -(47) BroadcastExchange +(44) BroadcastExchange Input [1]: [ca_address_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(48) BroadcastHashJoin [codegen id : 14] +(45) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ws_ship_addr_sk#2] Right keys [1]: [ca_address_sk#17] Join type: Inner Join condition: None -(49) Project [codegen id : 14] +(46) Project [codegen id : 20] Output [4]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ca_address_sk#17] @@ -283,57 +271,57 @@ Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] ReadSchema: struct -(51) CometFilter +(48) CometFilter Input [2]: [web_site_sk#19, web_company_name#20] Condition : ((isnotnull(web_company_name#20) AND (web_company_name#20 = pri )) AND isnotnull(web_site_sk#19)) -(52) CometProject +(49) CometProject Input [2]: [web_site_sk#19, web_company_name#20] Arguments: [web_site_sk#19], [web_site_sk#19] -(53) ColumnarToRow [codegen id : 13] +(50) ColumnarToRow [codegen id : 19] Input [1]: [web_site_sk#19] -(54) BroadcastExchange +(51) BroadcastExchange Input [1]: [web_site_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(55) BroadcastHashJoin [codegen id : 14] +(52) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ws_web_site_sk#3] Right keys [1]: [web_site_sk#19] Join type: Inner Join condition: None -(56) Project [codegen id : 14] +(53) Project [codegen id : 20] Output [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Input [5]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, web_site_sk#19] -(57) HashAggregate [codegen id : 14] +(54) HashAggregate [codegen id : 20] Input [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Keys [1]: [ws_order_number#4] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#5)), partial_sum(UnscaledValue(ws_net_profit#6))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#21, sum(UnscaledValue(ws_net_profit#6))#22] Results [3]: [ws_order_number#4, sum#23, sum#24] -(58) HashAggregate [codegen id : 14] +(55) HashAggregate [codegen id : 20] Input [3]: [ws_order_number#4, sum#23, sum#24] Keys [1]: [ws_order_number#4] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6))] Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#21, sum(UnscaledValue(ws_net_profit#6))#22] Results [3]: [ws_order_number#4, sum#23, sum#24] -(59) HashAggregate [codegen id : 14] +(56) HashAggregate [codegen id : 20] Input [3]: [ws_order_number#4, sum#23, sum#24] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6)), partial_count(distinct ws_order_number#4)] Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#21, sum(UnscaledValue(ws_net_profit#6))#22, count(ws_order_number#4)#25] Results [3]: [sum#23, sum#24, count#26] -(60) Exchange +(57) Exchange Input [3]: [sum#23, sum#24, count#26] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(61) HashAggregate [codegen id : 15] +(58) HashAggregate [codegen id : 21] Input [3]: [sum#23, sum#24, count#26] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#5)), sum(UnscaledValue(ws_net_profit#6)), count(distinct ws_order_number#4)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt index 2ad651cb6..5b699890c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (15) +WholeStageCodegen (21) HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] InputAdapter Exchange #1 - WholeStageCodegen (14) + WholeStageCodegen (20) HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] @@ -14,68 +14,71 @@ WholeStageCodegen (15) BroadcastHashJoin [ws_ship_date_sk,d_date_sk] SortMergeJoin [ws_order_number,wr_order_number] InputAdapter - WholeStageCodegen (5) + WholeStageCodegen (8) SortMergeJoin [ws_order_number,ws_order_number] InputAdapter - WholeStageCodegen (1) - ColumnarToRow + WholeStageCodegen (2) + Sort [ws_order_number] InputAdapter - CometSort [ws_order_number] - CometExchange [ws_order_number] #2 - CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] - CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] + Exchange [ws_order_number] #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] InputAdapter - WholeStageCodegen (4) + WholeStageCodegen (7) Project [ws_order_number] SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] InputAdapter - WholeStageCodegen (2) - ColumnarToRow + WholeStageCodegen (4) + Sort [ws_order_number] InputAdapter - CometSort [ws_order_number] - CometExchange [ws_order_number] #3 - CometProject [ws_warehouse_sk,ws_order_number] - CometFilter [ws_order_number,ws_warehouse_sk] - CometScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] + Exchange [ws_order_number] #3 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [ws_warehouse_sk,ws_order_number] + CometFilter [ws_order_number,ws_warehouse_sk] + CometScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] InputAdapter - WholeStageCodegen (3) - ColumnarToRow + WholeStageCodegen (6) + Sort [ws_order_number] InputAdapter - CometSort [ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 InputAdapter - WholeStageCodegen (10) + WholeStageCodegen (16) Project [wr_order_number] SortMergeJoin [wr_order_number,ws_order_number] InputAdapter - WholeStageCodegen (6) - ColumnarToRow + WholeStageCodegen (10) + Sort [wr_order_number] InputAdapter - CometSort [wr_order_number] - CometExchange [wr_order_number] #4 - CometProject [wr_order_number] - CometFilter [wr_order_number] - CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] + Exchange [wr_order_number] #4 + WholeStageCodegen (9) + ColumnarToRow + InputAdapter + CometProject [wr_order_number] + CometFilter [wr_order_number] + CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] InputAdapter - WholeStageCodegen (9) + WholeStageCodegen (15) Project [ws_order_number] SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] InputAdapter - WholeStageCodegen (7) - ColumnarToRow + WholeStageCodegen (12) + Sort [ws_order_number] InputAdapter - CometSort [ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 InputAdapter - WholeStageCodegen (8) - ColumnarToRow + WholeStageCodegen (14) + Sort [ws_order_number] InputAdapter - CometSort [ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 InputAdapter BroadcastExchange #5 - WholeStageCodegen (11) + WholeStageCodegen (17) ColumnarToRow InputAdapter CometProject [d_date_sk] @@ -83,7 +86,7 @@ WholeStageCodegen (15) CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 - WholeStageCodegen (12) + WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [ca_address_sk] @@ -91,7 +94,7 @@ WholeStageCodegen (15) CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 - WholeStageCodegen (13) + WholeStageCodegen (19) ColumnarToRow InputAdapter CometProject [web_site_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt index 9bbf5e984..bc8b8657e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt @@ -1,51 +1,52 @@ == Physical Plan == -* Sort (47) -+- Exchange (46) - +- * Filter (45) - +- * HashAggregate (44) - +- Exchange (43) - +- * HashAggregate (42) - +- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (20) - : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : :- * ColumnarToRow (13) - : : : : +- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometScan parquet spark_catalog.default.store_returns (6) - : : : +- BroadcastExchange (18) - : : : +- * ColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometFilter (15) - : : : +- CometScan parquet spark_catalog.default.store (14) - : : +- BroadcastExchange (24) - : : +- * ColumnarToRow (23) - : : +- CometFilter (22) - : : +- CometScan parquet spark_catalog.default.item (21) - : +- BroadcastExchange (30) - : +- * ColumnarToRow (29) - : +- CometFilter (28) - : +- CometScan parquet spark_catalog.default.customer (27) - +- BroadcastExchange (36) - +- * ColumnarToRow (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.customer_address (33) +* Sort (48) ++- Exchange (47) + +- * Filter (46) + +- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * Project (21) + : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : :- * Project (14) + : : : : +- * SortMergeJoin Inner (13) + : : : : :- * Sort (6) + : : : : : +- Exchange (5) + : : : : : +- * ColumnarToRow (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- * Sort (12) + : : : : +- Exchange (11) + : : : : +- * ColumnarToRow (10) + : : : : +- CometProject (9) + : : : : +- CometFilter (8) + : : : : +- CometScan parquet spark_catalog.default.store_returns (7) + : : : +- BroadcastExchange (19) + : : : +- * ColumnarToRow (18) + : : : +- CometProject (17) + : : : +- CometFilter (16) + : : : +- CometScan parquet spark_catalog.default.store (15) + : : +- BroadcastExchange (25) + : : +- * ColumnarToRow (24) + : : +- CometFilter (23) + : : +- CometScan parquet spark_catalog.default.item (22) + : +- BroadcastExchange (31) + : +- * ColumnarToRow (30) + : +- CometFilter (29) + : +- CometScan parquet spark_catalog.default.customer (28) + +- BroadcastExchange (37) + +- * ColumnarToRow (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.customer_address (34) (unknown) Scan parquet spark_catalog.default.store_sales @@ -63,13 +64,16 @@ Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND is Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -(4) CometExchange +(4) ColumnarToRow [codegen id : 1] Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(5) CometSort +(5) Exchange Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] +Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(6) Sort [codegen id : 2] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Arguments: [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.store_returns Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] @@ -78,33 +82,34 @@ Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(7) CometFilter +(8) CometFilter Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) -(8) CometProject +(9) CometProject Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] -(9) CometExchange +(10) ColumnarToRow [codegen id : 3] Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(10) CometSort +(11) Exchange Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] +Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(11) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner +(12) Sort [codegen id : 4] +Input [2]: [sr_item_sk#7, sr_ticket_number#8] +Arguments: [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST], false, 0 -(12) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +(13) SortMergeJoin [codegen id : 9] +Left keys [2]: [ss_ticket_number#4, ss_item_sk#1] +Right keys [2]: [sr_ticket_number#8, sr_item_sk#7] +Join type: Inner +Join condition: None -(13) ColumnarToRow [codegen id : 5] -Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +(14) Project [codegen id : 9] +Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] (unknown) Scan parquet spark_catalog.default.store Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] @@ -113,28 +118,28 @@ Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(15) CometFilter +(16) CometFilter Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(s_zip#14)) -(16) CometProject +(17) CometProject Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Arguments: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14], [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(17) ColumnarToRow [codegen id : 1] +(18) ColumnarToRow [codegen id : 5] Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(18) BroadcastExchange +(19) BroadcastExchange Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(19) BroadcastHashJoin [codegen id : 5] +(20) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#10] Join type: Inner Join condition: None -(20) Project [codegen id : 5] +(21) Project [codegen id : 9] Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] @@ -145,24 +150,24 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale ), IsNotNull(i_item_sk)] ReadSchema: struct -(22) CometFilter +(23) CometFilter Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Condition : ((isnotnull(i_color#18) AND (i_color#18 = pale )) AND isnotnull(i_item_sk#15)) -(23) ColumnarToRow [codegen id : 2] +(24) ColumnarToRow [codegen id : 6] Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] -(24) BroadcastExchange +(25) BroadcastExchange Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(25) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#15] Join type: Inner Join condition: None -(26) Project [codegen id : 5] +(27) Project [codegen id : 9] Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] @@ -173,24 +178,24 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] ReadSchema: struct -(28) CometFilter +(29) CometFilter Input [5]: [c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] Condition : ((isnotnull(c_customer_sk#21) AND isnotnull(c_current_addr_sk#22)) AND isnotnull(c_birth_country#25)) -(29) ColumnarToRow [codegen id : 3] +(30) ColumnarToRow [codegen id : 7] Input [5]: [c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] -(30) BroadcastExchange +(31) BroadcastExchange Input [5]: [c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(31) BroadcastHashJoin [codegen id : 5] +(32) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_customer_sk#2] Right keys [1]: [c_customer_sk#21] Join type: Inner Join condition: None -(32) Project [codegen id : 5] +(33) Project [codegen id : 9] Output [13]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] @@ -201,78 +206,78 @@ Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct -(34) CometFilter +(35) CometFilter Input [4]: [ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] Condition : ((isnotnull(ca_address_sk#26) AND isnotnull(ca_country#29)) AND isnotnull(ca_zip#28)) -(35) ColumnarToRow [codegen id : 4] +(36) ColumnarToRow [codegen id : 8] Input [4]: [ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] -(36) BroadcastExchange +(37) BroadcastExchange Input [4]: [ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] Arguments: HashedRelationBroadcastMode(List(input[0, int, false], upper(input[3, string, false]), input[2, string, false]),false), [plan_id=6] -(37) BroadcastHashJoin [codegen id : 5] +(38) BroadcastHashJoin [codegen id : 9] Left keys [3]: [c_current_addr_sk#22, c_birth_country#25, s_zip#14] Right keys [3]: [ca_address_sk#26, upper(ca_country#29), ca_zip#28] Join type: Inner Join condition: None -(38) Project [codegen id : 5] +(39) Project [codegen id : 9] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, ca_state#27] Input [17]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25, ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] -(39) HashAggregate [codegen id : 5] +(40) HashAggregate [codegen id : 9] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, ca_state#27] Keys [10]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum#30] Results [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#31] -(40) Exchange +(41) Exchange Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#31] Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(41) HashAggregate [codegen id : 6] +(42) HashAggregate [codegen id : 10] Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#31] Keys [10]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#32] Results [4]: [c_last_name#24, c_first_name#23, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#32,17,2) AS netpaid#33] -(42) HashAggregate [codegen id : 6] +(43) HashAggregate [codegen id : 10] Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, netpaid#33] Keys [3]: [c_last_name#24, c_first_name#23, s_store_name#11] Functions [1]: [partial_sum(netpaid#33)] Aggregate Attributes [2]: [sum#34, isEmpty#35] Results [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#36, isEmpty#37] -(43) Exchange +(44) Exchange Input [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#36, isEmpty#37] Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(44) HashAggregate [codegen id : 7] +(45) HashAggregate [codegen id : 11] Input [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#36, isEmpty#37] Keys [3]: [c_last_name#24, c_first_name#23, s_store_name#11] Functions [1]: [sum(netpaid#33)] Aggregate Attributes [1]: [sum(netpaid#33)#38] Results [4]: [c_last_name#24, c_first_name#23, s_store_name#11, sum(netpaid#33)#38 AS paid#39] -(45) Filter [codegen id : 7] +(46) Filter [codegen id : 11] Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] Condition : (isnotnull(paid#39) AND (cast(paid#39 as decimal(33,8)) > cast(Subquery scalar-subquery#40, [id=#41] as decimal(33,8)))) -(46) Exchange +(47) Exchange Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] Arguments: rangepartitioning(c_last_name#24 ASC NULLS FIRST, c_first_name#23 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(47) Sort [codegen id : 8] +(48) Sort [codegen id : 12] Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] Arguments: [c_last_name#24 ASC NULLS FIRST, c_first_name#23 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST], true, 0 ===== Subqueries ===== -Subquery:1 Hosting operator id = 45 Hosting Expression = Subquery scalar-subquery#40, [id=#41] +Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#40, [id=#41] * HashAggregate (75) +- Exchange (74) +- * HashAggregate (73) @@ -287,13 +292,12 @@ Subquery:1 Hosting operator id = 45 Hosting Expression = Subquery scalar-subquer : : +- * BroadcastHashJoin Inner BuildRight (62) : : :- * Project (57) : : : +- * BroadcastHashJoin Inner BuildRight (56) - : : : :- * ColumnarToRow (54) - : : : : +- CometProject (53) - : : : : +- CometSortMergeJoin (52) - : : : : :- CometSort (49) - : : : : : +- ReusedExchange (48) - : : : : +- CometSort (51) - : : : : +- ReusedExchange (50) + : : : :- * Project (54) + : : : : +- * SortMergeJoin Inner (53) + : : : : :- * Sort (50) + : : : : : +- ReusedExchange (49) + : : : : +- * Sort (52) + : : : : +- ReusedExchange (51) : : : +- ReusedExchange (55) : : +- BroadcastExchange (61) : : +- * ColumnarToRow (60) @@ -303,42 +307,40 @@ Subquery:1 Hosting operator id = 45 Hosting Expression = Subquery scalar-subquer +- ReusedExchange (67) -(48) ReusedExchange [Reuses operator id: 4] +(49) ReusedExchange [Reuses operator id: 5] Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -(49) CometSort +(50) Sort [codegen id : 2] Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] +Arguments: [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST], false, 0 -(50) ReusedExchange [Reuses operator id: 9] +(51) ReusedExchange [Reuses operator id: 11] Output [2]: [sr_item_sk#7, sr_ticket_number#8] -(51) CometSort +(52) Sort [codegen id : 4] Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] +Arguments: [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST], false, 0 -(52) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner +(53) SortMergeJoin [codegen id : 9] +Left keys [2]: [ss_ticket_number#4, ss_item_sk#1] +Right keys [2]: [sr_ticket_number#8, sr_item_sk#7] +Join type: Inner +Join condition: None -(53) CometProject +(54) Project [codegen id : 9] +Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] - -(54) ColumnarToRow [codegen id : 5] -Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -(55) ReusedExchange [Reuses operator id: 18] +(55) ReusedExchange [Reuses operator id: 19] Output [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(56) BroadcastHashJoin [codegen id : 5] +(56) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#10] Join type: Inner Join condition: None -(57) Project [codegen id : 5] +(57) Project [codegen id : 9] Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] @@ -353,50 +355,50 @@ ReadSchema: struct Input [2]: [ca_address_sk#1, ca_state#2] Condition : isnotnull(ca_address_sk#1) -(3) ColumnarToRow [codegen id : 6] +(3) ColumnarToRow [codegen id : 7] Input [2]: [ca_address_sk#1, ca_state#2] (unknown) Scan parquet spark_catalog.default.customer @@ -72,13 +72,13 @@ Input [2]: [c_customer_sk#3, c_current_addr_sk#4] Input [2]: [c_customer_sk#3, c_current_addr_sk#4] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 6] +(8) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ca_address_sk#1] Right keys [1]: [c_current_addr_sk#4] Join type: Inner Join condition: None -(9) Project [codegen id : 6] +(9) Project [codegen id : 7] Output [2]: [ca_state#2, c_customer_sk#3] Input [4]: [ca_address_sk#1, ca_state#2, c_customer_sk#3, c_current_addr_sk#4] @@ -101,26 +101,26 @@ Input [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] Input [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=2] -(14) BroadcastHashJoin [codegen id : 6] +(14) BroadcastHashJoin [codegen id : 7] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#6] Join type: Inner Join condition: None -(15) Project [codegen id : 6] +(15) Project [codegen id : 7] Output [3]: [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7] Input [5]: [ca_state#2, c_customer_sk#3, ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] (16) ReusedExchange [Reuses operator id: 44] Output [1]: [d_date_sk#9] -(17) BroadcastHashJoin [codegen id : 6] +(17) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#9] Join type: Inner Join condition: None -(18) Project [codegen id : 6] +(18) Project [codegen id : 7] Output [2]: [ca_state#2, ss_item_sk#5] Input [4]: [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7, d_date_sk#9] @@ -135,7 +135,7 @@ ReadSchema: struct Input [3]: [i_item_sk#10, i_current_price#11, i_category#12] Condition : ((isnotnull(i_current_price#11) AND isnotnull(i_category#12)) AND isnotnull(i_item_sk#10)) -(21) ColumnarToRow [codegen id : 5] +(21) ColumnarToRow [codegen id : 6] Input [3]: [i_item_sk#10, i_current_price#11, i_category#12] (unknown) Scan parquet spark_catalog.default.item @@ -154,75 +154,77 @@ Input [2]: [i_current_price#13, i_category#14] Keys [1]: [i_category#14] Functions [1]: [partial_avg(UnscaledValue(i_current_price#13))] -(25) CometExchange +(25) ColumnarToRow [codegen id : 4] Input [3]: [i_category#14, sum#15, count#16] -Arguments: hashpartitioning(i_category#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(26) CometHashAggregate +(26) Exchange +Input [3]: [i_category#14, sum#15, count#16] +Arguments: hashpartitioning(i_category#14, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(27) HashAggregate [codegen id : 5] Input [3]: [i_category#14, sum#15, count#16] Keys [1]: [i_category#14] Functions [1]: [avg(UnscaledValue(i_current_price#13))] +Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#13))#17] +Results [2]: [cast((avg(UnscaledValue(i_current_price#13))#17 / 100.0) as decimal(11,6)) AS avg(i_current_price)#18, i_category#14] -(27) CometFilter -Input [2]: [avg(i_current_price)#17, i_category#14] -Condition : isnotnull(avg(i_current_price)#17) - -(28) ColumnarToRow [codegen id : 4] -Input [2]: [avg(i_current_price)#17, i_category#14] +(28) Filter [codegen id : 5] +Input [2]: [avg(i_current_price)#18, i_category#14] +Condition : isnotnull(avg(i_current_price)#18) (29) BroadcastExchange -Input [2]: [avg(i_current_price)#17, i_category#14] +Input [2]: [avg(i_current_price)#18, i_category#14] Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=4] -(30) BroadcastHashJoin [codegen id : 5] +(30) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_category#12] Right keys [1]: [i_category#14] Join type: Inner -Join condition: (cast(i_current_price#11 as decimal(14,7)) > (1.2 * avg(i_current_price)#17)) +Join condition: (cast(i_current_price#11 as decimal(14,7)) > (1.2 * avg(i_current_price)#18)) -(31) Project [codegen id : 5] +(31) Project [codegen id : 6] Output [1]: [i_item_sk#10] -Input [5]: [i_item_sk#10, i_current_price#11, i_category#12, avg(i_current_price)#17, i_category#14] +Input [5]: [i_item_sk#10, i_current_price#11, i_category#12, avg(i_current_price)#18, i_category#14] (32) BroadcastExchange Input [1]: [i_item_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(33) BroadcastHashJoin [codegen id : 6] +(33) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ss_item_sk#5] Right keys [1]: [i_item_sk#10] Join type: Inner Join condition: None -(34) Project [codegen id : 6] +(34) Project [codegen id : 7] Output [1]: [ca_state#2] Input [3]: [ca_state#2, ss_item_sk#5, i_item_sk#10] -(35) HashAggregate [codegen id : 6] +(35) HashAggregate [codegen id : 7] Input [1]: [ca_state#2] Keys [1]: [ca_state#2] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#18] -Results [2]: [ca_state#2, count#19] +Aggregate Attributes [1]: [count#19] +Results [2]: [ca_state#2, count#20] (36) Exchange -Input [2]: [ca_state#2, count#19] +Input [2]: [ca_state#2, count#20] Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(37) HashAggregate [codegen id : 7] -Input [2]: [ca_state#2, count#19] +(37) HashAggregate [codegen id : 8] +Input [2]: [ca_state#2, count#20] Keys [1]: [ca_state#2] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#20] -Results [3]: [ca_state#2 AS state#21, count(1)#20 AS cnt#22, ca_state#2] +Aggregate Attributes [1]: [count(1)#21] +Results [3]: [ca_state#2 AS state#22, count(1)#21 AS cnt#23, ca_state#2] -(38) Filter [codegen id : 7] -Input [3]: [state#21, cnt#22, ca_state#2] -Condition : (cnt#22 >= 10) +(38) Filter [codegen id : 8] +Input [3]: [state#22, cnt#23, ca_state#2] +Condition : (cnt#23 >= 10) (39) TakeOrderedAndProject -Input [3]: [state#21, cnt#22, ca_state#2] -Arguments: 100, [cnt#22 ASC NULLS FIRST, ca_state#2 ASC NULLS FIRST], [state#21, cnt#22] +Input [3]: [state#22, cnt#23, ca_state#2] +Arguments: 100, [cnt#23 ASC NULLS FIRST, ca_state#2 ASC NULLS FIRST], [state#22, cnt#23] ===== Subqueries ===== @@ -235,18 +237,18 @@ BroadcastExchange (44) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#9, d_month_seq#23] +Output [2]: [d_date_sk#9, d_month_seq#24] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct (41) CometFilter -Input [2]: [d_date_sk#9, d_month_seq#23] -Condition : ((isnotnull(d_month_seq#23) AND (d_month_seq#23 = Subquery scalar-subquery#24, [id=#25])) AND isnotnull(d_date_sk#9)) +Input [2]: [d_date_sk#9, d_month_seq#24] +Condition : ((isnotnull(d_month_seq#24) AND (d_month_seq#24 = Subquery scalar-subquery#25, [id=#26])) AND isnotnull(d_date_sk#9)) (42) CometProject -Input [2]: [d_date_sk#9, d_month_seq#23] +Input [2]: [d_date_sk#9, d_month_seq#24] Arguments: [d_date_sk#9], [d_date_sk#9] (43) ColumnarToRow [codegen id : 1] @@ -256,10 +258,10 @@ Input [1]: [d_date_sk#9] Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 41 Hosting Expression = Subquery scalar-subquery#24, [id=#25] -* ColumnarToRow (51) -+- CometHashAggregate (50) - +- CometExchange (49) +Subquery:2 Hosting operator id = 41 Hosting Expression = Subquery scalar-subquery#25, [id=#26] +* HashAggregate (51) ++- Exchange (50) + +- * ColumnarToRow (49) +- CometHashAggregate (48) +- CometProject (47) +- CometFilter (46) @@ -267,35 +269,37 @@ Subquery:2 Hosting operator id = 41 Hosting Expression = Subquery scalar-subquer (unknown) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#26, d_year#27, d_moy#28] +Output [3]: [d_month_seq#27, d_year#28, 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,1)] ReadSchema: struct (46) CometFilter -Input [3]: [d_month_seq#26, d_year#27, d_moy#28] -Condition : (((isnotnull(d_year#27) AND isnotnull(d_moy#28)) AND (d_year#27 = 2000)) AND (d_moy#28 = 1)) +Input [3]: [d_month_seq#27, d_year#28, d_moy#29] +Condition : (((isnotnull(d_year#28) AND isnotnull(d_moy#29)) AND (d_year#28 = 2000)) AND (d_moy#29 = 1)) (47) CometProject -Input [3]: [d_month_seq#26, d_year#27, d_moy#28] -Arguments: [d_month_seq#26], [d_month_seq#26] +Input [3]: [d_month_seq#27, d_year#28, d_moy#29] +Arguments: [d_month_seq#27], [d_month_seq#27] (48) CometHashAggregate -Input [1]: [d_month_seq#26] -Keys [1]: [d_month_seq#26] +Input [1]: [d_month_seq#27] +Keys [1]: [d_month_seq#27] Functions: [] -(49) CometExchange -Input [1]: [d_month_seq#26] -Arguments: hashpartitioning(d_month_seq#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +(49) ColumnarToRow [codegen id : 1] +Input [1]: [d_month_seq#27] -(50) CometHashAggregate -Input [1]: [d_month_seq#26] -Keys [1]: [d_month_seq#26] -Functions: [] +(50) Exchange +Input [1]: [d_month_seq#27] +Arguments: hashpartitioning(d_month_seq#27, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(51) ColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#26] +(51) HashAggregate [codegen id : 2] +Input [1]: [d_month_seq#27] +Keys [1]: [d_month_seq#27] +Functions: [] +Aggregate Attributes: [] +Results [1]: [d_month_seq#27] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt index d2126126d..57ef6ebd5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [cnt,ca_state,state] - WholeStageCodegen (7) + WholeStageCodegen (8) Filter [cnt] HashAggregate [ca_state,count] [count(1),state,cnt,count] InputAdapter Exchange [ca_state] #1 - WholeStageCodegen (6) + WholeStageCodegen (7) HashAggregate [ca_state] [count,count] Project [ca_state] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -40,21 +40,23 @@ TakeOrderedAndProject [cnt,ca_state,state] CometProject [d_date_sk] CometFilter [d_month_seq,d_date_sk] Subquery #2 - WholeStageCodegen (1) - ColumnarToRow + WholeStageCodegen (2) + HashAggregate [d_month_seq] InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + Exchange [d_month_seq] #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #6 - WholeStageCodegen (5) + WholeStageCodegen (6) Project [i_item_sk] BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] ColumnarToRow @@ -63,12 +65,14 @@ TakeOrderedAndProject [cnt,ca_state,state] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] InputAdapter BroadcastExchange #7 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [avg(i_current_price)] - CometHashAggregate [i_category,sum,count] - CometExchange [i_category] #8 - CometHashAggregate [i_category,i_current_price] - CometFilter [i_category] - CometScan parquet spark_catalog.default.item [i_current_price,i_category] + WholeStageCodegen (5) + Filter [avg(i_current_price)] + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count] + InputAdapter + Exchange [i_category] #8 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometHashAggregate [i_category,i_current_price] + CometFilter [i_category] + CometScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt index 1429e39da..b4d100762 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt @@ -1,183 +1,185 @@ == Physical Plan == -* Sort (179) -+- Exchange (178) - +- * Project (177) - +- * SortMergeJoin Inner (176) - :- * Sort (108) - : +- Exchange (107) - : +- * HashAggregate (106) - : +- * HashAggregate (105) - : +- * Project (104) - : +- * BroadcastHashJoin Inner BuildRight (103) - : :- * Project (97) - : : +- * BroadcastHashJoin Inner BuildRight (96) - : : :- * Project (94) - : : : +- * BroadcastHashJoin Inner BuildRight (93) - : : : :- * Project (88) - : : : : +- * BroadcastHashJoin Inner BuildRight (87) - : : : : :- * Project (85) - : : : : : +- * BroadcastHashJoin Inner BuildRight (84) - : : : : : :- * Project (79) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (78) - : : : : : : :- * Project (76) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (75) - : : : : : : : :- * Project (70) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (69) - : : : : : : : : :- * Project (64) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (63) - : : : : : : : : : :- * Project (61) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (60) - : : : : : : : : : : :- * Project (55) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : : : : : : : : : :- * Project (52) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (51) - : : : : : : : : : : : : :- * Project (46) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (45) - : : : : : : : : : : : : : :- * Project (40) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (39) - : : : : : : : : : : : : : : :- * Project (34) - : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : : : : : : : : : : : : : :- * ColumnarToRow (31) - : : : : : : : : : : : : : : : : +- CometProject (30) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (29) - : : : : : : : : : : : : : : : : :- CometSort (10) - : : : : : : : : : : : : : : : : : +- CometExchange (9) - : : : : : : : : : : : : : : : : : +- CometProject (8) - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) - : : : : : : : : : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- CometProject (6) - : : : : : : : : : : : : : : : : : +- CometFilter (5) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : : : : : : : : : : : : : : : +- CometSort (28) - : : : : : : : : : : : : : : : : +- CometProject (27) - : : : : : : : : : : : : : : : : +- CometFilter (26) - : : : : : : : : : : : : : : : : +- CometHashAggregate (25) - : : : : : : : : : : : : : : : : +- CometExchange (24) - : : : : : : : : : : : : : : : : +- CometHashAggregate (23) - : : : : : : : : : : : : : : : : +- CometProject (22) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (21) - : : : : : : : : : : : : : : : : :- CometSort (15) - : : : : : : : : : : : : : : : : : +- CometExchange (14) - : : : : : : : : : : : : : : : : : +- CometProject (13) - : : : : : : : : : : : : : : : : : +- CometFilter (12) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (11) - : : : : : : : : : : : : : : : : +- CometSort (20) - : : : : : : : : : : : : : : : : +- CometExchange (19) - : : : : : : : : : : : : : : : : +- CometProject (18) - : : : : : : : : : : : : : : : : +- CometFilter (17) - : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (16) - : : : : : : : : : : : : : : : +- ReusedExchange (32) - : : : : : : : : : : : : : : +- BroadcastExchange (38) - : : : : : : : : : : : : : : +- * ColumnarToRow (37) - : : : : : : : : : : : : : : +- CometFilter (36) - : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (35) - : : : : : : : : : : : : : +- BroadcastExchange (44) - : : : : : : : : : : : : : +- * ColumnarToRow (43) - : : : : : : : : : : : : : +- CometFilter (42) - : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (41) - : : : : : : : : : : : : +- BroadcastExchange (50) - : : : : : : : : : : : : +- * ColumnarToRow (49) - : : : : : : : : : : : : +- CometFilter (48) - : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (47) - : : : : : : : : : : : +- ReusedExchange (53) - : : : : : : : : : : +- BroadcastExchange (59) - : : : : : : : : : : +- * ColumnarToRow (58) - : : : : : : : : : : +- CometFilter (57) - : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (56) - : : : : : : : : : +- ReusedExchange (62) - : : : : : : : : +- BroadcastExchange (68) - : : : : : : : : +- * ColumnarToRow (67) - : : : : : : : : +- CometFilter (66) - : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (65) - : : : : : : : +- BroadcastExchange (74) - : : : : : : : +- * ColumnarToRow (73) - : : : : : : : +- CometFilter (72) - : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (71) - : : : : : : +- ReusedExchange (77) - : : : : : +- BroadcastExchange (83) - : : : : : +- * ColumnarToRow (82) - : : : : : +- CometFilter (81) - : : : : : +- CometScan parquet spark_catalog.default.customer_address (80) - : : : : +- ReusedExchange (86) - : : : +- BroadcastExchange (92) - : : : +- * ColumnarToRow (91) - : : : +- CometFilter (90) - : : : +- CometScan parquet spark_catalog.default.income_band (89) - : : +- ReusedExchange (95) - : +- BroadcastExchange (102) - : +- * ColumnarToRow (101) - : +- CometProject (100) - : +- CometFilter (99) - : +- CometScan parquet spark_catalog.default.item (98) - +- * Sort (175) - +- Exchange (174) - +- * HashAggregate (173) - +- * HashAggregate (172) - +- * Project (171) - +- * BroadcastHashJoin Inner BuildRight (170) - :- * Project (168) - : +- * BroadcastHashJoin Inner BuildRight (167) - : :- * Project (165) - : : +- * BroadcastHashJoin Inner BuildRight (164) - : : :- * Project (162) - : : : +- * BroadcastHashJoin Inner BuildRight (161) - : : : :- * Project (159) - : : : : +- * BroadcastHashJoin Inner BuildRight (158) - : : : : :- * Project (156) - : : : : : +- * BroadcastHashJoin Inner BuildRight (155) - : : : : : :- * Project (153) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (152) - : : : : : : :- * Project (150) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (149) - : : : : : : : :- * Project (147) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (146) - : : : : : : : : :- * Project (144) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (143) - : : : : : : : : : :- * Project (141) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (140) - : : : : : : : : : : :- * Project (138) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (137) - : : : : : : : : : : : :- * Project (135) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (134) - : : : : : : : : : : : : :- * Project (132) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (131) - : : : : : : : : : : : : : :- * Project (129) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (128) - : : : : : : : : : : : : : : :- * ColumnarToRow (126) - : : : : : : : : : : : : : : : +- CometProject (125) - : : : : : : : : : : : : : : : +- CometSortMergeJoin (124) - : : : : : : : : : : : : : : : :- CometSort (118) - : : : : : : : : : : : : : : : : +- CometExchange (117) - : : : : : : : : : : : : : : : : +- CometProject (116) - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (115) - : : : : : : : : : : : : : : : : :- CometBroadcastExchange (111) - : : : : : : : : : : : : : : : : : +- CometFilter (110) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (109) - : : : : : : : : : : : : : : : : +- CometProject (114) - : : : : : : : : : : : : : : : : +- CometFilter (113) - : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (112) - : : : : : : : : : : : : : : : +- CometSort (123) - : : : : : : : : : : : : : : : +- CometProject (122) - : : : : : : : : : : : : : : : +- CometFilter (121) - : : : : : : : : : : : : : : : +- CometHashAggregate (120) - : : : : : : : : : : : : : : : +- ReusedExchange (119) - : : : : : : : : : : : : : : +- ReusedExchange (127) - : : : : : : : : : : : : : +- ReusedExchange (130) - : : : : : : : : : : : : +- ReusedExchange (133) - : : : : : : : : : : : +- ReusedExchange (136) - : : : : : : : : : : +- ReusedExchange (139) - : : : : : : : : : +- ReusedExchange (142) - : : : : : : : : +- ReusedExchange (145) - : : : : : : : +- ReusedExchange (148) - : : : : : : +- ReusedExchange (151) - : : : : : +- ReusedExchange (154) - : : : : +- ReusedExchange (157) - : : : +- ReusedExchange (160) - : : +- ReusedExchange (163) - : +- ReusedExchange (166) - +- ReusedExchange (169) +* Sort (181) ++- Exchange (180) + +- * Project (179) + +- * SortMergeJoin Inner (178) + :- * Sort (110) + : +- Exchange (109) + : +- * HashAggregate (108) + : +- * HashAggregate (107) + : +- * Project (106) + : +- * BroadcastHashJoin Inner BuildRight (105) + : :- * Project (99) + : : +- * BroadcastHashJoin Inner BuildRight (98) + : : :- * Project (96) + : : : +- * BroadcastHashJoin Inner BuildRight (95) + : : : :- * Project (90) + : : : : +- * BroadcastHashJoin Inner BuildRight (89) + : : : : :- * Project (87) + : : : : : +- * BroadcastHashJoin Inner BuildRight (86) + : : : : : :- * Project (81) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (80) + : : : : : : :- * Project (78) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (77) + : : : : : : : :- * Project (72) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (71) + : : : : : : : : :- * Project (66) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (65) + : : : : : : : : : :- * Project (63) + : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (62) + : : : : : : : : : : :- * Project (57) + : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (56) + : : : : : : : : : : : :- * Project (54) + : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (53) + : : : : : : : : : : : : :- * Project (48) + : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (47) + : : : : : : : : : : : : : :- * Project (42) + : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (41) + : : : : : : : : : : : : : : :- * Project (36) + : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (35) + : : : : : : : : : : : : : : : :- * Project (33) + : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (32) + : : : : : : : : : : : : : : : : :- * Sort (11) + : : : : : : : : : : : : : : : : : +- Exchange (10) + : : : : : : : : : : : : : : : : : +- * ColumnarToRow (9) + : : : : : : : : : : : : : : : : : +- CometProject (8) + : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) + : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) + : : : : : : : : : : : : : : : : : : +- CometFilter (2) + : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : : : : : : : : : : : : +- CometProject (6) + : : : : : : : : : : : : : : : : : +- CometFilter (5) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : : : : : : : : : : : : : : : +- * Sort (31) + : : : : : : : : : : : : : : : : +- * Project (30) + : : : : : : : : : : : : : : : : +- * Filter (29) + : : : : : : : : : : : : : : : : +- * HashAggregate (28) + : : : : : : : : : : : : : : : : +- Exchange (27) + : : : : : : : : : : : : : : : : +- * HashAggregate (26) + : : : : : : : : : : : : : : : : +- * Project (25) + : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (24) + : : : : : : : : : : : : : : : : :- * Sort (17) + : : : : : : : : : : : : : : : : : +- Exchange (16) + : : : : : : : : : : : : : : : : : +- * ColumnarToRow (15) + : : : : : : : : : : : : : : : : : +- CometProject (14) + : : : : : : : : : : : : : : : : : +- CometFilter (13) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (12) + : : : : : : : : : : : : : : : : +- * Sort (23) + : : : : : : : : : : : : : : : : +- Exchange (22) + : : : : : : : : : : : : : : : : +- * ColumnarToRow (21) + : : : : : : : : : : : : : : : : +- CometProject (20) + : : : : : : : : : : : : : : : : +- CometFilter (19) + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (18) + : : : : : : : : : : : : : : : +- ReusedExchange (34) + : : : : : : : : : : : : : : +- BroadcastExchange (40) + : : : : : : : : : : : : : : +- * ColumnarToRow (39) + : : : : : : : : : : : : : : +- CometFilter (38) + : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (37) + : : : : : : : : : : : : : +- BroadcastExchange (46) + : : : : : : : : : : : : : +- * ColumnarToRow (45) + : : : : : : : : : : : : : +- CometFilter (44) + : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (43) + : : : : : : : : : : : : +- BroadcastExchange (52) + : : : : : : : : : : : : +- * ColumnarToRow (51) + : : : : : : : : : : : : +- CometFilter (50) + : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (49) + : : : : : : : : : : : +- ReusedExchange (55) + : : : : : : : : : : +- BroadcastExchange (61) + : : : : : : : : : : +- * ColumnarToRow (60) + : : : : : : : : : : +- CometFilter (59) + : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (58) + : : : : : : : : : +- ReusedExchange (64) + : : : : : : : : +- BroadcastExchange (70) + : : : : : : : : +- * ColumnarToRow (69) + : : : : : : : : +- CometFilter (68) + : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (67) + : : : : : : : +- BroadcastExchange (76) + : : : : : : : +- * ColumnarToRow (75) + : : : : : : : +- CometFilter (74) + : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (73) + : : : : : : +- ReusedExchange (79) + : : : : : +- BroadcastExchange (85) + : : : : : +- * ColumnarToRow (84) + : : : : : +- CometFilter (83) + : : : : : +- CometScan parquet spark_catalog.default.customer_address (82) + : : : : +- ReusedExchange (88) + : : : +- BroadcastExchange (94) + : : : +- * ColumnarToRow (93) + : : : +- CometFilter (92) + : : : +- CometScan parquet spark_catalog.default.income_band (91) + : : +- ReusedExchange (97) + : +- BroadcastExchange (104) + : +- * ColumnarToRow (103) + : +- CometProject (102) + : +- CometFilter (101) + : +- CometScan parquet spark_catalog.default.item (100) + +- * Sort (177) + +- Exchange (176) + +- * HashAggregate (175) + +- * HashAggregate (174) + +- * Project (173) + +- * BroadcastHashJoin Inner BuildRight (172) + :- * Project (170) + : +- * BroadcastHashJoin Inner BuildRight (169) + : :- * Project (167) + : : +- * BroadcastHashJoin Inner BuildRight (166) + : : :- * Project (164) + : : : +- * BroadcastHashJoin Inner BuildRight (163) + : : : :- * Project (161) + : : : : +- * BroadcastHashJoin Inner BuildRight (160) + : : : : :- * Project (158) + : : : : : +- * BroadcastHashJoin Inner BuildRight (157) + : : : : : :- * Project (155) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (154) + : : : : : : :- * Project (152) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (151) + : : : : : : : :- * Project (149) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (148) + : : : : : : : : :- * Project (146) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (145) + : : : : : : : : : :- * Project (143) + : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (142) + : : : : : : : : : : :- * Project (140) + : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (139) + : : : : : : : : : : : :- * Project (137) + : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (136) + : : : : : : : : : : : : :- * Project (134) + : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (133) + : : : : : : : : : : : : : :- * Project (131) + : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (130) + : : : : : : : : : : : : : : :- * Project (128) + : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (127) + : : : : : : : : : : : : : : : :- * Sort (121) + : : : : : : : : : : : : : : : : +- Exchange (120) + : : : : : : : : : : : : : : : : +- * ColumnarToRow (119) + : : : : : : : : : : : : : : : : +- CometProject (118) + : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (117) + : : : : : : : : : : : : : : : : :- CometBroadcastExchange (113) + : : : : : : : : : : : : : : : : : +- CometFilter (112) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (111) + : : : : : : : : : : : : : : : : +- CometProject (116) + : : : : : : : : : : : : : : : : +- CometFilter (115) + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (114) + : : : : : : : : : : : : : : : +- * Sort (126) + : : : : : : : : : : : : : : : +- * Project (125) + : : : : : : : : : : : : : : : +- * Filter (124) + : : : : : : : : : : : : : : : +- * HashAggregate (123) + : : : : : : : : : : : : : : : +- ReusedExchange (122) + : : : : : : : : : : : : : : +- ReusedExchange (129) + : : : : : : : : : : : : : +- ReusedExchange (132) + : : : : : : : : : : : : +- ReusedExchange (135) + : : : : : : : : : : : +- ReusedExchange (138) + : : : : : : : : : : +- ReusedExchange (141) + : : : : : : : : : +- ReusedExchange (144) + : : : : : : : : +- ReusedExchange (147) + : : : : : : : +- ReusedExchange (150) + : : : : : : +- ReusedExchange (153) + : : : : : +- ReusedExchange (156) + : : : : +- ReusedExchange (159) + : : : +- ReusedExchange (162) + : : +- ReusedExchange (165) + : +- ReusedExchange (168) + +- ReusedExchange (171) (unknown) Scan parquet spark_catalog.default.store_sales @@ -220,13 +222,16 @@ Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number# Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -(9) CometExchange +(9) ColumnarToRow [codegen id : 1] Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(10) CometSort +(10) Exchange Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1 ASC NULLS FIRST] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(11) Sort [codegen id : 2] +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.catalog_sales Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] @@ -235,21 +240,24 @@ Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_order_number)] ReadSchema: struct -(12) CometFilter +(13) CometFilter Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) -(13) CometProject +(14) CometProject Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -(14) CometExchange +(15) ColumnarToRow [codegen id : 3] Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(15) CometSort +(16) Exchange Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST] +Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(17) Sort [codegen id : 4] +Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] +Arguments: [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.catalog_returns Output [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] @@ -258,790 +266,799 @@ Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(17) CometFilter +(19) CometFilter Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) -(18) CometProject +(20) CometProject Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -(19) CometExchange +(21) ColumnarToRow [codegen id : 5] Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(20) CometSort +(22) Exchange Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST] +Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) CometSortMergeJoin -Left output [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Right output [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number#22], Inner +(23) Sort [codegen id : 6] +Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +Arguments: [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST], false, 0 -(22) CometProject +(24) SortMergeJoin [codegen id : 7] +Left keys [2]: [cs_item_sk#17, cs_order_number#18] +Right keys [2]: [cr_item_sk#21, cr_order_number#22] +Join type: Inner +Join condition: None + +(25) Project [codegen id : 7] +Output [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -(23) CometHashAggregate +(26) HashAggregate [codegen id : 7] Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Keys [1]: [cs_item_sk#17] Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] +Aggregate Attributes [3]: [sum#27, sum#28, isEmpty#29] +Results [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] -(24) CometExchange -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(27) Exchange +Input [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] +Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(25) CometHashAggregate -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] +(28) HashAggregate [codegen id : 8] +Input [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] Keys [1]: [cs_item_sk#17] Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#19))#33, sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))#34] +Results [3]: [cs_item_sk#17, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#19))#33,17,2) AS sale#35, sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))#34 AS refund#36] -(26) CometFilter -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) +(29) Filter [codegen id : 8] +Input [3]: [cs_item_sk#17, sale#35, refund#36] +Condition : ((isnotnull(sale#35) AND isnotnull(refund#36)) AND (cast(sale#35 as decimal(21,2)) > (2 * refund#36))) -(27) CometProject -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Arguments: [cs_item_sk#17], [cs_item_sk#17] +(30) Project [codegen id : 8] +Output [1]: [cs_item_sk#17] +Input [3]: [cs_item_sk#17, sale#35, refund#36] -(28) CometSort +(31) Sort [codegen id : 8] Input [1]: [cs_item_sk#17] -Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] +Arguments: [cs_item_sk#17 ASC NULLS FIRST], false, 0 -(29) CometSortMergeJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [1]: [cs_item_sk#17] -Arguments: [ss_item_sk#1], [cs_item_sk#17], Inner +(32) SortMergeJoin [codegen id : 24] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [cs_item_sk#17] +Join type: Inner +Join condition: None -(30) CometProject +(33) Project [codegen id : 24] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#17] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(31) ColumnarToRow [codegen id : 16] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -(32) ReusedExchange [Reuses operator id: 183] -Output [2]: [d_date_sk#32, d_year#33] +(34) ReusedExchange [Reuses operator id: 185] +Output [2]: [d_date_sk#37, d_year#38] -(33) BroadcastHashJoin [codegen id : 16] +(35) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#32] +Right keys [1]: [d_date_sk#37] Join type: Inner Join condition: None -(34) Project [codegen id : 16] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#32, d_year#33] +(36) Project [codegen id : 24] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38] +Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#37, d_year#38] (unknown) Scan parquet spark_catalog.default.store -Output [3]: [s_store_sk#34, s_store_name#35, s_zip#36] +Output [3]: [s_store_sk#39, s_store_name#40, s_zip#41] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_zip)] ReadSchema: struct -(36) CometFilter -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Condition : ((isnotnull(s_store_sk#34) AND isnotnull(s_store_name#35)) AND isnotnull(s_zip#36)) +(38) CometFilter +Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] +Condition : ((isnotnull(s_store_sk#39) AND isnotnull(s_store_name#40)) AND isnotnull(s_zip#41)) -(37) ColumnarToRow [codegen id : 2] -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] +(39) ColumnarToRow [codegen id : 10] +Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] -(38) BroadcastExchange -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] +(40) BroadcastExchange +Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(39) BroadcastHashJoin [codegen id : 16] +(41) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ss_store_sk#6] -Right keys [1]: [s_store_sk#34] +Right keys [1]: [s_store_sk#39] Join type: Inner Join condition: None -(40) Project [codegen id : 16] -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_sk#34, s_store_name#35, s_zip#36] +(42) Project [codegen id : 24] +Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41] +Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_sk#39, s_store_name#40, s_zip#41] (unknown) Scan parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +Output [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(42) CometFilter -Input [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] -Condition : (((((isnotnull(c_customer_sk#37) AND isnotnull(c_first_sales_date_sk#42)) AND isnotnull(c_first_shipto_date_sk#41)) AND isnotnull(c_current_cdemo_sk#38)) AND isnotnull(c_current_hdemo_sk#39)) AND isnotnull(c_current_addr_sk#40)) +(44) CometFilter +Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] +Condition : (((((isnotnull(c_customer_sk#42) AND isnotnull(c_first_sales_date_sk#47)) AND isnotnull(c_first_shipto_date_sk#46)) AND isnotnull(c_current_cdemo_sk#43)) AND isnotnull(c_current_hdemo_sk#44)) AND isnotnull(c_current_addr_sk#45)) -(43) ColumnarToRow [codegen id : 3] -Input [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +(45) ColumnarToRow [codegen id : 11] +Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] -(44) BroadcastExchange -Input [6]: [c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +(46) BroadcastExchange +Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(45) BroadcastHashJoin [codegen id : 16] +(47) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#37] +Right keys [1]: [c_customer_sk#42] Join type: Inner Join condition: None -(46) Project [codegen id : 16] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_customer_sk#37, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42] +(48) Project [codegen id : 24] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] +Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_year#44] +Output [2]: [d_date_sk#48, d_year#49] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct -(48) CometFilter -Input [2]: [d_date_sk#43, d_year#44] -Condition : isnotnull(d_date_sk#43) +(50) CometFilter +Input [2]: [d_date_sk#48, d_year#49] +Condition : isnotnull(d_date_sk#48) -(49) ColumnarToRow [codegen id : 4] -Input [2]: [d_date_sk#43, d_year#44] +(51) ColumnarToRow [codegen id : 12] +Input [2]: [d_date_sk#48, d_year#49] -(50) BroadcastExchange -Input [2]: [d_date_sk#43, d_year#44] +(52) BroadcastExchange +Input [2]: [d_date_sk#48, d_year#49] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(51) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [c_first_sales_date_sk#42] -Right keys [1]: [d_date_sk#43] +(53) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [c_first_sales_date_sk#47] +Right keys [1]: [d_date_sk#48] Join type: Inner Join condition: None -(52) Project [codegen id : 16] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, d_year#44] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, c_first_sales_date_sk#42, d_date_sk#43, d_year#44] +(54) Project [codegen id : 24] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, d_year#49] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47, d_date_sk#48, d_year#49] -(53) ReusedExchange [Reuses operator id: 50] -Output [2]: [d_date_sk#45, d_year#46] +(55) ReusedExchange [Reuses operator id: 52] +Output [2]: [d_date_sk#50, d_year#51] -(54) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [c_first_shipto_date_sk#41] -Right keys [1]: [d_date_sk#45] +(56) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [c_first_shipto_date_sk#46] +Right keys [1]: [d_date_sk#50] Join type: Inner Join condition: None -(55) Project [codegen id : 16] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, c_first_shipto_date_sk#41, d_year#44, d_date_sk#45, d_year#46] +(57) Project [codegen id : 24] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, d_year#49, d_date_sk#50, d_year#51] (unknown) Scan parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#47, cd_marital_status#48] +Output [2]: [cd_demo_sk#52, cd_marital_status#53] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status)] ReadSchema: struct -(57) CometFilter -Input [2]: [cd_demo_sk#47, cd_marital_status#48] -Condition : (isnotnull(cd_demo_sk#47) AND isnotnull(cd_marital_status#48)) +(59) CometFilter +Input [2]: [cd_demo_sk#52, cd_marital_status#53] +Condition : (isnotnull(cd_demo_sk#52) AND isnotnull(cd_marital_status#53)) -(58) ColumnarToRow [codegen id : 6] -Input [2]: [cd_demo_sk#47, cd_marital_status#48] +(60) ColumnarToRow [codegen id : 14] +Input [2]: [cd_demo_sk#52, cd_marital_status#53] -(59) BroadcastExchange -Input [2]: [cd_demo_sk#47, cd_marital_status#48] +(61) BroadcastExchange +Input [2]: [cd_demo_sk#52, cd_marital_status#53] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(60) BroadcastHashJoin [codegen id : 16] +(62) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ss_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#47] +Right keys [1]: [cd_demo_sk#52] Join type: Inner Join condition: None -(61) Project [codegen id : 16] -Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_marital_status#48] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_demo_sk#47, cd_marital_status#48] +(63) Project [codegen id : 24] +Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_marital_status#53] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_demo_sk#52, cd_marital_status#53] -(62) ReusedExchange [Reuses operator id: 59] -Output [2]: [cd_demo_sk#49, cd_marital_status#50] +(64) ReusedExchange [Reuses operator id: 61] +Output [2]: [cd_demo_sk#54, cd_marital_status#55] -(63) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [c_current_cdemo_sk#38] -Right keys [1]: [cd_demo_sk#49] +(65) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [c_current_cdemo_sk#43] +Right keys [1]: [cd_demo_sk#54] Join type: Inner -Join condition: NOT (cd_marital_status#48 = cd_marital_status#50) +Join condition: NOT (cd_marital_status#53 = cd_marital_status#55) -(64) Project [codegen id : 16] -Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_cdemo_sk#38, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, cd_marital_status#48, cd_demo_sk#49, cd_marital_status#50] +(66) Project [codegen id : 24] +Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] +Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_marital_status#53, cd_demo_sk#54, cd_marital_status#55] (unknown) Scan parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#51] +Output [1]: [p_promo_sk#56] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_promo_sk)] ReadSchema: struct -(66) CometFilter -Input [1]: [p_promo_sk#51] -Condition : isnotnull(p_promo_sk#51) +(68) CometFilter +Input [1]: [p_promo_sk#56] +Condition : isnotnull(p_promo_sk#56) -(67) ColumnarToRow [codegen id : 8] -Input [1]: [p_promo_sk#51] +(69) ColumnarToRow [codegen id : 16] +Input [1]: [p_promo_sk#56] -(68) BroadcastExchange -Input [1]: [p_promo_sk#51] +(70) BroadcastExchange +Input [1]: [p_promo_sk#56] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(69) BroadcastHashJoin [codegen id : 16] +(71) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ss_promo_sk#7] -Right keys [1]: [p_promo_sk#51] +Right keys [1]: [p_promo_sk#56] Join type: Inner Join condition: None -(70) Project [codegen id : 16] -Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, p_promo_sk#51] +(72) Project [codegen id : 24] +Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, p_promo_sk#56] (unknown) Scan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#52, hd_income_band_sk#53] +Output [2]: [hd_demo_sk#57, hd_income_band_sk#58] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] ReadSchema: struct -(72) CometFilter -Input [2]: [hd_demo_sk#52, hd_income_band_sk#53] -Condition : (isnotnull(hd_demo_sk#52) AND isnotnull(hd_income_band_sk#53)) +(74) CometFilter +Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] +Condition : (isnotnull(hd_demo_sk#57) AND isnotnull(hd_income_band_sk#58)) -(73) ColumnarToRow [codegen id : 9] -Input [2]: [hd_demo_sk#52, hd_income_band_sk#53] +(75) ColumnarToRow [codegen id : 17] +Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] -(74) BroadcastExchange -Input [2]: [hd_demo_sk#52, hd_income_band_sk#53] +(76) BroadcastExchange +Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -(75) BroadcastHashJoin [codegen id : 16] +(77) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ss_hdemo_sk#4] -Right keys [1]: [hd_demo_sk#52] +Right keys [1]: [hd_demo_sk#57] Join type: Inner Join condition: None -(76) Project [codegen id : 16] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_demo_sk#52, hd_income_band_sk#53] +(78) Project [codegen id : 24] +Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58] +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_demo_sk#57, hd_income_band_sk#58] -(77) ReusedExchange [Reuses operator id: 74] -Output [2]: [hd_demo_sk#54, hd_income_band_sk#55] +(79) ReusedExchange [Reuses operator id: 76] +Output [2]: [hd_demo_sk#59, hd_income_band_sk#60] -(78) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [c_current_hdemo_sk#39] -Right keys [1]: [hd_demo_sk#54] +(80) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [c_current_hdemo_sk#44] +Right keys [1]: [hd_demo_sk#59] Join type: Inner Join condition: None -(79) Project [codegen id : 16] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55] -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_hdemo_sk#39, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_demo_sk#54, hd_income_band_sk#55] +(81) Project [codegen id : 24] +Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60] +Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_demo_sk#59, hd_income_band_sk#60] (unknown) Scan parquet spark_catalog.default.customer_address -Output [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +Output [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(81) CometFilter -Input [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] -Condition : isnotnull(ca_address_sk#56) +(83) CometFilter +Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Condition : isnotnull(ca_address_sk#61) -(82) ColumnarToRow [codegen id : 11] -Input [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +(84) ColumnarToRow [codegen id : 19] +Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -(83) BroadcastExchange -Input [5]: [ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +(85) BroadcastExchange +Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=11] -(84) BroadcastHashJoin [codegen id : 16] +(86) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ss_addr_sk#5] -Right keys [1]: [ca_address_sk#56] +Right keys [1]: [ca_address_sk#61] Join type: Inner Join condition: None -(85) Project [codegen id : 16] -Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_address_sk#56, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60] +(87) Project [codegen id : 24] +Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -(86) ReusedExchange [Reuses operator id: 83] -Output [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +(88) ReusedExchange [Reuses operator id: 85] +Output [5]: [ca_address_sk#66, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] -(87) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [c_current_addr_sk#40] -Right keys [1]: [ca_address_sk#61] +(89) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [c_current_addr_sk#45] +Right keys [1]: [ca_address_sk#66] Join type: Inner Join condition: None -(88) Project [codegen id : 16] -Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, c_current_addr_sk#40, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +(90) Project [codegen id : 24] +Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] +Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_address_sk#66, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] (unknown) Scan parquet spark_catalog.default.income_band -Output [1]: [ib_income_band_sk#66] +Output [1]: [ib_income_band_sk#71] Batched: true Location [not included in comparison]/{warehouse_dir}/income_band] PushedFilters: [IsNotNull(ib_income_band_sk)] ReadSchema: struct -(90) CometFilter -Input [1]: [ib_income_band_sk#66] -Condition : isnotnull(ib_income_band_sk#66) +(92) CometFilter +Input [1]: [ib_income_band_sk#71] +Condition : isnotnull(ib_income_band_sk#71) -(91) ColumnarToRow [codegen id : 13] -Input [1]: [ib_income_band_sk#66] +(93) ColumnarToRow [codegen id : 21] +Input [1]: [ib_income_band_sk#71] -(92) BroadcastExchange -Input [1]: [ib_income_band_sk#66] +(94) BroadcastExchange +Input [1]: [ib_income_band_sk#71] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -(93) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [hd_income_band_sk#53] -Right keys [1]: [ib_income_band_sk#66] +(95) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [hd_income_band_sk#58] +Right keys [1]: [ib_income_band_sk#71] Join type: Inner Join condition: None -(94) Project [codegen id : 16] -Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#53, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ib_income_band_sk#66] +(96) Project [codegen id : 24] +Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] +Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, ib_income_band_sk#71] -(95) ReusedExchange [Reuses operator id: 92] -Output [1]: [ib_income_band_sk#67] +(97) ReusedExchange [Reuses operator id: 94] +Output [1]: [ib_income_band_sk#72] -(96) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [hd_income_band_sk#55] -Right keys [1]: [ib_income_band_sk#67] +(98) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [hd_income_band_sk#60] +Right keys [1]: [ib_income_band_sk#72] Join type: Inner Join condition: None -(97) Project [codegen id : 16] -Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, hd_income_band_sk#55, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ib_income_band_sk#67] +(99) Project [codegen id : 24] +Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, ib_income_band_sk#72] (unknown) Scan parquet spark_catalog.default.item -Output [4]: [i_item_sk#68, i_current_price#69, i_color#70, i_product_name#71] +Output [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), In(i_color, [burlywood ,floral ,indian ,medium ,purple ,spring ]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] ReadSchema: struct -(99) CometFilter -Input [4]: [i_item_sk#68, i_current_price#69, i_color#70, i_product_name#71] -Condition : ((((((isnotnull(i_current_price#69) AND i_color#70 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#69 >= 64.00)) AND (i_current_price#69 <= 74.00)) AND (i_current_price#69 >= 65.00)) AND (i_current_price#69 <= 79.00)) AND isnotnull(i_item_sk#68)) +(101) CometFilter +Input [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] +Condition : ((((((isnotnull(i_current_price#74) AND i_color#75 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#74 >= 64.00)) AND (i_current_price#74 <= 74.00)) AND (i_current_price#74 >= 65.00)) AND (i_current_price#74 <= 79.00)) AND isnotnull(i_item_sk#73)) -(100) CometProject -Input [4]: [i_item_sk#68, i_current_price#69, i_color#70, i_product_name#71] -Arguments: [i_item_sk#68, i_product_name#71], [i_item_sk#68, i_product_name#71] +(102) CometProject +Input [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] +Arguments: [i_item_sk#73, i_product_name#76], [i_item_sk#73, i_product_name#76] -(101) ColumnarToRow [codegen id : 15] -Input [2]: [i_item_sk#68, i_product_name#71] +(103) ColumnarToRow [codegen id : 23] +Input [2]: [i_item_sk#73, i_product_name#76] -(102) BroadcastExchange -Input [2]: [i_item_sk#68, i_product_name#71] +(104) BroadcastExchange +Input [2]: [i_item_sk#73, i_product_name#76] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -(103) BroadcastHashJoin [codegen id : 16] +(105) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#68] +Right keys [1]: [i_item_sk#73] Join type: Inner Join condition: None -(104) Project [codegen id : 16] -Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#44, d_year#46, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, i_item_sk#68, i_product_name#71] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#36, d_year#44, d_year#46, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, i_item_sk#68, i_product_name#71] +(106) Project [codegen id : 24] +Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, d_year#49, d_year#51, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] -(105) HashAggregate [codegen id : 16] -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#44, d_year#46, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, i_item_sk#68, i_product_name#71] -Keys [15]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46] +(107) HashAggregate [codegen id : 24] +Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, d_year#49, d_year#51, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] +Keys [15]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51] Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count#72, sum#73, sum#74, sum#75] -Results [19]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46, count#76, sum#77, sum#78, sum#79] +Aggregate Attributes [4]: [count#77, sum#78, sum#79, sum#80] +Results [19]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51, count#81, sum#82, sum#83, sum#84] -(106) HashAggregate [codegen id : 16] -Input [19]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46, count#76, sum#77, sum#78, sum#79] -Keys [15]: [i_product_name#71, i_item_sk#68, s_store_name#35, s_zip#36, ca_street_number#57, ca_street_name#58, ca_city#59, ca_zip#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, d_year#33, d_year#44, d_year#46] +(108) HashAggregate [codegen id : 24] +Input [19]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51, count#81, sum#82, sum#83, sum#84] +Keys [15]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51] Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count(1)#80, sum(UnscaledValue(ss_wholesale_cost#9))#81, sum(UnscaledValue(ss_list_price#10))#82, sum(UnscaledValue(ss_coupon_amt#11))#83] -Results [17]: [i_product_name#71 AS product_name#84, i_item_sk#68 AS item_sk#85, s_store_name#35 AS store_name#86, s_zip#36 AS store_zip#87, ca_street_number#57 AS b_street_number#88, ca_street_name#58 AS b_streen_name#89, ca_city#59 AS b_city#90, ca_zip#60 AS b_zip#91, ca_street_number#62 AS c_street_number#92, ca_street_name#63 AS c_street_name#93, ca_city#64 AS c_city#94, ca_zip#65 AS c_zip#95, d_year#33 AS syear#96, count(1)#80 AS cnt#97, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#81,17,2) AS s1#98, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#82,17,2) AS s2#99, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#83,17,2) AS s3#100] +Aggregate Attributes [4]: [count(1)#85, sum(UnscaledValue(ss_wholesale_cost#9))#86, sum(UnscaledValue(ss_list_price#10))#87, sum(UnscaledValue(ss_coupon_amt#11))#88] +Results [17]: [i_product_name#76 AS product_name#89, i_item_sk#73 AS item_sk#90, s_store_name#40 AS store_name#91, s_zip#41 AS store_zip#92, ca_street_number#62 AS b_street_number#93, ca_street_name#63 AS b_streen_name#94, ca_city#64 AS b_city#95, ca_zip#65 AS b_zip#96, ca_street_number#67 AS c_street_number#97, ca_street_name#68 AS c_street_name#98, ca_city#69 AS c_city#99, ca_zip#70 AS c_zip#100, d_year#38 AS syear#101, count(1)#85 AS cnt#102, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#86,17,2) AS s1#103, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#87,17,2) AS s2#104, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#88,17,2) AS s3#105] -(107) Exchange -Input [17]: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100] -Arguments: hashpartitioning(item_sk#85, store_name#86, store_zip#87, 5), ENSURE_REQUIREMENTS, [plan_id=14] +(109) Exchange +Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] +Arguments: hashpartitioning(item_sk#90, store_name#91, store_zip#92, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(108) Sort [codegen id : 17] -Input [17]: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100] -Arguments: [item_sk#85 ASC NULLS FIRST, store_name#86 ASC NULLS FIRST, store_zip#87 ASC NULLS FIRST], false, 0 +(110) Sort [codegen id : 25] +Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] +Arguments: [item_sk#90 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, store_zip#92 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] +Output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#112), dynamicpruningexpression(ss_sold_date_sk#112 IN dynamicpruning#113)] +PartitionFilters: [isnotnull(ss_sold_date_sk#117), dynamicpruningexpression(ss_sold_date_sk#117 IN dynamicpruning#118)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct -(110) CometFilter -Input [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -Condition : (((((((isnotnull(ss_item_sk#101) AND isnotnull(ss_ticket_number#108)) AND isnotnull(ss_store_sk#106)) AND isnotnull(ss_customer_sk#102)) AND isnotnull(ss_cdemo_sk#103)) AND isnotnull(ss_promo_sk#107)) AND isnotnull(ss_hdemo_sk#104)) AND isnotnull(ss_addr_sk#105)) +(112) CometFilter +Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Condition : (((((((isnotnull(ss_item_sk#106) AND isnotnull(ss_ticket_number#113)) AND isnotnull(ss_store_sk#111)) AND isnotnull(ss_customer_sk#107)) AND isnotnull(ss_cdemo_sk#108)) AND isnotnull(ss_promo_sk#112)) AND isnotnull(ss_hdemo_sk#109)) AND isnotnull(ss_addr_sk#110)) -(111) CometBroadcastExchange -Input [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -Arguments: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] +(113) CometBroadcastExchange +Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Arguments: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] (unknown) Scan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#114, sr_ticket_number#115, sr_returned_date_sk#116] +Output [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] ReadSchema: struct -(113) CometFilter -Input [3]: [sr_item_sk#114, sr_ticket_number#115, sr_returned_date_sk#116] -Condition : (isnotnull(sr_item_sk#114) AND isnotnull(sr_ticket_number#115)) - -(114) CometProject -Input [3]: [sr_item_sk#114, sr_ticket_number#115, sr_returned_date_sk#116] -Arguments: [sr_item_sk#114, sr_ticket_number#115], [sr_item_sk#114, sr_ticket_number#115] - -(115) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -Right output [2]: [sr_item_sk#114, sr_ticket_number#115] -Arguments: [ss_item_sk#101, ss_ticket_number#108], [sr_item_sk#114, sr_ticket_number#115], Inner +(115) CometFilter +Input [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] +Condition : (isnotnull(sr_item_sk#119) AND isnotnull(sr_ticket_number#120)) (116) CometProject -Input [14]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_ticket_number#108, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112, sr_item_sk#114, sr_ticket_number#115] -Arguments: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112], [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] - -(117) CometExchange -Input [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -Arguments: hashpartitioning(ss_item_sk#101, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] - -(118) CometSort -Input [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -Arguments: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112], [ss_item_sk#101 ASC NULLS FIRST] - -(119) ReusedExchange [Reuses operator id: 24] -Output [4]: [cs_item_sk#117, sum#118, sum#119, isEmpty#120] - -(120) CometHashAggregate -Input [4]: [cs_item_sk#117, sum#118, sum#119, isEmpty#120] -Keys [1]: [cs_item_sk#117] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#121)), sum(((cr_refunded_cash#122 + cr_reversed_charge#123) + cr_store_credit#124))] - -(121) CometFilter -Input [3]: [cs_item_sk#117, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) - -(122) CometProject -Input [3]: [cs_item_sk#117, sale#30, refund#31] -Arguments: [cs_item_sk#117], [cs_item_sk#117] - -(123) CometSort -Input [1]: [cs_item_sk#117] -Arguments: [cs_item_sk#117], [cs_item_sk#117 ASC NULLS FIRST] - -(124) CometSortMergeJoin -Left output [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] -Right output [1]: [cs_item_sk#117] -Arguments: [ss_item_sk#101], [cs_item_sk#117], Inner - -(125) CometProject -Input [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112, cs_item_sk#117] -Arguments: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112], [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] +Input [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] +Arguments: [sr_item_sk#119, sr_ticket_number#120], [sr_item_sk#119, sr_ticket_number#120] + +(117) CometBroadcastHashJoin +Left output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Right output [2]: [sr_item_sk#119, sr_ticket_number#120] +Arguments: [ss_item_sk#106, ss_ticket_number#113], [sr_item_sk#119, sr_ticket_number#120], Inner + +(118) CometProject +Input [14]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, sr_item_sk#119, sr_ticket_number#120] +Arguments: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117], [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] + +(119) ColumnarToRow [codegen id : 26] +Input [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] + +(120) Exchange +Input [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Arguments: hashpartitioning(ss_item_sk#106, 5), ENSURE_REQUIREMENTS, [plan_id=15] + +(121) Sort [codegen id : 27] +Input [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Arguments: [ss_item_sk#106 ASC NULLS FIRST], false, 0 + +(122) ReusedExchange [Reuses operator id: 27] +Output [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] + +(123) HashAggregate [codegen id : 33] +Input [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] +Keys [1]: [cs_item_sk#122] +Functions [2]: [sum(UnscaledValue(cs_ext_list_price#126)), sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#126))#33, sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))#34] +Results [3]: [cs_item_sk#122, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#126))#33,17,2) AS sale#35, sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))#34 AS refund#36] + +(124) Filter [codegen id : 33] +Input [3]: [cs_item_sk#122, sale#35, refund#36] +Condition : ((isnotnull(sale#35) AND isnotnull(refund#36)) AND (cast(sale#35 as decimal(21,2)) > (2 * refund#36))) + +(125) Project [codegen id : 33] +Output [1]: [cs_item_sk#122] +Input [3]: [cs_item_sk#122, sale#35, refund#36] + +(126) Sort [codegen id : 33] +Input [1]: [cs_item_sk#122] +Arguments: [cs_item_sk#122 ASC NULLS FIRST], false, 0 + +(127) SortMergeJoin [codegen id : 49] +Left keys [1]: [ss_item_sk#106] +Right keys [1]: [cs_item_sk#122] +Join type: Inner +Join condition: None -(126) ColumnarToRow [codegen id : 33] -Input [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112] +(128) Project [codegen id : 49] +Output [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, cs_item_sk#122] -(127) ReusedExchange [Reuses operator id: 187] -Output [2]: [d_date_sk#125, d_year#126] +(129) ReusedExchange [Reuses operator id: 189] +Output [2]: [d_date_sk#130, d_year#131] -(128) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_sold_date_sk#112] -Right keys [1]: [d_date_sk#125] +(130) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_sold_date_sk#117] +Right keys [1]: [d_date_sk#130] Join type: Inner Join condition: None -(129) Project [codegen id : 33] -Output [11]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126] -Input [13]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, ss_sold_date_sk#112, d_date_sk#125, d_year#126] +(131) Project [codegen id : 49] +Output [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131] +Input [13]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, d_date_sk#130, d_year#131] -(130) ReusedExchange [Reuses operator id: 38] -Output [3]: [s_store_sk#127, s_store_name#128, s_zip#129] +(132) ReusedExchange [Reuses operator id: 40] +Output [3]: [s_store_sk#132, s_store_name#133, s_zip#134] -(131) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_store_sk#106] -Right keys [1]: [s_store_sk#127] +(133) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_store_sk#111] +Right keys [1]: [s_store_sk#132] Join type: Inner Join condition: None -(132) Project [codegen id : 33] -Output [12]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129] -Input [14]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_store_sk#106, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_sk#127, s_store_name#128, s_zip#129] +(134) Project [codegen id : 49] +Output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134] +Input [14]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_sk#132, s_store_name#133, s_zip#134] -(133) ReusedExchange [Reuses operator id: 44] -Output [6]: [c_customer_sk#130, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, c_first_sales_date_sk#135] +(135) ReusedExchange [Reuses operator id: 46] +Output [6]: [c_customer_sk#135, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, c_first_sales_date_sk#140] -(134) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_customer_sk#102] -Right keys [1]: [c_customer_sk#130] +(136) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_customer_sk#107] +Right keys [1]: [c_customer_sk#135] Join type: Inner Join condition: None -(135) Project [codegen id : 33] -Output [16]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, c_first_sales_date_sk#135] -Input [18]: [ss_item_sk#101, ss_customer_sk#102, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_customer_sk#130, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, c_first_sales_date_sk#135] +(137) Project [codegen id : 49] +Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, c_first_sales_date_sk#140] +Input [18]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_customer_sk#135, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, c_first_sales_date_sk#140] -(136) ReusedExchange [Reuses operator id: 50] -Output [2]: [d_date_sk#136, d_year#137] +(138) ReusedExchange [Reuses operator id: 52] +Output [2]: [d_date_sk#141, d_year#142] -(137) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [c_first_sales_date_sk#135] -Right keys [1]: [d_date_sk#136] +(139) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [c_first_sales_date_sk#140] +Right keys [1]: [d_date_sk#141] Join type: Inner Join condition: None -(138) Project [codegen id : 33] -Output [16]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, d_year#137] -Input [18]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, c_first_sales_date_sk#135, d_date_sk#136, d_year#137] +(140) Project [codegen id : 49] +Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, d_year#142] +Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, c_first_sales_date_sk#140, d_date_sk#141, d_year#142] -(139) ReusedExchange [Reuses operator id: 50] -Output [2]: [d_date_sk#138, d_year#139] +(141) ReusedExchange [Reuses operator id: 52] +Output [2]: [d_date_sk#143, d_year#144] -(140) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [c_first_shipto_date_sk#134] -Right keys [1]: [d_date_sk#138] +(142) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [c_first_shipto_date_sk#139] +Right keys [1]: [d_date_sk#143] Join type: Inner Join condition: None -(141) Project [codegen id : 33] -Output [16]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139] -Input [18]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, c_first_shipto_date_sk#134, d_year#137, d_date_sk#138, d_year#139] +(143) Project [codegen id : 49] +Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144] +Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, c_first_shipto_date_sk#139, d_year#142, d_date_sk#143, d_year#144] -(142) ReusedExchange [Reuses operator id: 59] -Output [2]: [cd_demo_sk#140, cd_marital_status#141] +(144) ReusedExchange [Reuses operator id: 61] +Output [2]: [cd_demo_sk#145, cd_marital_status#146] -(143) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_cdemo_sk#103] -Right keys [1]: [cd_demo_sk#140] +(145) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_cdemo_sk#108] +Right keys [1]: [cd_demo_sk#145] Join type: Inner Join condition: None -(144) Project [codegen id : 33] -Output [16]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, cd_marital_status#141] -Input [18]: [ss_item_sk#101, ss_cdemo_sk#103, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, cd_demo_sk#140, cd_marital_status#141] +(146) Project [codegen id : 49] +Output [16]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, cd_marital_status#146] +Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, cd_demo_sk#145, cd_marital_status#146] -(145) ReusedExchange [Reuses operator id: 59] -Output [2]: [cd_demo_sk#142, cd_marital_status#143] +(147) ReusedExchange [Reuses operator id: 61] +Output [2]: [cd_demo_sk#147, cd_marital_status#148] -(146) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [c_current_cdemo_sk#131] -Right keys [1]: [cd_demo_sk#142] +(148) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [c_current_cdemo_sk#136] +Right keys [1]: [cd_demo_sk#147] Join type: Inner -Join condition: NOT (cd_marital_status#141 = cd_marital_status#143) +Join condition: NOT (cd_marital_status#146 = cd_marital_status#148) -(147) Project [codegen id : 33] -Output [14]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139] -Input [18]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_cdemo_sk#131, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, cd_marital_status#141, cd_demo_sk#142, cd_marital_status#143] +(149) Project [codegen id : 49] +Output [14]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144] +Input [18]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_cdemo_sk#136, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, cd_marital_status#146, cd_demo_sk#147, cd_marital_status#148] -(148) ReusedExchange [Reuses operator id: 68] -Output [1]: [p_promo_sk#144] +(150) ReusedExchange [Reuses operator id: 70] +Output [1]: [p_promo_sk#149] -(149) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_promo_sk#107] -Right keys [1]: [p_promo_sk#144] +(151) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_promo_sk#112] +Right keys [1]: [p_promo_sk#149] Join type: Inner Join condition: None -(150) Project [codegen id : 33] -Output [13]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139] -Input [15]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_promo_sk#107, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, p_promo_sk#144] +(152) Project [codegen id : 49] +Output [13]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144] +Input [15]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, p_promo_sk#149] -(151) ReusedExchange [Reuses operator id: 74] -Output [2]: [hd_demo_sk#145, hd_income_band_sk#146] +(153) ReusedExchange [Reuses operator id: 76] +Output [2]: [hd_demo_sk#150, hd_income_band_sk#151] -(152) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_hdemo_sk#104] -Right keys [1]: [hd_demo_sk#145] +(154) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_hdemo_sk#109] +Right keys [1]: [hd_demo_sk#150] Join type: Inner Join condition: None -(153) Project [codegen id : 33] -Output [13]: [ss_item_sk#101, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146] -Input [15]: [ss_item_sk#101, ss_hdemo_sk#104, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, hd_demo_sk#145, hd_income_band_sk#146] +(155) Project [codegen id : 49] +Output [13]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151] +Input [15]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, hd_demo_sk#150, hd_income_band_sk#151] -(154) ReusedExchange [Reuses operator id: 74] -Output [2]: [hd_demo_sk#147, hd_income_band_sk#148] +(156) ReusedExchange [Reuses operator id: 76] +Output [2]: [hd_demo_sk#152, hd_income_band_sk#153] -(155) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [c_current_hdemo_sk#132] -Right keys [1]: [hd_demo_sk#147] +(157) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [c_current_hdemo_sk#137] +Right keys [1]: [hd_demo_sk#152] Join type: Inner Join condition: None -(156) Project [codegen id : 33] -Output [13]: [ss_item_sk#101, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148] -Input [15]: [ss_item_sk#101, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_hdemo_sk#132, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_demo_sk#147, hd_income_band_sk#148] +(158) Project [codegen id : 49] +Output [13]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153] +Input [15]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_hdemo_sk#137, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_demo_sk#152, hd_income_band_sk#153] -(157) ReusedExchange [Reuses operator id: 83] -Output [5]: [ca_address_sk#149, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153] +(159) ReusedExchange [Reuses operator id: 85] +Output [5]: [ca_address_sk#154, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] -(158) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_addr_sk#105] -Right keys [1]: [ca_address_sk#149] +(160) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_addr_sk#110] +Right keys [1]: [ca_address_sk#154] Join type: Inner Join condition: None -(159) Project [codegen id : 33] -Output [16]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153] -Input [18]: [ss_item_sk#101, ss_addr_sk#105, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_address_sk#149, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153] +(161) Project [codegen id : 49] +Output [16]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] +Input [18]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_address_sk#154, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] -(160) ReusedExchange [Reuses operator id: 83] -Output [5]: [ca_address_sk#154, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] +(162) ReusedExchange [Reuses operator id: 85] +Output [5]: [ca_address_sk#159, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] -(161) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [c_current_addr_sk#133] -Right keys [1]: [ca_address_sk#154] +(163) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [c_current_addr_sk#138] +Right keys [1]: [ca_address_sk#159] Join type: Inner Join condition: None -(162) Project [codegen id : 33] -Output [19]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] -Input [21]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, c_current_addr_sk#133, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_address_sk#154, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] +(164) Project [codegen id : 49] +Output [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] +Input [21]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, c_current_addr_sk#138, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_address_sk#159, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] -(163) ReusedExchange [Reuses operator id: 92] -Output [1]: [ib_income_band_sk#159] +(165) ReusedExchange [Reuses operator id: 94] +Output [1]: [ib_income_band_sk#164] -(164) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [hd_income_band_sk#146] -Right keys [1]: [ib_income_band_sk#159] +(166) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [hd_income_band_sk#151] +Right keys [1]: [ib_income_band_sk#164] Join type: Inner Join condition: None -(165) Project [codegen id : 33] -Output [18]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] -Input [20]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, hd_income_band_sk#146, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ib_income_band_sk#159] +(167) Project [codegen id : 49] +Output [18]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] +Input [20]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, hd_income_band_sk#151, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, ib_income_band_sk#164] -(166) ReusedExchange [Reuses operator id: 92] -Output [1]: [ib_income_band_sk#160] +(168) ReusedExchange [Reuses operator id: 94] +Output [1]: [ib_income_band_sk#165] -(167) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [hd_income_band_sk#148] -Right keys [1]: [ib_income_band_sk#160] +(169) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [hd_income_band_sk#153] +Right keys [1]: [ib_income_band_sk#165] Join type: Inner Join condition: None -(168) Project [codegen id : 33] -Output [17]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158] -Input [19]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, hd_income_band_sk#148, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ib_income_band_sk#160] +(170) Project [codegen id : 49] +Output [17]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163] +Input [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, hd_income_band_sk#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, ib_income_band_sk#165] -(169) ReusedExchange [Reuses operator id: 102] -Output [2]: [i_item_sk#161, i_product_name#162] +(171) ReusedExchange [Reuses operator id: 104] +Output [2]: [i_item_sk#166, i_product_name#167] -(170) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_item_sk#101] -Right keys [1]: [i_item_sk#161] +(172) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_item_sk#106] +Right keys [1]: [i_item_sk#166] Join type: Inner Join condition: None -(171) Project [codegen id : 33] -Output [18]: [ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, d_year#137, d_year#139, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, i_item_sk#161, i_product_name#162] -Input [19]: [ss_item_sk#101, ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, s_store_name#128, s_zip#129, d_year#137, d_year#139, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, i_item_sk#161, i_product_name#162] - -(172) HashAggregate [codegen id : 33] -Input [18]: [ss_wholesale_cost#109, ss_list_price#110, ss_coupon_amt#111, d_year#126, d_year#137, d_year#139, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, i_item_sk#161, i_product_name#162] -Keys [15]: [i_product_name#162, i_item_sk#161, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, d_year#126, d_year#137, d_year#139] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#109)), partial_sum(UnscaledValue(ss_list_price#110)), partial_sum(UnscaledValue(ss_coupon_amt#111))] -Aggregate Attributes [4]: [count#72, sum#163, sum#164, sum#165] -Results [19]: [i_product_name#162, i_item_sk#161, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, d_year#126, d_year#137, d_year#139, count#76, sum#166, sum#167, sum#168] - -(173) HashAggregate [codegen id : 33] -Input [19]: [i_product_name#162, i_item_sk#161, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, d_year#126, d_year#137, d_year#139, count#76, sum#166, sum#167, sum#168] -Keys [15]: [i_product_name#162, i_item_sk#161, s_store_name#128, s_zip#129, ca_street_number#150, ca_street_name#151, ca_city#152, ca_zip#153, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, d_year#126, d_year#137, d_year#139] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#109)), sum(UnscaledValue(ss_list_price#110)), sum(UnscaledValue(ss_coupon_amt#111))] -Aggregate Attributes [4]: [count(1)#80, sum(UnscaledValue(ss_wholesale_cost#109))#81, sum(UnscaledValue(ss_list_price#110))#82, sum(UnscaledValue(ss_coupon_amt#111))#83] -Results [8]: [i_item_sk#161 AS item_sk#169, s_store_name#128 AS store_name#170, s_zip#129 AS store_zip#171, d_year#126 AS syear#172, count(1)#80 AS cnt#173, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#109))#81,17,2) AS s1#174, MakeDecimal(sum(UnscaledValue(ss_list_price#110))#82,17,2) AS s2#175, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#111))#83,17,2) AS s3#176] - -(174) Exchange -Input [8]: [item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176] -Arguments: hashpartitioning(item_sk#169, store_name#170, store_zip#171, 5), ENSURE_REQUIREMENTS, [plan_id=16] - -(175) Sort [codegen id : 34] -Input [8]: [item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176] -Arguments: [item_sk#169 ASC NULLS FIRST, store_name#170 ASC NULLS FIRST, store_zip#171 ASC NULLS FIRST], false, 0 - -(176) SortMergeJoin [codegen id : 35] -Left keys [3]: [item_sk#85, store_name#86, store_zip#87] -Right keys [3]: [item_sk#169, store_name#170, store_zip#171] +(173) Project [codegen id : 49] +Output [18]: [ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, d_year#142, d_year#144, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, i_item_sk#166, i_product_name#167] +Input [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#134, d_year#142, d_year#144, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, i_item_sk#166, i_product_name#167] + +(174) HashAggregate [codegen id : 49] +Input [18]: [ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, d_year#142, d_year#144, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, i_item_sk#166, i_product_name#167] +Keys [15]: [i_product_name#167, i_item_sk#166, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, d_year#131, d_year#142, d_year#144] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#114)), partial_sum(UnscaledValue(ss_list_price#115)), partial_sum(UnscaledValue(ss_coupon_amt#116))] +Aggregate Attributes [4]: [count#77, sum#168, sum#169, sum#170] +Results [19]: [i_product_name#167, i_item_sk#166, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, d_year#131, d_year#142, d_year#144, count#81, sum#171, sum#172, sum#173] + +(175) HashAggregate [codegen id : 49] +Input [19]: [i_product_name#167, i_item_sk#166, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, d_year#131, d_year#142, d_year#144, count#81, sum#171, sum#172, sum#173] +Keys [15]: [i_product_name#167, i_item_sk#166, s_store_name#133, s_zip#134, ca_street_number#155, ca_street_name#156, ca_city#157, ca_zip#158, ca_street_number#160, ca_street_name#161, ca_city#162, ca_zip#163, d_year#131, d_year#142, d_year#144] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#114)), sum(UnscaledValue(ss_list_price#115)), sum(UnscaledValue(ss_coupon_amt#116))] +Aggregate Attributes [4]: [count(1)#85, sum(UnscaledValue(ss_wholesale_cost#114))#86, sum(UnscaledValue(ss_list_price#115))#87, sum(UnscaledValue(ss_coupon_amt#116))#88] +Results [8]: [i_item_sk#166 AS item_sk#174, s_store_name#133 AS store_name#175, s_zip#134 AS store_zip#176, d_year#131 AS syear#177, count(1)#85 AS cnt#178, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#114))#86,17,2) AS s1#179, MakeDecimal(sum(UnscaledValue(ss_list_price#115))#87,17,2) AS s2#180, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#116))#88,17,2) AS s3#181] + +(176) Exchange +Input [8]: [item_sk#174, store_name#175, store_zip#176, syear#177, cnt#178, s1#179, s2#180, s3#181] +Arguments: hashpartitioning(item_sk#174, store_name#175, store_zip#176, 5), ENSURE_REQUIREMENTS, [plan_id=16] + +(177) Sort [codegen id : 50] +Input [8]: [item_sk#174, store_name#175, store_zip#176, syear#177, cnt#178, s1#179, s2#180, s3#181] +Arguments: [item_sk#174 ASC NULLS FIRST, store_name#175 ASC NULLS FIRST, store_zip#176 ASC NULLS FIRST], false, 0 + +(178) SortMergeJoin [codegen id : 51] +Left keys [3]: [item_sk#90, store_name#91, store_zip#92] +Right keys [3]: [item_sk#174, store_name#175, store_zip#176] Join type: Inner -Join condition: (cnt#173 <= cnt#97) +Join condition: (cnt#178 <= cnt#102) -(177) Project [codegen id : 35] -Output [21]: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173] -Input [25]: [product_name#84, item_sk#85, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, item_sk#169, store_name#170, store_zip#171, syear#172, cnt#173, s1#174, s2#175, s3#176] +(179) Project [codegen id : 51] +Output [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#179, s2#180, s3#181, syear#177, cnt#178] +Input [25]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, item_sk#174, store_name#175, store_zip#176, syear#177, cnt#178, s1#179, s2#180, s3#181] -(178) Exchange -Input [21]: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173] -Arguments: rangepartitioning(product_name#84 ASC NULLS FIRST, store_name#86 ASC NULLS FIRST, cnt#173 ASC NULLS FIRST, s1#98 ASC NULLS FIRST, s1#174 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=17] +(180) Exchange +Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#179, s2#180, s3#181, syear#177, cnt#178] +Arguments: rangepartitioning(product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#178 ASC NULLS FIRST, s1#103 ASC NULLS FIRST, s1#179 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=17] -(179) Sort [codegen id : 36] -Input [21]: [product_name#84, store_name#86, store_zip#87, b_street_number#88, b_streen_name#89, b_city#90, b_zip#91, c_street_number#92, c_street_name#93, c_city#94, c_zip#95, syear#96, cnt#97, s1#98, s2#99, s3#100, s1#174, s2#175, s3#176, syear#172, cnt#173] -Arguments: [product_name#84 ASC NULLS FIRST, store_name#86 ASC NULLS FIRST, cnt#173 ASC NULLS FIRST, s1#98 ASC NULLS FIRST, s1#174 ASC NULLS FIRST], true, 0 +(181) Sort [codegen id : 52] +Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#179, s2#180, s3#181, syear#177, cnt#178] +Arguments: [product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#178 ASC NULLS FIRST, s1#103 ASC NULLS FIRST, s1#179 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (183) -+- * ColumnarToRow (182) - +- CometFilter (181) - +- CometScan parquet spark_catalog.default.date_dim (180) +BroadcastExchange (185) ++- * ColumnarToRow (184) + +- CometFilter (183) + +- CometScan parquet spark_catalog.default.date_dim (182) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] +Output [2]: [d_date_sk#37, d_year#38] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct -(181) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) +(183) CometFilter +Input [2]: [d_date_sk#37, d_year#38] +Condition : ((isnotnull(d_year#38) AND (d_year#38 = 1999)) AND isnotnull(d_date_sk#37)) -(182) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#32, d_year#33] +(184) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#37, d_year#38] -(183) BroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] +(185) BroadcastExchange +Input [2]: [d_date_sk#37, d_year#38] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] -Subquery:2 Hosting operator id = 109 Hosting Expression = ss_sold_date_sk#112 IN dynamicpruning#113 -BroadcastExchange (187) -+- * ColumnarToRow (186) - +- CometFilter (185) - +- CometScan parquet spark_catalog.default.date_dim (184) +Subquery:2 Hosting operator id = 111 Hosting Expression = ss_sold_date_sk#117 IN dynamicpruning#118 +BroadcastExchange (189) ++- * ColumnarToRow (188) + +- CometFilter (187) + +- CometScan parquet spark_catalog.default.date_dim (186) (unknown) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#125, d_year#126] +Output [2]: [d_date_sk#130, d_year#131] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(185) CometFilter -Input [2]: [d_date_sk#125, d_year#126] -Condition : ((isnotnull(d_year#126) AND (d_year#126 = 2000)) AND isnotnull(d_date_sk#125)) +(187) CometFilter +Input [2]: [d_date_sk#130, d_year#131] +Condition : ((isnotnull(d_year#131) AND (d_year#131 = 2000)) AND isnotnull(d_date_sk#130)) -(186) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#125, d_year#126] +(188) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#130, d_year#131] -(187) BroadcastExchange -Input [2]: [d_date_sk#125, d_year#126] +(189) BroadcastExchange +Input [2]: [d_date_sk#130, d_year#131] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=19] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt index e2accd741..2a0bc5bce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt @@ -1,16 +1,16 @@ -WholeStageCodegen (36) +WholeStageCodegen (52) Sort [product_name,store_name,cnt,s1,s1] InputAdapter Exchange [product_name,store_name,cnt,s1,s1] #1 - WholeStageCodegen (35) + WholeStageCodegen (51) Project [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] SortMergeJoin [item_sk,store_name,store_zip,item_sk,store_name,store_zip,cnt,cnt] InputAdapter - WholeStageCodegen (17) + WholeStageCodegen (25) Sort [item_sk,store_name,store_zip] InputAdapter Exchange [item_sk,store_name,store_zip] #2 - WholeStageCodegen (16) + WholeStageCodegen (24) HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] @@ -43,64 +43,84 @@ WholeStageCodegen (36) BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometSortMergeJoin [ss_item_sk,cs_item_sk] - CometSort [ss_item_sk] - CometExchange [ss_item_sk] #3 - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #4 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [sale,refund] - CometHashAggregate [cs_item_sk,sum,sum,isEmpty] - CometExchange [cs_item_sk] #6 - CometHashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number] - CometExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SortMergeJoin [ss_item_sk,cs_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ss_item_sk] + InputAdapter + Exchange [ss_item_sk] #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + CometBroadcastExchange #4 + CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + WholeStageCodegen (8) + Sort [cs_item_sk] + Project [cs_item_sk] + Filter [sale,refund] + HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] + InputAdapter + Exchange [cs_item_sk] #6 + WholeStageCodegen (7) + HashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [sum,sum,isEmpty,sum,sum,isEmpty] + Project [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] + InputAdapter + WholeStageCodegen (4) + Sort [cs_item_sk,cs_order_number] + InputAdapter + Exchange [cs_item_sk,cs_order_number] #7 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] + CometFilter [cs_item_sk,cs_order_number] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + InputAdapter + WholeStageCodegen (6) + Sort [cr_item_sk,cr_order_number] + InputAdapter + Exchange [cr_item_sk,cr_order_number] #8 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometFilter [cr_item_sk,cr_order_number] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #5 InputAdapter BroadcastExchange #9 - WholeStageCodegen (2) + WholeStageCodegen (10) ColumnarToRow InputAdapter CometFilter [s_store_sk,s_store_name,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] InputAdapter BroadcastExchange #10 - WholeStageCodegen (3) + WholeStageCodegen (11) ColumnarToRow InputAdapter CometFilter [c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] InputAdapter BroadcastExchange #11 - WholeStageCodegen (4) + WholeStageCodegen (12) ColumnarToRow InputAdapter CometFilter [d_date_sk] @@ -109,7 +129,7 @@ WholeStageCodegen (36) ReusedExchange [d_date_sk,d_year] #11 InputAdapter BroadcastExchange #12 - WholeStageCodegen (6) + WholeStageCodegen (14) ColumnarToRow InputAdapter CometFilter [cd_demo_sk,cd_marital_status] @@ -118,14 +138,14 @@ WholeStageCodegen (36) ReusedExchange [cd_demo_sk,cd_marital_status] #12 InputAdapter BroadcastExchange #13 - WholeStageCodegen (8) + WholeStageCodegen (16) ColumnarToRow InputAdapter CometFilter [p_promo_sk] CometScan parquet spark_catalog.default.promotion [p_promo_sk] InputAdapter BroadcastExchange #14 - WholeStageCodegen (9) + WholeStageCodegen (17) ColumnarToRow InputAdapter CometFilter [hd_demo_sk,hd_income_band_sk] @@ -134,7 +154,7 @@ WholeStageCodegen (36) ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 InputAdapter BroadcastExchange #15 - WholeStageCodegen (11) + WholeStageCodegen (19) ColumnarToRow InputAdapter CometFilter [ca_address_sk] @@ -143,7 +163,7 @@ WholeStageCodegen (36) ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 InputAdapter BroadcastExchange #16 - WholeStageCodegen (13) + WholeStageCodegen (21) ColumnarToRow InputAdapter CometFilter [ib_income_band_sk] @@ -152,18 +172,18 @@ WholeStageCodegen (36) ReusedExchange [ib_income_band_sk] #16 InputAdapter BroadcastExchange #17 - WholeStageCodegen (15) + WholeStageCodegen (23) ColumnarToRow InputAdapter CometProject [i_item_sk,i_product_name] CometFilter [i_current_price,i_color,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] InputAdapter - WholeStageCodegen (34) + WholeStageCodegen (50) Sort [item_sk,store_name,store_zip] InputAdapter Exchange [item_sk,store_name,store_zip] #18 - WholeStageCodegen (33) + WholeStageCodegen (49) HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] @@ -196,32 +216,39 @@ WholeStageCodegen (36) BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometSortMergeJoin [ss_item_sk,cs_item_sk] - CometSort [ss_item_sk] - CometExchange [ss_item_sk] #19 - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #20 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #21 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [sale,refund] - CometHashAggregate [cs_item_sk,sum,sum,isEmpty] - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SortMergeJoin [ss_item_sk,cs_item_sk] + InputAdapter + WholeStageCodegen (27) + Sort [ss_item_sk] + InputAdapter + Exchange [ss_item_sk] #19 + WholeStageCodegen (26) + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + CometBroadcastExchange #20 + CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #21 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + WholeStageCodegen (33) + Sort [cs_item_sk] + Project [cs_item_sk] + Filter [sale,refund] + HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] + InputAdapter + ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 InputAdapter ReusedExchange [d_date_sk,d_year] #21 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt index 46b9e51d5..00c2dabde 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt @@ -63,9 +63,9 @@ TakeOrderedAndProject (70) : +- * ColumnarToRow (53) : +- CometFilter (52) : +- CometScan parquet spark_catalog.default.promotion (51) - +- * ColumnarToRow (64) - +- CometSort (63) - +- CometExchange (62) + +- * Sort (64) + +- Exchange (63) + +- * ColumnarToRow (62) +- CometProject (61) +- CometFilter (60) +- CometScan parquet spark_catalog.default.catalog_returns (59) @@ -355,28 +355,28 @@ Condition : (isnotnull(cr_item_sk#30) AND isnotnull(cr_order_number#31)) Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30, cr_order_number#31] -(62) CometExchange +(62) ColumnarToRow [codegen id : 12] Input [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: hashpartitioning(cr_item_sk#30, cr_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] -(63) CometSort +(63) Exchange Input [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30 ASC NULLS FIRST, cr_order_number#31 ASC NULLS FIRST] +Arguments: hashpartitioning(cr_item_sk#30, cr_order_number#31, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(64) ColumnarToRow [codegen id : 12] +(64) Sort [codegen id : 13] Input [2]: [cr_item_sk#30, cr_order_number#31] +Arguments: [cr_item_sk#30 ASC NULLS FIRST, cr_order_number#31 ASC NULLS FIRST], false, 0 -(65) SortMergeJoin [codegen id : 13] +(65) SortMergeJoin [codegen id : 14] Left keys [2]: [cs_item_sk#4, cs_order_number#6] Right keys [2]: [cr_item_sk#30, cr_order_number#31] Join type: LeftOuter Join condition: None -(66) Project [codegen id : 13] +(66) Project [codegen id : 14] Output [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#30, cr_order_number#31] -(67) HashAggregate [codegen id : 13] +(67) HashAggregate [codegen id : 14] Input [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] Functions [1]: [partial_count(1)] @@ -387,7 +387,7 @@ Results [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(69) HashAggregate [codegen id : 14] +(69) HashAggregate [codegen id : 15] Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] Functions [1]: [count(1)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt index 5eb8ea527..17fc9dee7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt @@ -1,9 +1,9 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo] - WholeStageCodegen (14) + WholeStageCodegen (15) HashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] [count(1),no_promo,promo,total_cnt,count] InputAdapter Exchange [i_item_desc,w_warehouse_name,d_week_seq] #1 - WholeStageCodegen (13) + WholeStageCodegen (14) HashAggregate [i_item_desc,w_warehouse_name,d_week_seq] [count,count] Project [w_warehouse_name,i_item_desc,d_week_seq] SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] @@ -104,11 +104,13 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom CometFilter [p_promo_sk] CometScan parquet spark_catalog.default.promotion [p_promo_sk] InputAdapter - WholeStageCodegen (12) - ColumnarToRow + WholeStageCodegen (13) + Sort [cr_item_sk,cr_order_number] InputAdapter - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] + Exchange [cr_item_sk,cr_order_number] #12 + WholeStageCodegen (12) + ColumnarToRow + InputAdapter + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt index 3922f7efe..1c5416407 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -TakeOrderedAndProject (132) -+- * Project (131) - +- * SortMergeJoin Inner (130) +TakeOrderedAndProject (129) ++- * Project (128) + +- * SortMergeJoin Inner (127) :- * Sort (71) : +- Exchange (70) : +- * Filter (69) @@ -29,9 +29,9 @@ TakeOrderedAndProject (132) : : : : +- CometFilter (5) : : : : +- CometScan parquet spark_catalog.default.item (4) : : : +- ReusedExchange (11) - : : +- * ColumnarToRow (21) - : : +- CometSort (20) - : : +- CometExchange (19) + : : +- * Sort (21) + : : +- Exchange (20) + : : +- * ColumnarToRow (19) : : +- CometProject (18) : : +- CometFilter (17) : : +- CometScan parquet spark_catalog.default.catalog_returns (16) @@ -48,9 +48,9 @@ TakeOrderedAndProject (132) : : : : : +- CometScan parquet spark_catalog.default.store_sales (24) : : : : +- ReusedExchange (27) : : : +- ReusedExchange (30) - : : +- * ColumnarToRow (40) - : : +- CometSort (39) - : : +- CometExchange (38) + : : +- * Sort (40) + : : +- Exchange (39) + : : +- * ColumnarToRow (38) : : +- CometProject (37) : : +- CometFilter (36) : : +- CometScan parquet spark_catalog.default.store_returns (35) @@ -67,24 +67,24 @@ TakeOrderedAndProject (132) : : : : +- CometScan parquet spark_catalog.default.web_sales (43) : : : +- ReusedExchange (46) : : +- ReusedExchange (49) - : +- * ColumnarToRow (59) - : +- CometSort (58) - : +- CometExchange (57) + : +- * Sort (59) + : +- Exchange (58) + : +- * ColumnarToRow (57) : +- CometProject (56) : +- CometFilter (55) : +- CometScan parquet spark_catalog.default.web_returns (54) - +- * Sort (129) - +- Exchange (128) - +- * Filter (127) - +- * HashAggregate (126) - +- Exchange (125) - +- * HashAggregate (124) - +- * HashAggregate (123) - +- Exchange (122) - +- * HashAggregate (121) - +- Union (120) - :- * Project (87) - : +- * SortMergeJoin LeftOuter (86) + +- * Sort (126) + +- Exchange (125) + +- * Filter (124) + +- * HashAggregate (123) + +- Exchange (122) + +- * HashAggregate (121) + +- * HashAggregate (120) + +- Exchange (119) + +- * HashAggregate (118) + +- Union (117) + :- * Project (86) + : +- * SortMergeJoin LeftOuter (85) : :- * Sort (82) : : +- Exchange (81) : : +- * Project (80) @@ -96,41 +96,38 @@ TakeOrderedAndProject (132) : : : : +- CometScan parquet spark_catalog.default.catalog_sales (72) : : : +- ReusedExchange (75) : : +- ReusedExchange (78) - : +- * ColumnarToRow (85) - : +- CometSort (84) - : +- ReusedExchange (83) - :- * Project (103) - : +- * SortMergeJoin LeftOuter (102) - : :- * Sort (98) - : : +- Exchange (97) - : : +- * Project (96) - : : +- * BroadcastHashJoin Inner BuildRight (95) - : : :- * Project (93) - : : : +- * BroadcastHashJoin Inner BuildRight (92) - : : : :- * ColumnarToRow (90) - : : : : +- CometFilter (89) - : : : : +- CometScan parquet spark_catalog.default.store_sales (88) - : : : +- ReusedExchange (91) - : : +- ReusedExchange (94) - : +- * ColumnarToRow (101) - : +- CometSort (100) - : +- ReusedExchange (99) - +- * Project (119) - +- * SortMergeJoin LeftOuter (118) - :- * Sort (114) - : +- Exchange (113) - : +- * Project (112) - : +- * BroadcastHashJoin Inner BuildRight (111) - : :- * Project (109) - : : +- * BroadcastHashJoin Inner BuildRight (108) - : : :- * ColumnarToRow (106) - : : : +- CometFilter (105) - : : : +- CometScan parquet spark_catalog.default.web_sales (104) - : : +- ReusedExchange (107) - : +- ReusedExchange (110) - +- * ColumnarToRow (117) - +- CometSort (116) - +- ReusedExchange (115) + : +- * Sort (84) + : +- ReusedExchange (83) + :- * Project (101) + : +- * SortMergeJoin LeftOuter (100) + : :- * Sort (97) + : : +- Exchange (96) + : : +- * Project (95) + : : +- * BroadcastHashJoin Inner BuildRight (94) + : : :- * Project (92) + : : : +- * BroadcastHashJoin Inner BuildRight (91) + : : : :- * ColumnarToRow (89) + : : : : +- CometFilter (88) + : : : : +- CometScan parquet spark_catalog.default.store_sales (87) + : : : +- ReusedExchange (90) + : : +- ReusedExchange (93) + : +- * Sort (99) + : +- ReusedExchange (98) + +- * Project (116) + +- * SortMergeJoin LeftOuter (115) + :- * Sort (112) + : +- Exchange (111) + : +- * Project (110) + : +- * BroadcastHashJoin Inner BuildRight (109) + : :- * Project (107) + : : +- * BroadcastHashJoin Inner BuildRight (106) + : : :- * ColumnarToRow (104) + : : : +- CometFilter (103) + : : : +- CometScan parquet spark_catalog.default.web_sales (102) + : : +- ReusedExchange (105) + : +- ReusedExchange (108) + +- * Sort (114) + +- ReusedExchange (113) (unknown) Scan parquet spark_catalog.default.catalog_sales @@ -180,7 +177,7 @@ Join condition: None Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -(11) ReusedExchange [Reuses operator id: 136] +(11) ReusedExchange [Reuses operator id: 133] Output [2]: [d_date_sk#13, d_year#14] (12) BroadcastHashJoin [codegen id : 3] @@ -216,24 +213,24 @@ Condition : (isnotnull(cr_order_number#16) AND isnotnull(cr_item_sk#15)) Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -(19) CometExchange +(19) ColumnarToRow [codegen id : 5] Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(20) CometSort +(20) Exchange Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST] +Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) ColumnarToRow [codegen id : 5] +(21) Sort [codegen id : 6] Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] +Arguments: [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST], false, 0 -(22) SortMergeJoin [codegen id : 6] +(22) SortMergeJoin [codegen id : 7] Left keys [2]: [cs_order_number#2, cs_item_sk#1] Right keys [2]: [cr_order_number#16, cr_item_sk#15] Join type: LeftOuter Join condition: None -(23) Project [codegen id : 6] +(23) Project [codegen id : 7] Output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, (cs_ext_sales_price#4 - coalesce(cr_return_amount#18, 0.00)) AS sales_amt#21] Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14, cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] @@ -249,32 +246,32 @@ ReadSchema: struct -(89) CometFilter +(88) CometFilter Input [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] Condition : isnotnull(ss_item_sk#87) -(90) ColumnarToRow [codegen id : 31] +(89) ColumnarToRow [codegen id : 35] Input [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] -(91) ReusedExchange [Reuses operator id: 8] +(90) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] -(92) BroadcastHashJoin [codegen id : 31] +(91) BroadcastHashJoin [codegen id : 35] Left keys [1]: [ss_item_sk#87] Right keys [1]: [i_item_sk#93] Join type: Inner Join condition: None -(93) Project [codegen id : 31] +(92) Project [codegen id : 35] Output [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] Input [10]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] -(94) ReusedExchange [Reuses operator id: 140] +(93) ReusedExchange [Reuses operator id: 137] Output [2]: [d_date_sk#98, d_year#99] -(95) BroadcastHashJoin [codegen id : 31] +(94) BroadcastHashJoin [codegen id : 35] Left keys [1]: [ss_sold_date_sk#91] Right keys [1]: [d_date_sk#98] Join type: Inner Join condition: None -(96) Project [codegen id : 31] +(95) Project [codegen id : 35] Output [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] Input [11]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_date_sk#98, d_year#99] -(97) Exchange +(96) Exchange Input [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] Arguments: hashpartitioning(ss_ticket_number#88, ss_item_sk#87, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(98) Sort [codegen id : 32] +(97) Sort [codegen id : 36] Input [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] Arguments: [ss_ticket_number#88 ASC NULLS FIRST, ss_item_sk#87 ASC NULLS FIRST], false, 0 -(99) ReusedExchange [Reuses operator id: 38] +(98) ReusedExchange [Reuses operator id: 39] Output [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] -(100) CometSort +(99) Sort [codegen id : 38] Input [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] -Arguments: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103], [sr_ticket_number#101 ASC NULLS FIRST, sr_item_sk#100 ASC NULLS FIRST] +Arguments: [sr_ticket_number#101 ASC NULLS FIRST, sr_item_sk#100 ASC NULLS FIRST], false, 0 -(101) ColumnarToRow [codegen id : 33] -Input [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] - -(102) SortMergeJoin [codegen id : 34] +(100) SortMergeJoin [codegen id : 39] Left keys [2]: [ss_ticket_number#88, ss_item_sk#87] Right keys [2]: [sr_ticket_number#101, sr_item_sk#100] Join type: LeftOuter Join condition: None -(103) Project [codegen id : 34] +(101) Project [codegen id : 39] Output [7]: [d_year#99, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, (ss_quantity#89 - coalesce(sr_return_quantity#102, 0)) AS sales_cnt#40, (ss_ext_sales_price#90 - coalesce(sr_return_amt#103, 0.00)) AS sales_amt#41] Input [13]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99, sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] @@ -603,138 +594,135 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#108), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(105) CometFilter +(103) CometFilter Input [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] Condition : isnotnull(ws_item_sk#104) -(106) ColumnarToRow [codegen id : 37] +(104) ColumnarToRow [codegen id : 42] Input [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] -(107) ReusedExchange [Reuses operator id: 8] +(105) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -(108) BroadcastHashJoin [codegen id : 37] +(106) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ws_item_sk#104] Right keys [1]: [i_item_sk#110] Join type: Inner Join condition: None -(109) Project [codegen id : 37] +(107) Project [codegen id : 42] Output [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] Input [10]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -(110) ReusedExchange [Reuses operator id: 140] +(108) ReusedExchange [Reuses operator id: 137] Output [2]: [d_date_sk#115, d_year#116] -(111) BroadcastHashJoin [codegen id : 37] +(109) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ws_sold_date_sk#108] Right keys [1]: [d_date_sk#115] Join type: Inner Join condition: None -(112) Project [codegen id : 37] +(110) Project [codegen id : 42] Output [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] Input [11]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_date_sk#115, d_year#116] -(113) Exchange +(111) Exchange Input [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] Arguments: hashpartitioning(ws_order_number#105, ws_item_sk#104, 5), ENSURE_REQUIREMENTS, [plan_id=13] -(114) Sort [codegen id : 38] +(112) Sort [codegen id : 43] Input [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] Arguments: [ws_order_number#105 ASC NULLS FIRST, ws_item_sk#104 ASC NULLS FIRST], false, 0 -(115) ReusedExchange [Reuses operator id: 57] +(113) ReusedExchange [Reuses operator id: 58] Output [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] -(116) CometSort -Input [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] -Arguments: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120], [wr_order_number#118 ASC NULLS FIRST, wr_item_sk#117 ASC NULLS FIRST] - -(117) ColumnarToRow [codegen id : 39] +(114) Sort [codegen id : 45] Input [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] +Arguments: [wr_order_number#118 ASC NULLS FIRST, wr_item_sk#117 ASC NULLS FIRST], false, 0 -(118) SortMergeJoin [codegen id : 40] +(115) SortMergeJoin [codegen id : 46] Left keys [2]: [ws_order_number#105, ws_item_sk#104] Right keys [2]: [wr_order_number#118, wr_item_sk#117] Join type: LeftOuter Join condition: None -(119) Project [codegen id : 40] +(116) Project [codegen id : 46] Output [7]: [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, (ws_quantity#106 - coalesce(wr_return_quantity#119, 0)) AS sales_cnt#60, (ws_ext_sales_price#107 - coalesce(wr_return_amt#120, 0.00)) AS sales_amt#61] Input [13]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116, wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] -(120) Union +(117) Union -(121) HashAggregate [codegen id : 41] +(118) HashAggregate [codegen id : 47] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] Functions: [] Aggregate Attributes: [] Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] -(122) Exchange +(119) Exchange Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] Arguments: hashpartitioning(d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(123) HashAggregate [codegen id : 42] +(120) HashAggregate [codegen id : 48] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] Functions: [] Aggregate Attributes: [] Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] -(124) HashAggregate [codegen id : 42] +(121) HashAggregate [codegen id : 48] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] Keys [5]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] Aggregate Attributes [2]: [sum#62, sum#121] Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#64, sum#122] -(125) Exchange +(122) Exchange Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#64, sum#122] Arguments: hashpartitioning(d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, 5), ENSURE_REQUIREMENTS, [plan_id=15] -(126) HashAggregate [codegen id : 43] +(123) HashAggregate [codegen id : 49] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#64, sum#122] Keys [5]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] Aggregate Attributes [2]: [sum(sales_cnt#20)#66, sum(UnscaledValue(sales_amt#21))#67] Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum(sales_cnt#20)#66 AS sales_cnt#123, MakeDecimal(sum(UnscaledValue(sales_amt#21))#67,18,2) AS sales_amt#124] -(127) Filter [codegen id : 43] +(124) Filter [codegen id : 49] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#123, sales_amt#124] Condition : isnotnull(sales_cnt#123) -(128) Exchange +(125) Exchange Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#123, sales_amt#124] Arguments: hashpartitioning(i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, 5), ENSURE_REQUIREMENTS, [plan_id=16] -(129) Sort [codegen id : 44] +(126) Sort [codegen id : 50] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#123, sales_amt#124] Arguments: [i_brand_id#77 ASC NULLS FIRST, i_class_id#78 ASC NULLS FIRST, i_category_id#79 ASC NULLS FIRST, i_manufact_id#80 ASC NULLS FIRST], false, 0 -(130) SortMergeJoin [codegen id : 45] +(127) SortMergeJoin [codegen id : 51] Left keys [4]: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Right keys [4]: [i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] Join type: Inner Join condition: ((cast(sales_cnt#68 as decimal(17,2)) / cast(sales_cnt#123 as decimal(17,2))) < 0.90000000000000000000) -(131) Project [codegen id : 45] +(128) Project [codegen id : 51] Output [10]: [d_year#82 AS prev_year#125, d_year#14 AS year#126, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#123 AS prev_yr_cnt#127, sales_cnt#68 AS curr_yr_cnt#128, (sales_cnt#68 - sales_cnt#123) AS sales_cnt_diff#129, (sales_amt#69 - sales_amt#124) AS sales_amt_diff#130] Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69, d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#123, sales_amt#124] -(132) TakeOrderedAndProject +(129) TakeOrderedAndProject Input [10]: [prev_year#125, year#126, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#127, curr_yr_cnt#128, sales_cnt_diff#129, sales_amt_diff#130] Arguments: 100, [sales_cnt_diff#129 ASC NULLS FIRST, sales_amt_diff#130 ASC NULLS FIRST], [prev_year#125, year#126, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#127, curr_yr_cnt#128, sales_cnt_diff#129, sales_amt_diff#130] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (136) -+- * ColumnarToRow (135) - +- CometFilter (134) - +- CometScan parquet spark_catalog.default.date_dim (133) +BroadcastExchange (133) ++- * ColumnarToRow (132) + +- CometFilter (131) + +- CometScan parquet spark_catalog.default.date_dim (130) (unknown) Scan parquet spark_catalog.default.date_dim @@ -744,14 +732,14 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(134) CometFilter +(131) CometFilter Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(135) ColumnarToRow [codegen id : 1] +(132) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] -(136) BroadcastExchange +(133) BroadcastExchange Input [2]: [d_date_sk#13, d_year#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] @@ -760,10 +748,10 @@ Subquery:2 Hosting operator id = 24 Hosting Expression = ss_sold_date_sk#26 IN d Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 Subquery:4 Hosting operator id = 72 Hosting Expression = cs_sold_date_sk#74 IN dynamicpruning#75 -BroadcastExchange (140) -+- * ColumnarToRow (139) - +- CometFilter (138) - +- CometScan parquet spark_catalog.default.date_dim (137) +BroadcastExchange (137) ++- * ColumnarToRow (136) + +- CometFilter (135) + +- CometScan parquet spark_catalog.default.date_dim (134) (unknown) Scan parquet spark_catalog.default.date_dim @@ -773,19 +761,19 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(138) CometFilter +(135) CometFilter Input [2]: [d_date_sk#81, d_year#82] Condition : ((isnotnull(d_year#82) AND (d_year#82 = 2001)) AND isnotnull(d_date_sk#81)) -(139) ColumnarToRow [codegen id : 1] +(136) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#81, d_year#82] -(140) BroadcastExchange +(137) BroadcastExchange Input [2]: [d_date_sk#81, d_year#82] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] -Subquery:5 Hosting operator id = 88 Hosting Expression = ss_sold_date_sk#91 IN dynamicpruning#75 +Subquery:5 Hosting operator id = 87 Hosting Expression = ss_sold_date_sk#91 IN dynamicpruning#75 -Subquery:6 Hosting operator id = 104 Hosting Expression = ws_sold_date_sk#108 IN dynamicpruning#75 +Subquery:6 Hosting operator id = 102 Hosting Expression = ws_sold_date_sk#108 IN dynamicpruning#75 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt index 25dd0f946..9939b2fe2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt @@ -1,27 +1,27 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt] - WholeStageCodegen (45) + WholeStageCodegen (51) Project [d_year,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt,sales_amt,sales_amt] SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_manufact_id,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt] InputAdapter - WholeStageCodegen (22) + WholeStageCodegen (25) Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - WholeStageCodegen (21) + WholeStageCodegen (24) Filter [sales_cnt] HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(sales_cnt),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] InputAdapter Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - WholeStageCodegen (20) + WholeStageCodegen (23) HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 - WholeStageCodegen (19) + WholeStageCodegen (22) HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Union - WholeStageCodegen (6) + WholeStageCodegen (7) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] InputAdapter @@ -56,23 +56,25 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i InputAdapter ReusedExchange [d_date_sk,d_year] #5 InputAdapter - WholeStageCodegen (5) - ColumnarToRow + WholeStageCodegen (6) + Sort [cr_order_number,cr_item_sk] InputAdapter - CometSort [cr_order_number,cr_item_sk] - CometExchange [cr_order_number,cr_item_sk] #7 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_order_number,cr_item_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - WholeStageCodegen (12) + Exchange [cr_order_number,cr_item_sk] #7 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_order_number,cr_item_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + WholeStageCodegen (14) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] InputAdapter - WholeStageCodegen (10) + WholeStageCodegen (11) Sort [ss_ticket_number,ss_item_sk] InputAdapter Exchange [ss_ticket_number,ss_item_sk] #8 - WholeStageCodegen (9) + WholeStageCodegen (10) Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] @@ -87,23 +89,25 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i InputAdapter ReusedExchange [d_date_sk,d_year] #5 InputAdapter - WholeStageCodegen (11) - ColumnarToRow + WholeStageCodegen (13) + Sort [sr_ticket_number,sr_item_sk] InputAdapter - CometSort [sr_ticket_number,sr_item_sk] - CometExchange [sr_ticket_number,sr_item_sk] #9 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_ticket_number,sr_item_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - WholeStageCodegen (18) + Exchange [sr_ticket_number,sr_item_sk] #9 + WholeStageCodegen (12) + ColumnarToRow + InputAdapter + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_ticket_number,sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + WholeStageCodegen (21) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] InputAdapter - WholeStageCodegen (16) + WholeStageCodegen (18) Sort [ws_order_number,ws_item_sk] InputAdapter Exchange [ws_order_number,ws_item_sk] #10 - WholeStageCodegen (15) + WholeStageCodegen (17) Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] @@ -118,42 +122,44 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i InputAdapter ReusedExchange [d_date_sk,d_year] #5 InputAdapter - WholeStageCodegen (17) - ColumnarToRow + WholeStageCodegen (20) + Sort [wr_order_number,wr_item_sk] InputAdapter - CometSort [wr_order_number,wr_item_sk] - CometExchange [wr_order_number,wr_item_sk] #11 - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_order_number,wr_item_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + Exchange [wr_order_number,wr_item_sk] #11 + WholeStageCodegen (19) + ColumnarToRow + InputAdapter + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_order_number,wr_item_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] InputAdapter - WholeStageCodegen (44) + WholeStageCodegen (50) Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 - WholeStageCodegen (43) + WholeStageCodegen (49) Filter [sales_cnt] HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(sales_cnt),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] InputAdapter Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 - WholeStageCodegen (42) + WholeStageCodegen (48) HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #14 - WholeStageCodegen (41) + WholeStageCodegen (47) HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Union - WholeStageCodegen (28) + WholeStageCodegen (32) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] InputAdapter - WholeStageCodegen (26) + WholeStageCodegen (29) Sort [cs_order_number,cs_item_sk] InputAdapter Exchange [cs_order_number,cs_item_sk] #15 - WholeStageCodegen (25) + WholeStageCodegen (28) Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] @@ -174,20 +180,19 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i InputAdapter ReusedExchange [d_date_sk,d_year] #16 InputAdapter - WholeStageCodegen (27) - ColumnarToRow + WholeStageCodegen (31) + Sort [cr_order_number,cr_item_sk] InputAdapter - CometSort [cr_order_number,cr_item_sk] - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 - WholeStageCodegen (34) + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 + WholeStageCodegen (39) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] InputAdapter - WholeStageCodegen (32) + WholeStageCodegen (36) Sort [ss_ticket_number,ss_item_sk] InputAdapter Exchange [ss_ticket_number,ss_item_sk] #17 - WholeStageCodegen (31) + WholeStageCodegen (35) Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] @@ -202,20 +207,19 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i InputAdapter ReusedExchange [d_date_sk,d_year] #16 InputAdapter - WholeStageCodegen (33) - ColumnarToRow + WholeStageCodegen (38) + Sort [sr_ticket_number,sr_item_sk] InputAdapter - CometSort [sr_ticket_number,sr_item_sk] - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 - WholeStageCodegen (40) + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 + WholeStageCodegen (46) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] InputAdapter - WholeStageCodegen (38) + WholeStageCodegen (43) Sort [ws_order_number,ws_item_sk] InputAdapter Exchange [ws_order_number,ws_item_sk] #18 - WholeStageCodegen (37) + WholeStageCodegen (42) Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] @@ -230,8 +234,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i InputAdapter ReusedExchange [d_date_sk,d_year] #16 InputAdapter - WholeStageCodegen (39) - ColumnarToRow + WholeStageCodegen (45) + Sort [wr_order_number,wr_item_sk] InputAdapter - CometSort [wr_order_number,wr_item_sk] - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 + ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt index cfb9ed7d0..da7812b31 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt @@ -1,71 +1,74 @@ == Physical Plan == -TakeOrderedAndProject (67) -+- * Project (66) - +- * SortMergeJoin Inner (65) - :- * Project (43) - : +- * SortMergeJoin Inner (42) - : :- * Sort (20) - : : +- * HashAggregate (19) - : : +- Exchange (18) - : : +- * HashAggregate (17) - : : +- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * ColumnarToRow (13) - : : : +- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometSortMergeJoin (10) - : : : :- CometSort (4) - : : : : +- CometExchange (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- CometSort (9) - : : : +- CometExchange (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan parquet spark_catalog.default.store_returns (5) - : : +- ReusedExchange (14) - : +- * Sort (41) - : +- * Filter (40) - : +- * HashAggregate (39) - : +- Exchange (38) - : +- * HashAggregate (37) - : +- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * ColumnarToRow (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometSortMergeJoin (30) - : : :- CometSort (24) - : : : +- CometExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan parquet spark_catalog.default.web_sales (21) - : : +- CometSort (29) - : : +- CometExchange (28) - : : +- CometProject (27) - : : +- CometFilter (26) - : : +- CometScan parquet spark_catalog.default.web_returns (25) - : +- ReusedExchange (34) - +- * Sort (64) - +- * Filter (63) - +- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- * Project (59) - +- * BroadcastHashJoin Inner BuildRight (58) - :- * ColumnarToRow (56) - : +- CometProject (55) - : +- CometFilter (54) - : +- CometSortMergeJoin (53) - : :- CometSort (47) - : : +- CometExchange (46) - : : +- CometFilter (45) - : : +- CometScan parquet spark_catalog.default.catalog_sales (44) - : +- CometSort (52) - : +- CometExchange (51) - : +- CometProject (50) - : +- CometFilter (49) - : +- CometScan parquet spark_catalog.default.catalog_returns (48) - +- ReusedExchange (57) +TakeOrderedAndProject (70) ++- * Project (69) + +- * SortMergeJoin Inner (68) + :- * Project (45) + : +- * SortMergeJoin Inner (44) + : :- * Sort (21) + : : +- * HashAggregate (20) + : : +- Exchange (19) + : : +- * HashAggregate (18) + : : +- * Project (17) + : : +- * BroadcastHashJoin Inner BuildRight (16) + : : :- * Project (14) + : : : +- * Filter (13) + : : : +- * SortMergeJoin LeftOuter (12) + : : : :- * Sort (5) + : : : : +- Exchange (4) + : : : : +- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- * Sort (11) + : : : +- Exchange (10) + : : : +- * ColumnarToRow (9) + : : : +- CometProject (8) + : : : +- CometFilter (7) + : : : +- CometScan parquet spark_catalog.default.store_returns (6) + : : +- ReusedExchange (15) + : +- * Sort (43) + : +- * Filter (42) + : +- * HashAggregate (41) + : +- Exchange (40) + : +- * HashAggregate (39) + : +- * Project (38) + : +- * BroadcastHashJoin Inner BuildRight (37) + : :- * Project (35) + : : +- * Filter (34) + : : +- * SortMergeJoin LeftOuter (33) + : : :- * Sort (26) + : : : +- Exchange (25) + : : : +- * ColumnarToRow (24) + : : : +- CometFilter (23) + : : : +- CometScan parquet spark_catalog.default.web_sales (22) + : : +- * Sort (32) + : : +- Exchange (31) + : : +- * ColumnarToRow (30) + : : +- CometProject (29) + : : +- CometFilter (28) + : : +- CometScan parquet spark_catalog.default.web_returns (27) + : +- ReusedExchange (36) + +- * Sort (67) + +- * Filter (66) + +- * HashAggregate (65) + +- Exchange (64) + +- * HashAggregate (63) + +- * Project (62) + +- * BroadcastHashJoin Inner BuildRight (61) + :- * Project (59) + : +- * Filter (58) + : +- * SortMergeJoin LeftOuter (57) + : :- * Sort (50) + : : +- Exchange (49) + : : +- * ColumnarToRow (48) + : : +- CometFilter (47) + : : +- CometScan parquet spark_catalog.default.catalog_sales (46) + : +- * Sort (56) + : +- Exchange (55) + : +- * ColumnarToRow (54) + : +- CometProject (53) + : +- CometFilter (52) + : +- CometScan parquet spark_catalog.default.catalog_returns (51) + +- ReusedExchange (60) (unknown) Scan parquet spark_catalog.default.store_sales @@ -80,13 +83,16 @@ ReadSchema: struct -(6) CometFilter +(7) CometFilter Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] Condition : (isnotnull(sr_ticket_number#10) AND isnotnull(sr_item_sk#9)) -(7) CometProject +(8) CometProject Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_item_sk#9, sr_ticket_number#10] -(8) CometExchange +(9) ColumnarToRow [codegen id : 3] Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(9) CometSort +(10) Exchange Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST] +Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(10) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_ticket_number#3, ss_item_sk#1], [sr_ticket_number#10, sr_item_sk#9], LeftOuter +(11) Sort [codegen id : 4] +Input [2]: [sr_item_sk#9, sr_ticket_number#10] +Arguments: [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST], false, 0 + +(12) SortMergeJoin [codegen id : 6] +Left keys [2]: [ss_ticket_number#3, ss_item_sk#1] +Right keys [2]: [sr_ticket_number#10, sr_item_sk#9] +Join type: LeftOuter +Join condition: None -(11) CometFilter +(13) Filter [codegen id : 6] Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] Condition : isnull(sr_ticket_number#10) -(12) CometProject +(14) Project [codegen id : 6] +Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -(13) ColumnarToRow [codegen id : 2] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] - -(14) ReusedExchange [Reuses operator id: 71] +(15) ReusedExchange [Reuses operator id: 74] Output [2]: [d_date_sk#12, d_year#13] -(15) BroadcastHashJoin [codegen id : 2] +(16) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#12] Join type: Inner Join condition: None -(16) Project [codegen id : 2] +(17) Project [codegen id : 6] Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] -(17) HashAggregate [codegen id : 2] +(18) HashAggregate [codegen id : 6] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesale_cost#5)), partial_sum(UnscaledValue(ss_sales_price#6))] Aggregate Attributes [3]: [sum#14, sum#15, sum#16] Results [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] -(18) Exchange +(19) Exchange Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(19) HashAggregate [codegen id : 3] +(20) HashAggregate [codegen id : 7] Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] Aggregate Attributes [3]: [sum(ss_quantity#4)#20, sum(UnscaledValue(ss_wholesale_cost#5))#21, sum(UnscaledValue(ss_sales_price#6))#22] Results [6]: [d_year#13 AS ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, sum(ss_quantity#4)#20 AS ss_qty#24, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#5))#21,17,2) AS ss_wc#25, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#22,17,2) AS ss_sp#26] -(20) Sort [codegen id : 3] +(21) Sort [codegen id : 7] Input [6]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] Arguments: [ss_sold_year#23 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], false, 0 @@ -170,17 +177,20 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(22) CometFilter +(23) CometFilter Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] Condition : (isnotnull(ws_item_sk#27) AND isnotnull(ws_bill_customer_sk#28)) -(23) CometExchange +(24) ColumnarToRow [codegen id : 8] +Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] + +(25) Exchange Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Arguments: hashpartitioning(ws_order_number#29, ws_item_sk#27, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +Arguments: hashpartitioning(ws_order_number#29, ws_item_sk#27, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(24) CometSort +(26) Sort [codegen id : 9] Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Arguments: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33], [ws_order_number#29 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST] +Arguments: [ws_order_number#29 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.web_returns Output [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] @@ -189,84 +199,85 @@ Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct -(26) CometFilter +(28) CometFilter Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] Condition : (isnotnull(wr_order_number#36) AND isnotnull(wr_item_sk#35)) -(27) CometProject +(29) CometProject Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] Arguments: [wr_item_sk#35, wr_order_number#36], [wr_item_sk#35, wr_order_number#36] -(28) CometExchange +(30) ColumnarToRow [codegen id : 10] +Input [2]: [wr_item_sk#35, wr_order_number#36] + +(31) Exchange Input [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: hashpartitioning(wr_order_number#36, wr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +Arguments: hashpartitioning(wr_order_number#36, wr_item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(29) CometSort +(32) Sort [codegen id : 11] Input [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: [wr_item_sk#35, wr_order_number#36], [wr_order_number#36 ASC NULLS FIRST, wr_item_sk#35 ASC NULLS FIRST] +Arguments: [wr_order_number#36 ASC NULLS FIRST, wr_item_sk#35 ASC NULLS FIRST], false, 0 -(30) CometSortMergeJoin -Left output [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Right output [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: [ws_order_number#29, ws_item_sk#27], [wr_order_number#36, wr_item_sk#35], LeftOuter +(33) SortMergeJoin [codegen id : 13] +Left keys [2]: [ws_order_number#29, ws_item_sk#27] +Right keys [2]: [wr_order_number#36, wr_item_sk#35] +Join type: LeftOuter +Join condition: None -(31) CometFilter +(34) Filter [codegen id : 13] Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] Condition : isnull(wr_order_number#36) -(32) CometProject +(35) Project [codegen id : 13] +Output [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] -Arguments: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33], [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] - -(33) ColumnarToRow [codegen id : 5] -Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -(34) ReusedExchange [Reuses operator id: 71] +(36) ReusedExchange [Reuses operator id: 74] Output [2]: [d_date_sk#38, d_year#39] -(35) BroadcastHashJoin [codegen id : 5] +(37) BroadcastHashJoin [codegen id : 13] Left keys [1]: [ws_sold_date_sk#33] Right keys [1]: [d_date_sk#38] Join type: Inner Join condition: None -(36) Project [codegen id : 5] +(38) Project [codegen id : 13] Output [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] Input [8]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, d_date_sk#38, d_year#39] -(37) HashAggregate [codegen id : 5] +(39) HashAggregate [codegen id : 13] Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] Functions [3]: [partial_sum(ws_quantity#30), partial_sum(UnscaledValue(ws_wholesale_cost#31)), partial_sum(UnscaledValue(ws_sales_price#32))] Aggregate Attributes [3]: [sum#40, sum#41, sum#42] Results [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] -(38) Exchange +(40) Exchange Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] Arguments: hashpartitioning(d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(39) HashAggregate [codegen id : 6] +(41) HashAggregate [codegen id : 14] Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] Functions [3]: [sum(ws_quantity#30), sum(UnscaledValue(ws_wholesale_cost#31)), sum(UnscaledValue(ws_sales_price#32))] Aggregate Attributes [3]: [sum(ws_quantity#30)#46, sum(UnscaledValue(ws_wholesale_cost#31))#47, sum(UnscaledValue(ws_sales_price#32))#48] Results [6]: [d_year#39 AS ws_sold_year#49, ws_item_sk#27, ws_bill_customer_sk#28 AS ws_customer_sk#50, sum(ws_quantity#30)#46 AS ws_qty#51, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#31))#47,17,2) AS ws_wc#52, MakeDecimal(sum(UnscaledValue(ws_sales_price#32))#48,17,2) AS ws_sp#53] -(40) Filter [codegen id : 6] +(42) Filter [codegen id : 14] Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] Condition : (coalesce(ws_qty#51, 0) > 0) -(41) Sort [codegen id : 6] +(43) Sort [codegen id : 14] Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] Arguments: [ws_sold_year#49 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST, ws_customer_sk#50 ASC NULLS FIRST], false, 0 -(42) SortMergeJoin [codegen id : 7] +(44) SortMergeJoin [codegen id : 15] Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] Right keys [3]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50] Join type: Inner Join condition: None -(43) Project [codegen id : 7] +(45) Project [codegen id : 15] Output [9]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53] Input [12]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] @@ -278,17 +289,20 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(45) CometFilter +(47) CometFilter Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] Condition : (isnotnull(cs_item_sk#55) AND isnotnull(cs_bill_customer_sk#54)) -(46) CometExchange +(48) ColumnarToRow [codegen id : 16] +Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] + +(49) Exchange Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Arguments: hashpartitioning(cs_order_number#56, cs_item_sk#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +Arguments: hashpartitioning(cs_order_number#56, cs_item_sk#55, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(47) CometSort +(50) Sort [codegen id : 17] Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Arguments: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60], [cs_order_number#56 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST] +Arguments: [cs_order_number#56 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.catalog_returns Output [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] @@ -297,98 +311,99 @@ Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(49) CometFilter +(52) CometFilter Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] Condition : (isnotnull(cr_order_number#63) AND isnotnull(cr_item_sk#62)) -(50) CometProject +(53) CometProject Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] Arguments: [cr_item_sk#62, cr_order_number#63], [cr_item_sk#62, cr_order_number#63] -(51) CometExchange +(54) ColumnarToRow [codegen id : 18] Input [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: hashpartitioning(cr_order_number#63, cr_item_sk#62, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] -(52) CometSort +(55) Exchange Input [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: [cr_item_sk#62, cr_order_number#63], [cr_order_number#63 ASC NULLS FIRST, cr_item_sk#62 ASC NULLS FIRST] +Arguments: hashpartitioning(cr_order_number#63, cr_item_sk#62, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(53) CometSortMergeJoin -Left output [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Right output [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: [cs_order_number#56, cs_item_sk#55], [cr_order_number#63, cr_item_sk#62], LeftOuter +(56) Sort [codegen id : 19] +Input [2]: [cr_item_sk#62, cr_order_number#63] +Arguments: [cr_order_number#63 ASC NULLS FIRST, cr_item_sk#62 ASC NULLS FIRST], false, 0 + +(57) SortMergeJoin [codegen id : 21] +Left keys [2]: [cs_order_number#56, cs_item_sk#55] +Right keys [2]: [cr_order_number#63, cr_item_sk#62] +Join type: LeftOuter +Join condition: None -(54) CometFilter +(58) Filter [codegen id : 21] Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] Condition : isnull(cr_order_number#63) -(55) CometProject +(59) Project [codegen id : 21] +Output [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] -Arguments: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60], [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] - -(56) ColumnarToRow [codegen id : 9] -Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -(57) ReusedExchange [Reuses operator id: 71] +(60) ReusedExchange [Reuses operator id: 74] Output [2]: [d_date_sk#65, d_year#66] -(58) BroadcastHashJoin [codegen id : 9] +(61) BroadcastHashJoin [codegen id : 21] Left keys [1]: [cs_sold_date_sk#60] Right keys [1]: [d_date_sk#65] Join type: Inner Join condition: None -(59) Project [codegen id : 9] +(62) Project [codegen id : 21] Output [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] Input [8]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, d_date_sk#65, d_year#66] -(60) HashAggregate [codegen id : 9] +(63) HashAggregate [codegen id : 21] Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] Functions [3]: [partial_sum(cs_quantity#57), partial_sum(UnscaledValue(cs_wholesale_cost#58)), partial_sum(UnscaledValue(cs_sales_price#59))] Aggregate Attributes [3]: [sum#67, sum#68, sum#69] Results [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] -(61) Exchange +(64) Exchange Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] Arguments: hashpartitioning(d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(62) HashAggregate [codegen id : 10] +(65) HashAggregate [codegen id : 22] Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] Functions [3]: [sum(cs_quantity#57), sum(UnscaledValue(cs_wholesale_cost#58)), sum(UnscaledValue(cs_sales_price#59))] Aggregate Attributes [3]: [sum(cs_quantity#57)#73, sum(UnscaledValue(cs_wholesale_cost#58))#74, sum(UnscaledValue(cs_sales_price#59))#75] Results [6]: [d_year#66 AS cs_sold_year#76, cs_item_sk#55, cs_bill_customer_sk#54 AS cs_customer_sk#77, sum(cs_quantity#57)#73 AS cs_qty#78, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#58))#74,17,2) AS cs_wc#79, MakeDecimal(sum(UnscaledValue(cs_sales_price#59))#75,17,2) AS cs_sp#80] -(63) Filter [codegen id : 10] +(66) Filter [codegen id : 22] Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] Condition : (coalesce(cs_qty#78, 0) > 0) -(64) Sort [codegen id : 10] +(67) Sort [codegen id : 22] Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] Arguments: [cs_sold_year#76 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST, cs_customer_sk#77 ASC NULLS FIRST], false, 0 -(65) SortMergeJoin [codegen id : 11] +(68) SortMergeJoin [codegen id : 23] Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] Right keys [3]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77] Join type: Inner Join condition: None -(66) Project [codegen id : 11] +(69) Project [codegen id : 23] Output [13]: [round((cast(ss_qty#24 as double) / cast(coalesce((ws_qty#51 + cs_qty#78), 1) as double)), 2) AS ratio#81, ss_qty#24 AS store_qty#82, ss_wc#25 AS store_wholesale_cost#83, ss_sp#26 AS store_sales_price#84, (coalesce(ws_qty#51, 0) + coalesce(cs_qty#78, 0)) AS other_chan_qty#85, (coalesce(ws_wc#52, 0.00) + coalesce(cs_wc#79, 0.00)) AS other_chan_wholesale_cost#86, (coalesce(ws_sp#53, 0.00) + coalesce(cs_sp#80, 0.00)) AS other_chan_sales_price#87, ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] Input [15]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53, cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] -(67) TakeOrderedAndProject +(70) TakeOrderedAndProject Input [13]: [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87, ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] Arguments: 100, [ss_sold_year#23 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#24 DESC NULLS LAST, ss_wc#25 DESC NULLS LAST, ss_sp#26 DESC NULLS LAST, other_chan_qty#85 ASC NULLS FIRST, other_chan_wholesale_cost#86 ASC NULLS FIRST, other_chan_sales_price#87 ASC NULLS FIRST, ratio#81 ASC NULLS FIRST], [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (71) -+- * ColumnarToRow (70) - +- CometFilter (69) - +- CometScan parquet spark_catalog.default.date_dim (68) +BroadcastExchange (74) ++- * ColumnarToRow (73) + +- CometFilter (72) + +- CometScan parquet spark_catalog.default.date_dim (71) (unknown) Scan parquet spark_catalog.default.date_dim @@ -398,19 +413,19 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(69) CometFilter +(72) CometFilter Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(70) ColumnarToRow [codegen id : 1] +(73) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] -(71) BroadcastExchange +(74) BroadcastExchange Input [2]: [d_date_sk#12, d_year#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -Subquery:2 Hosting operator id = 21 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt index ea1de2fd7..49bd173f6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt @@ -1,97 +1,127 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ratio,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (11) + WholeStageCodegen (23) Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp,ss_sold_year,ss_item_sk,ss_customer_sk] SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,cs_sold_year,cs_item_sk,cs_customer_sk] InputAdapter - WholeStageCodegen (7) + WholeStageCodegen (15) Project [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ws_sold_year,ws_item_sk,ws_customer_sk] InputAdapter - WholeStageCodegen (3) + WholeStageCodegen (7) Sort [ss_sold_year,ss_item_sk,ss_customer_sk] HashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] [sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price)),ss_sold_year,ss_qty,ss_wc,ss_sp,sum,sum,sum] InputAdapter Exchange [d_year,ss_item_sk,ss_customer_sk] #1 - WholeStageCodegen (2) + WholeStageCodegen (6) HashAggregate [d_year,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] [sum,sum,sum,sum,sum,sum] Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometFilter [sr_ticket_number] - CometSortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometSort [ss_ticket_number,ss_item_sk] - CometExchange [ss_ticket_number,ss_item_sk] #2 - CometFilter [ss_item_sk,ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [sr_ticket_number,sr_item_sk] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + Filter [sr_ticket_number] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ss_ticket_number,ss_item_sk] + InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + WholeStageCodegen (4) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + Exchange [sr_ticket_number,sr_item_sk] #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_ticket_number,sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 InputAdapter - WholeStageCodegen (6) + WholeStageCodegen (14) Sort [ws_sold_year,ws_item_sk,ws_customer_sk] Filter [ws_qty] HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] [sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price)),ws_sold_year,ws_customer_sk,ws_qty,ws_wc,ws_sp,sum,sum,sum] InputAdapter Exchange [d_year,ws_item_sk,ws_bill_customer_sk] #5 - WholeStageCodegen (5) + WholeStageCodegen (13) HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] [sum,sum,sum,sum,sum,sum] Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometFilter [wr_order_number] - CometSortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - CometSort [ws_order_number,ws_item_sk] - CometExchange [ws_order_number,ws_item_sk] #6 - CometFilter [ws_item_sk,ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [wr_order_number,wr_item_sk] - CometExchange [wr_order_number,wr_item_sk] #7 - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_order_number,wr_item_sk] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] + Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + Filter [wr_order_number] + SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + InputAdapter + WholeStageCodegen (9) + Sort [ws_order_number,ws_item_sk] + InputAdapter + Exchange [ws_order_number,ws_item_sk] #6 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk,ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + WholeStageCodegen (11) + Sort [wr_order_number,wr_item_sk] + InputAdapter + Exchange [wr_order_number,wr_item_sk] #7 + WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometProject [wr_item_sk,wr_order_number] + CometFilter [wr_order_number,wr_item_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 InputAdapter - WholeStageCodegen (10) + WholeStageCodegen (22) Sort [cs_sold_year,cs_item_sk,cs_customer_sk] Filter [cs_qty] HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] [sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price)),cs_sold_year,cs_customer_sk,cs_qty,cs_wc,cs_sp,sum,sum,sum] InputAdapter Exchange [d_year,cs_item_sk,cs_bill_customer_sk] #8 - WholeStageCodegen (9) + WholeStageCodegen (21) HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] [sum,sum,sum,sum,sum,sum] Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometFilter [cr_order_number] - CometSortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometSort [cs_order_number,cs_item_sk] - CometExchange [cs_order_number,cs_item_sk] #9 - CometFilter [cs_item_sk,cs_bill_customer_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [cr_order_number,cr_item_sk] - CometExchange [cr_order_number,cr_item_sk] #10 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_order_number,cr_item_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + Filter [cr_order_number] + SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + InputAdapter + WholeStageCodegen (17) + Sort [cs_order_number,cs_item_sk] + InputAdapter + Exchange [cs_order_number,cs_item_sk] #9 + WholeStageCodegen (16) + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk,cs_bill_customer_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + WholeStageCodegen (19) + Sort [cr_order_number,cr_item_sk] + InputAdapter + Exchange [cr_order_number,cr_item_sk] #10 + WholeStageCodegen (18) + ColumnarToRow + InputAdapter + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_order_number,cr_item_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt index dfd081b44..0c9c97812 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt @@ -1,121 +1,124 @@ == Physical Plan == -TakeOrderedAndProject (117) -+- * HashAggregate (116) - +- Exchange (115) - +- * HashAggregate (114) - +- Union (113) - :- * HashAggregate (102) - : +- Exchange (101) - : +- * HashAggregate (100) - : +- Union (99) - : :- * HashAggregate (38) - : : +- Exchange (37) - : : +- * HashAggregate (36) - : : +- * Project (35) - : : +- * BroadcastHashJoin Inner BuildRight (34) - : : :- * Project (28) - : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : :- * Project (21) - : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : :- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * ColumnarToRow (12) - : : : : : : +- CometProject (11) - : : : : : : +- CometSortMergeJoin (10) - : : : : : : :- CometSort (4) - : : : : : : : +- CometExchange (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometSort (9) - : : : : : : +- CometExchange (8) - : : : : : : +- CometProject (7) - : : : : : : +- CometFilter (6) - : : : : : : +- CometScan parquet spark_catalog.default.store_returns (5) - : : : : : +- ReusedExchange (13) - : : : : +- BroadcastExchange (19) - : : : : +- * ColumnarToRow (18) - : : : : +- CometFilter (17) - : : : : +- CometScan parquet spark_catalog.default.store (16) - : : : +- BroadcastExchange (26) - : : : +- * ColumnarToRow (25) - : : : +- CometProject (24) - : : : +- CometFilter (23) - : : : +- CometScan parquet spark_catalog.default.item (22) - : : +- BroadcastExchange (33) - : : +- * ColumnarToRow (32) - : : +- CometProject (31) - : : +- CometFilter (30) - : : +- CometScan parquet spark_catalog.default.promotion (29) - : :- * HashAggregate (68) - : : +- Exchange (67) - : : +- * HashAggregate (66) - : : +- * Project (65) - : : +- * BroadcastHashJoin Inner BuildRight (64) - : : :- * Project (62) - : : : +- * BroadcastHashJoin Inner BuildRight (61) - : : : :- * Project (59) - : : : : +- * BroadcastHashJoin Inner BuildRight (58) - : : : : :- * Project (53) - : : : : : +- * BroadcastHashJoin Inner BuildRight (52) - : : : : : :- * ColumnarToRow (50) - : : : : : : +- CometProject (49) - : : : : : : +- CometSortMergeJoin (48) - : : : : : : :- CometSort (42) - : : : : : : : +- CometExchange (41) - : : : : : : : +- CometFilter (40) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (39) - : : : : : : +- CometSort (47) - : : : : : : +- CometExchange (46) - : : : : : : +- CometProject (45) - : : : : : : +- CometFilter (44) - : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (43) - : : : : : +- ReusedExchange (51) - : : : : +- BroadcastExchange (57) - : : : : +- * ColumnarToRow (56) - : : : : +- CometFilter (55) - : : : : +- CometScan parquet spark_catalog.default.catalog_page (54) - : : : +- ReusedExchange (60) - : : +- ReusedExchange (63) - : +- * HashAggregate (98) - : +- Exchange (97) - : +- * HashAggregate (96) - : +- * Project (95) - : +- * BroadcastHashJoin Inner BuildRight (94) - : :- * Project (92) - : : +- * BroadcastHashJoin Inner BuildRight (91) - : : :- * Project (89) - : : : +- * BroadcastHashJoin Inner BuildRight (88) - : : : :- * Project (83) - : : : : +- * BroadcastHashJoin Inner BuildRight (82) - : : : : :- * ColumnarToRow (80) - : : : : : +- CometProject (79) - : : : : : +- CometSortMergeJoin (78) - : : : : : :- CometSort (72) - : : : : : : +- CometExchange (71) - : : : : : : +- CometFilter (70) - : : : : : : +- CometScan parquet spark_catalog.default.web_sales (69) - : : : : : +- CometSort (77) - : : : : : +- CometExchange (76) - : : : : : +- CometProject (75) - : : : : : +- CometFilter (74) - : : : : : +- CometScan parquet spark_catalog.default.web_returns (73) - : : : : +- ReusedExchange (81) - : : : +- BroadcastExchange (87) - : : : +- * ColumnarToRow (86) - : : : +- CometFilter (85) - : : : +- CometScan parquet spark_catalog.default.web_site (84) - : : +- ReusedExchange (90) - : +- ReusedExchange (93) - :- * HashAggregate (107) - : +- Exchange (106) - : +- * HashAggregate (105) - : +- * HashAggregate (104) - : +- ReusedExchange (103) - +- * HashAggregate (112) - +- Exchange (111) - +- * HashAggregate (110) - +- * HashAggregate (109) - +- ReusedExchange (108) +TakeOrderedAndProject (120) ++- * HashAggregate (119) + +- Exchange (118) + +- * HashAggregate (117) + +- Union (116) + :- * HashAggregate (105) + : +- Exchange (104) + : +- * HashAggregate (103) + : +- Union (102) + : :- * HashAggregate (39) + : : +- Exchange (38) + : : +- * HashAggregate (37) + : : +- * Project (36) + : : +- * BroadcastHashJoin Inner BuildRight (35) + : : :- * Project (29) + : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : :- * Project (22) + : : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : : :- * Project (16) + : : : : : +- * BroadcastHashJoin Inner BuildRight (15) + : : : : : :- * Project (13) + : : : : : : +- * SortMergeJoin LeftOuter (12) + : : : : : : :- * Sort (5) + : : : : : : : +- Exchange (4) + : : : : : : : +- * ColumnarToRow (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- * Sort (11) + : : : : : : +- Exchange (10) + : : : : : : +- * ColumnarToRow (9) + : : : : : : +- CometProject (8) + : : : : : : +- CometFilter (7) + : : : : : : +- CometScan parquet spark_catalog.default.store_returns (6) + : : : : : +- ReusedExchange (14) + : : : : +- BroadcastExchange (20) + : : : : +- * ColumnarToRow (19) + : : : : +- CometFilter (18) + : : : : +- CometScan parquet spark_catalog.default.store (17) + : : : +- BroadcastExchange (27) + : : : +- * ColumnarToRow (26) + : : : +- CometProject (25) + : : : +- CometFilter (24) + : : : +- CometScan parquet spark_catalog.default.item (23) + : : +- BroadcastExchange (34) + : : +- * ColumnarToRow (33) + : : +- CometProject (32) + : : +- CometFilter (31) + : : +- CometScan parquet spark_catalog.default.promotion (30) + : :- * HashAggregate (70) + : : +- Exchange (69) + : : +- * HashAggregate (68) + : : +- * Project (67) + : : +- * BroadcastHashJoin Inner BuildRight (66) + : : :- * Project (64) + : : : +- * BroadcastHashJoin Inner BuildRight (63) + : : : :- * Project (61) + : : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : : :- * Project (55) + : : : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : : : :- * Project (52) + : : : : : : +- * SortMergeJoin LeftOuter (51) + : : : : : : :- * Sort (44) + : : : : : : : +- Exchange (43) + : : : : : : : +- * ColumnarToRow (42) + : : : : : : : +- CometFilter (41) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (40) + : : : : : : +- * Sort (50) + : : : : : : +- Exchange (49) + : : : : : : +- * ColumnarToRow (48) + : : : : : : +- CometProject (47) + : : : : : : +- CometFilter (46) + : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (45) + : : : : : +- ReusedExchange (53) + : : : : +- BroadcastExchange (59) + : : : : +- * ColumnarToRow (58) + : : : : +- CometFilter (57) + : : : : +- CometScan parquet spark_catalog.default.catalog_page (56) + : : : +- ReusedExchange (62) + : : +- ReusedExchange (65) + : +- * HashAggregate (101) + : +- Exchange (100) + : +- * HashAggregate (99) + : +- * Project (98) + : +- * BroadcastHashJoin Inner BuildRight (97) + : :- * Project (95) + : : +- * BroadcastHashJoin Inner BuildRight (94) + : : :- * Project (92) + : : : +- * BroadcastHashJoin Inner BuildRight (91) + : : : :- * Project (86) + : : : : +- * BroadcastHashJoin Inner BuildRight (85) + : : : : :- * Project (83) + : : : : : +- * SortMergeJoin LeftOuter (82) + : : : : : :- * Sort (75) + : : : : : : +- Exchange (74) + : : : : : : +- * ColumnarToRow (73) + : : : : : : +- CometFilter (72) + : : : : : : +- CometScan parquet spark_catalog.default.web_sales (71) + : : : : : +- * Sort (81) + : : : : : +- Exchange (80) + : : : : : +- * ColumnarToRow (79) + : : : : : +- CometProject (78) + : : : : : +- CometFilter (77) + : : : : : +- CometScan parquet spark_catalog.default.web_returns (76) + : : : : +- ReusedExchange (84) + : : : +- BroadcastExchange (90) + : : : +- * ColumnarToRow (89) + : : : +- CometFilter (88) + : : : +- CometScan parquet spark_catalog.default.web_site (87) + : : +- ReusedExchange (93) + : +- ReusedExchange (96) + :- * HashAggregate (110) + : +- Exchange (109) + : +- * HashAggregate (108) + : +- * HashAggregate (107) + : +- ReusedExchange (106) + +- * HashAggregate (115) + +- Exchange (114) + +- * HashAggregate (113) + +- * HashAggregate (112) + +- ReusedExchange (111) (unknown) Scan parquet spark_catalog.default.store_sales @@ -130,13 +133,16 @@ ReadSchema: struct -(6) CometFilter +(7) CometFilter Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] Condition : (isnotnull(sr_item_sk#9) AND isnotnull(sr_ticket_number#10)) -(7) CometProject +(8) CometProject Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -(8) CometExchange +(9) ColumnarToRow [codegen id : 3] Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(9) CometSort +(10) Exchange Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST] +Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(10) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Right output [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_ticket_number#4], [sr_item_sk#9, sr_ticket_number#10], LeftOuter +(11) Sort [codegen id : 4] +Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] +Arguments: [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST], false, 0 -(11) CometProject -Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] +(12) SortMergeJoin [codegen id : 9] +Left keys [2]: [ss_item_sk#1, ss_ticket_number#4] +Right keys [2]: [sr_item_sk#9, sr_ticket_number#10] +Join type: LeftOuter +Join condition: None -(12) ColumnarToRow [codegen id : 5] -Input [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] +(13) Project [codegen id : 9] +Output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] +Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -(13) ReusedExchange [Reuses operator id: 122] +(14) ReusedExchange [Reuses operator id: 125] Output [1]: [d_date_sk#14] -(14) BroadcastHashJoin [codegen id : 5] +(15) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(15) Project [codegen id : 5] +(16) Project [codegen id : 9] Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#14] @@ -193,24 +200,24 @@ Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(17) CometFilter +(18) CometFilter Input [2]: [s_store_sk#15, s_store_id#16] Condition : isnotnull(s_store_sk#15) -(18) ColumnarToRow [codegen id : 2] +(19) ColumnarToRow [codegen id : 6] Input [2]: [s_store_sk#15, s_store_id#16] -(19) BroadcastExchange +(20) BroadcastExchange Input [2]: [s_store_sk#15, s_store_id#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(20) BroadcastHashJoin [codegen id : 5] +(21) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#2] Right keys [1]: [s_store_sk#15] Join type: Inner Join condition: None -(21) Project [codegen id : 5] +(22) Project [codegen id : 9] Output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#15, s_store_id#16] @@ -221,28 +228,28 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), GreaterThan(i_current_price,50.00), IsNotNull(i_item_sk)] ReadSchema: struct -(23) CometFilter +(24) CometFilter Input [2]: [i_item_sk#17, i_current_price#18] Condition : ((isnotnull(i_current_price#18) AND (i_current_price#18 > 50.00)) AND isnotnull(i_item_sk#17)) -(24) CometProject +(25) CometProject Input [2]: [i_item_sk#17, i_current_price#18] Arguments: [i_item_sk#17], [i_item_sk#17] -(25) ColumnarToRow [codegen id : 3] +(26) ColumnarToRow [codegen id : 7] Input [1]: [i_item_sk#17] -(26) BroadcastExchange +(27) BroadcastExchange Input [1]: [i_item_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(27) BroadcastHashJoin [codegen id : 5] +(28) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#17] Join type: Inner Join condition: None -(28) Project [codegen id : 5] +(29) Project [codegen id : 9] Output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16, i_item_sk#17] @@ -253,43 +260,43 @@ Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_channel_tv), EqualTo(p_channel_tv,N), IsNotNull(p_promo_sk)] ReadSchema: struct -(30) CometFilter +(31) CometFilter Input [2]: [p_promo_sk#19, p_channel_tv#20] Condition : ((isnotnull(p_channel_tv#20) AND (p_channel_tv#20 = N)) AND isnotnull(p_promo_sk#19)) -(31) CometProject +(32) CometProject Input [2]: [p_promo_sk#19, p_channel_tv#20] Arguments: [p_promo_sk#19], [p_promo_sk#19] -(32) ColumnarToRow [codegen id : 4] +(33) ColumnarToRow [codegen id : 8] Input [1]: [p_promo_sk#19] -(33) BroadcastExchange +(34) BroadcastExchange Input [1]: [p_promo_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(34) BroadcastHashJoin [codegen id : 5] +(35) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_promo_sk#3] Right keys [1]: [p_promo_sk#19] Join type: Inner Join condition: None -(35) Project [codegen id : 5] +(36) Project [codegen id : 9] Output [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16, p_promo_sk#19] -(36) HashAggregate [codegen id : 5] +(37) HashAggregate [codegen id : 9] Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] Keys [1]: [s_store_id#16] Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] Aggregate Attributes [5]: [sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] Results [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] -(37) Exchange +(38) Exchange Input [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] Arguments: hashpartitioning(s_store_id#16, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(38) HashAggregate [codegen id : 6] +(39) HashAggregate [codegen id : 10] Input [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] Keys [1]: [s_store_id#16] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] @@ -304,17 +311,20 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#45), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] ReadSchema: struct -(40) CometFilter +(41) CometFilter Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] Condition : ((isnotnull(cs_catalog_page_sk#39) AND isnotnull(cs_item_sk#40)) AND isnotnull(cs_promo_sk#41)) -(41) CometExchange +(42) ColumnarToRow [codegen id : 11] +Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] + +(43) Exchange Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Arguments: hashpartitioning(cs_item_sk#40, cs_order_number#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +Arguments: hashpartitioning(cs_item_sk#40, cs_order_number#42, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(42) CometSort +(44) Sort [codegen id : 12] Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Arguments: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45], [cs_item_sk#40 ASC NULLS FIRST, cs_order_number#42 ASC NULLS FIRST] +Arguments: [cs_item_sk#40 ASC NULLS FIRST, cs_order_number#42 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.catalog_returns Output [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] @@ -323,44 +333,45 @@ Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(44) CometFilter +(46) CometFilter Input [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] Condition : (isnotnull(cr_item_sk#47) AND isnotnull(cr_order_number#48)) -(45) CometProject +(47) CometProject Input [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] Arguments: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50], [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -(46) CometExchange +(48) ColumnarToRow [codegen id : 13] +Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] + +(49) Exchange Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -Arguments: hashpartitioning(cr_item_sk#47, cr_order_number#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +Arguments: hashpartitioning(cr_item_sk#47, cr_order_number#48, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(47) CometSort +(50) Sort [codegen id : 14] Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -Arguments: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50], [cr_item_sk#47 ASC NULLS FIRST, cr_order_number#48 ASC NULLS FIRST] +Arguments: [cr_item_sk#47 ASC NULLS FIRST, cr_order_number#48 ASC NULLS FIRST], false, 0 -(48) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Right output [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -Arguments: [cs_item_sk#40, cs_order_number#42], [cr_item_sk#47, cr_order_number#48], LeftOuter +(51) SortMergeJoin [codegen id : 19] +Left keys [2]: [cs_item_sk#40, cs_order_number#42] +Right keys [2]: [cr_item_sk#47, cr_order_number#48] +Join type: LeftOuter +Join condition: None -(49) CometProject +(52) Project [codegen id : 19] +Output [8]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50] Input [11]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] -Arguments: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50], [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50] - -(50) ColumnarToRow [codegen id : 11] -Input [8]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50] -(51) ReusedExchange [Reuses operator id: 122] +(53) ReusedExchange [Reuses operator id: 125] Output [1]: [d_date_sk#52] -(52) BroadcastHashJoin [codegen id : 11] +(54) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_sold_date_sk#45] Right keys [1]: [d_date_sk#52] Join type: Inner Join condition: None -(53) Project [codegen id : 11] +(55) Project [codegen id : 19] Output [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50] Input [9]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50, d_date_sk#52] @@ -371,65 +382,65 @@ Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(55) CometFilter +(57) CometFilter Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] Condition : isnotnull(cp_catalog_page_sk#53) -(56) ColumnarToRow [codegen id : 8] +(58) ColumnarToRow [codegen id : 16] Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -(57) BroadcastExchange +(59) BroadcastExchange Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(58) BroadcastHashJoin [codegen id : 11] +(60) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_catalog_page_sk#39] Right keys [1]: [cp_catalog_page_sk#53] Join type: Inner Join condition: None -(59) Project [codegen id : 11] +(61) Project [codegen id : 19] Output [7]: [cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] Input [9]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_sk#53, cp_catalog_page_id#54] -(60) ReusedExchange [Reuses operator id: 26] +(62) ReusedExchange [Reuses operator id: 27] Output [1]: [i_item_sk#55] -(61) BroadcastHashJoin [codegen id : 11] +(63) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_item_sk#40] Right keys [1]: [i_item_sk#55] Join type: Inner Join condition: None -(62) Project [codegen id : 11] +(64) Project [codegen id : 19] Output [6]: [cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] Input [8]: [cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54, i_item_sk#55] -(63) ReusedExchange [Reuses operator id: 33] +(65) ReusedExchange [Reuses operator id: 34] Output [1]: [p_promo_sk#56] -(64) BroadcastHashJoin [codegen id : 11] +(66) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_promo_sk#41] Right keys [1]: [p_promo_sk#56] Join type: Inner Join condition: None -(65) Project [codegen id : 11] +(67) Project [codegen id : 19] Output [5]: [cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] Input [7]: [cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54, p_promo_sk#56] -(66) HashAggregate [codegen id : 11] +(68) HashAggregate [codegen id : 19] Input [5]: [cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] Keys [1]: [cp_catalog_page_id#54] Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#43)), partial_sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))] Aggregate Attributes [5]: [sum#57, sum#58, isEmpty#59, sum#60, isEmpty#61] Results [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] -(67) Exchange +(69) Exchange Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] Arguments: hashpartitioning(cp_catalog_page_id#54, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(68) HashAggregate [codegen id : 12] +(70) HashAggregate [codegen id : 20] Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] Keys [1]: [cp_catalog_page_id#54] Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#43)), sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00)), sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))] @@ -444,17 +455,20 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct -(70) CometFilter +(72) CometFilter Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] Condition : ((isnotnull(ws_web_site_sk#76) AND isnotnull(ws_item_sk#75)) AND isnotnull(ws_promo_sk#77)) -(71) CometExchange +(73) ColumnarToRow [codegen id : 21] +Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] + +(74) Exchange Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Arguments: hashpartitioning(ws_item_sk#75, ws_order_number#78, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] +Arguments: hashpartitioning(ws_item_sk#75, ws_order_number#78, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(72) CometSort +(75) Sort [codegen id : 22] Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Arguments: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81], [ws_item_sk#75 ASC NULLS FIRST, ws_order_number#78 ASC NULLS FIRST] +Arguments: [ws_item_sk#75 ASC NULLS FIRST, ws_order_number#78 ASC NULLS FIRST], false, 0 (unknown) Scan parquet spark_catalog.default.web_returns Output [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] @@ -463,44 +477,45 @@ Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct -(74) CometFilter +(77) CometFilter Input [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] Condition : (isnotnull(wr_item_sk#83) AND isnotnull(wr_order_number#84)) -(75) CometProject +(78) CometProject Input [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] Arguments: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86], [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -(76) CometExchange +(79) ColumnarToRow [codegen id : 23] Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -Arguments: hashpartitioning(wr_item_sk#83, wr_order_number#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] -(77) CometSort +(80) Exchange Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -Arguments: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86], [wr_item_sk#83 ASC NULLS FIRST, wr_order_number#84 ASC NULLS FIRST] +Arguments: hashpartitioning(wr_item_sk#83, wr_order_number#84, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(78) CometSortMergeJoin -Left output [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Right output [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -Arguments: [ws_item_sk#75, ws_order_number#78], [wr_item_sk#83, wr_order_number#84], LeftOuter +(81) Sort [codegen id : 24] +Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] +Arguments: [wr_item_sk#83 ASC NULLS FIRST, wr_order_number#84 ASC NULLS FIRST], false, 0 -(79) CometProject -Input [11]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -Arguments: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86], [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86] +(82) SortMergeJoin [codegen id : 29] +Left keys [2]: [ws_item_sk#75, ws_order_number#78] +Right keys [2]: [wr_item_sk#83, wr_order_number#84] +Join type: LeftOuter +Join condition: None -(80) ColumnarToRow [codegen id : 17] -Input [8]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86] +(83) Project [codegen id : 29] +Output [8]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86] +Input [11]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] -(81) ReusedExchange [Reuses operator id: 122] +(84) ReusedExchange [Reuses operator id: 125] Output [1]: [d_date_sk#88] -(82) BroadcastHashJoin [codegen id : 17] +(85) BroadcastHashJoin [codegen id : 29] Left keys [1]: [ws_sold_date_sk#81] Right keys [1]: [d_date_sk#88] Join type: Inner Join condition: None -(83) Project [codegen id : 17] +(86) Project [codegen id : 29] Output [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86] Input [9]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86, d_date_sk#88] @@ -511,179 +526,179 @@ Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(85) CometFilter +(88) CometFilter Input [2]: [web_site_sk#89, web_site_id#90] Condition : isnotnull(web_site_sk#89) -(86) ColumnarToRow [codegen id : 14] +(89) ColumnarToRow [codegen id : 26] Input [2]: [web_site_sk#89, web_site_id#90] -(87) BroadcastExchange +(90) BroadcastExchange Input [2]: [web_site_sk#89, web_site_id#90] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -(88) BroadcastHashJoin [codegen id : 17] +(91) BroadcastHashJoin [codegen id : 29] Left keys [1]: [ws_web_site_sk#76] Right keys [1]: [web_site_sk#89] Join type: Inner Join condition: None -(89) Project [codegen id : 17] +(92) Project [codegen id : 29] Output [7]: [ws_item_sk#75, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] Input [9]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_sk#89, web_site_id#90] -(90) ReusedExchange [Reuses operator id: 26] +(93) ReusedExchange [Reuses operator id: 27] Output [1]: [i_item_sk#91] -(91) BroadcastHashJoin [codegen id : 17] +(94) BroadcastHashJoin [codegen id : 29] Left keys [1]: [ws_item_sk#75] Right keys [1]: [i_item_sk#91] Join type: Inner Join condition: None -(92) Project [codegen id : 17] +(95) Project [codegen id : 29] Output [6]: [ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] Input [8]: [ws_item_sk#75, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90, i_item_sk#91] -(93) ReusedExchange [Reuses operator id: 33] +(96) ReusedExchange [Reuses operator id: 34] Output [1]: [p_promo_sk#92] -(94) BroadcastHashJoin [codegen id : 17] +(97) BroadcastHashJoin [codegen id : 29] Left keys [1]: [ws_promo_sk#77] Right keys [1]: [p_promo_sk#92] Join type: Inner Join condition: None -(95) Project [codegen id : 17] +(98) Project [codegen id : 29] Output [5]: [ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] Input [7]: [ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90, p_promo_sk#92] -(96) HashAggregate [codegen id : 17] +(99) HashAggregate [codegen id : 29] Input [5]: [ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] Keys [1]: [web_site_id#90] Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#79)), partial_sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))] Aggregate Attributes [5]: [sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] Results [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -(97) Exchange +(100) Exchange Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] Arguments: hashpartitioning(web_site_id#90, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(98) HashAggregate [codegen id : 18] +(101) HashAggregate [codegen id : 30] Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] Keys [1]: [web_site_id#90] Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#79)), sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00)), sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))] Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#79))#103, sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00))#104, sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))#105] Results [5]: [web channel AS channel#106, concat(web_site, web_site_id#90) AS id#107, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#79))#103,17,2) AS sales#108, sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00))#104 AS returns#109, sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))#105 AS profit#110] -(99) Union +(102) Union -(100) HashAggregate [codegen id : 19] +(103) HashAggregate [codegen id : 31] Input [5]: [channel#34, id#35, sales#36, returns#37, profit#38] Keys [2]: [channel#34, id#35] Functions [3]: [partial_sum(sales#36), partial_sum(returns#37), partial_sum(profit#38)] Aggregate Attributes [6]: [sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116] Results [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] -(101) Exchange +(104) Exchange Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] Arguments: hashpartitioning(channel#34, id#35, 5), ENSURE_REQUIREMENTS, [plan_id=15] -(102) HashAggregate [codegen id : 20] +(105) HashAggregate [codegen id : 32] Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] Keys [2]: [channel#34, id#35] Functions [3]: [sum(sales#36), sum(returns#37), sum(profit#38)] Aggregate Attributes [3]: [sum(sales#36)#123, sum(returns#37)#124, sum(profit#38)#125] Results [5]: [channel#34, id#35, cast(sum(sales#36)#123 as decimal(37,2)) AS sales#126, cast(sum(returns#37)#124 as decimal(38,2)) AS returns#127, cast(sum(profit#38)#125 as decimal(38,2)) AS profit#128] -(103) ReusedExchange [Reuses operator id: 101] +(106) ReusedExchange [Reuses operator id: 104] Output [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] -(104) HashAggregate [codegen id : 40] +(107) HashAggregate [codegen id : 64] Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] Keys [2]: [channel#34, id#35] Functions [3]: [sum(sales#36), sum(returns#37), sum(profit#38)] Aggregate Attributes [3]: [sum(sales#36)#123, sum(returns#37)#124, sum(profit#38)#125] Results [4]: [channel#34, sum(sales#36)#123 AS sales#129, sum(returns#37)#124 AS returns#130, sum(profit#38)#125 AS profit#131] -(105) HashAggregate [codegen id : 40] +(108) HashAggregate [codegen id : 64] Input [4]: [channel#34, sales#129, returns#130, profit#131] Keys [1]: [channel#34] Functions [3]: [partial_sum(sales#129), partial_sum(returns#130), partial_sum(profit#131)] Aggregate Attributes [6]: [sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] Results [7]: [channel#34, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] -(106) Exchange +(109) Exchange Input [7]: [channel#34, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] Arguments: hashpartitioning(channel#34, 5), ENSURE_REQUIREMENTS, [plan_id=16] -(107) HashAggregate [codegen id : 41] +(110) HashAggregate [codegen id : 65] Input [7]: [channel#34, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] Keys [1]: [channel#34] Functions [3]: [sum(sales#129), sum(returns#130), sum(profit#131)] Aggregate Attributes [3]: [sum(sales#129)#144, sum(returns#130)#145, sum(profit#131)#146] Results [5]: [channel#34, null AS id#147, sum(sales#129)#144 AS sales#148, sum(returns#130)#145 AS returns#149, sum(profit#131)#146 AS profit#150] -(108) ReusedExchange [Reuses operator id: 101] +(111) ReusedExchange [Reuses operator id: 104] Output [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] -(109) HashAggregate [codegen id : 61] +(112) HashAggregate [codegen id : 97] Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] Keys [2]: [channel#34, id#35] Functions [3]: [sum(sales#36), sum(returns#37), sum(profit#38)] Aggregate Attributes [3]: [sum(sales#36)#123, sum(returns#37)#124, sum(profit#38)#125] Results [3]: [sum(sales#36)#123 AS sales#129, sum(returns#37)#124 AS returns#130, sum(profit#38)#125 AS profit#131] -(110) HashAggregate [codegen id : 61] +(113) HashAggregate [codegen id : 97] Input [3]: [sales#129, returns#130, profit#131] Keys: [] Functions [3]: [partial_sum(sales#129), partial_sum(returns#130), partial_sum(profit#131)] Aggregate Attributes [6]: [sum#151, isEmpty#152, sum#153, isEmpty#154, sum#155, isEmpty#156] Results [6]: [sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] -(111) Exchange +(114) Exchange Input [6]: [sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=17] -(112) HashAggregate [codegen id : 62] +(115) HashAggregate [codegen id : 98] Input [6]: [sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] Keys: [] Functions [3]: [sum(sales#129), sum(returns#130), sum(profit#131)] Aggregate Attributes [3]: [sum(sales#129)#163, sum(returns#130)#164, sum(profit#131)#165] Results [5]: [null AS channel#166, null AS id#167, sum(sales#129)#163 AS sales#168, sum(returns#130)#164 AS returns#169, sum(profit#131)#165 AS profit#170] -(113) Union +(116) Union -(114) HashAggregate [codegen id : 63] +(117) HashAggregate [codegen id : 99] Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] Keys [5]: [channel#34, id#35, sales#126, returns#127, profit#128] Functions: [] Aggregate Attributes: [] Results [5]: [channel#34, id#35, sales#126, returns#127, profit#128] -(115) Exchange +(118) Exchange Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] Arguments: hashpartitioning(channel#34, id#35, sales#126, returns#127, profit#128, 5), ENSURE_REQUIREMENTS, [plan_id=18] -(116) HashAggregate [codegen id : 64] +(119) HashAggregate [codegen id : 100] Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] Keys [5]: [channel#34, id#35, sales#126, returns#127, profit#128] Functions: [] Aggregate Attributes: [] Results [5]: [channel#34, id#35, sales#126, returns#127, profit#128] -(117) TakeOrderedAndProject +(120) TakeOrderedAndProject Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] Arguments: 100, [channel#34 ASC NULLS FIRST, id#35 ASC NULLS FIRST], [channel#34, id#35, sales#126, returns#127, profit#128] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (122) -+- * ColumnarToRow (121) - +- CometProject (120) - +- CometFilter (119) - +- CometScan parquet spark_catalog.default.date_dim (118) +BroadcastExchange (125) ++- * ColumnarToRow (124) + +- CometProject (123) + +- CometFilter (122) + +- CometScan parquet spark_catalog.default.date_dim (121) (unknown) Scan parquet spark_catalog.default.date_dim @@ -693,23 +708,23 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] ReadSchema: struct -(119) CometFilter +(122) CometFilter Input [2]: [d_date_sk#14, d_date#171] Condition : (((isnotnull(d_date#171) AND (d_date#171 >= 1998-08-04)) AND (d_date#171 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) -(120) CometProject +(123) CometProject Input [2]: [d_date_sk#14, d_date#171] Arguments: [d_date_sk#14], [d_date_sk#14] -(121) ColumnarToRow [codegen id : 1] +(124) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(122) BroadcastExchange +(125) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=19] -Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 69 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 71 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt index f6a37c637..34e47dcba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt @@ -1,25 +1,25 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (64) + WholeStageCodegen (100) HashAggregate [channel,id,sales,returns,profit] InputAdapter Exchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (63) + WholeStageCodegen (99) HashAggregate [channel,id,sales,returns,profit] InputAdapter Union - WholeStageCodegen (20) + WholeStageCodegen (32) HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel,id] #2 - WholeStageCodegen (19) + WholeStageCodegen (31) HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (6) + WholeStageCodegen (10) HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [s_store_id] #3 - WholeStageCodegen (5) + WholeStageCodegen (9) HashAggregate [s_store_id,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] Project [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] BroadcastHashJoin [ss_promo_sk,p_promo_sk] @@ -29,39 +29,49 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_ticket_number] - CometExchange [ss_item_sk,ss_ticket_number] #4 - CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_item_sk,sr_ticket_number] #6 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + Project [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] + SortMergeJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + InputAdapter + WholeStageCodegen (2) + Sort [ss_item_sk,ss_ticket_number] + InputAdapter + Exchange [ss_item_sk,ss_ticket_number] #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + WholeStageCodegen (4) + Sort [sr_item_sk,sr_ticket_number] + InputAdapter + Exchange [sr_item_sk,sr_ticket_number] #6 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometFilter [sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 InputAdapter BroadcastExchange #7 - WholeStageCodegen (2) + WholeStageCodegen (6) ColumnarToRow InputAdapter CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #8 - WholeStageCodegen (3) + WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [i_item_sk] @@ -69,17 +79,17 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] InputAdapter BroadcastExchange #9 - WholeStageCodegen (4) + WholeStageCodegen (8) ColumnarToRow InputAdapter CometProject [p_promo_sk] CometFilter [p_channel_tv,p_promo_sk] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - WholeStageCodegen (12) + WholeStageCodegen (20) HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [cp_catalog_page_id] #10 - WholeStageCodegen (11) + WholeStageCodegen (19) HashAggregate [cp_catalog_page_id,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] Project [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] BroadcastHashJoin [cs_promo_sk,p_promo_sk] @@ -89,25 +99,35 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] BroadcastHashJoin [cs_catalog_page_sk,cp_catalog_page_sk] Project [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number] - CometExchange [cs_item_sk,cs_order_number] #11 - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number] - CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + Project [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] + SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] + InputAdapter + WholeStageCodegen (12) + Sort [cs_item_sk,cs_order_number] + InputAdapter + Exchange [cs_item_sk,cs_order_number] #11 + WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + WholeStageCodegen (14) + Sort [cr_item_sk,cr_order_number] + InputAdapter + Exchange [cr_item_sk,cr_order_number] #12 + WholeStageCodegen (13) + ColumnarToRow + InputAdapter + CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometFilter [cr_item_sk,cr_order_number] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 InputAdapter BroadcastExchange #13 - WholeStageCodegen (8) + WholeStageCodegen (16) ColumnarToRow InputAdapter CometFilter [cp_catalog_page_sk] @@ -116,11 +136,11 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedExchange [i_item_sk] #8 InputAdapter ReusedExchange [p_promo_sk] #9 - WholeStageCodegen (18) + WholeStageCodegen (30) HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [web_site_id] #14 - WholeStageCodegen (17) + WholeStageCodegen (29) HashAggregate [web_site_id,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] Project [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] BroadcastHashJoin [ws_promo_sk,p_promo_sk] @@ -130,25 +150,35 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] BroadcastHashJoin [ws_web_site_sk,web_site_sk] Project [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] - CometSort [ws_item_sk,ws_order_number] - CometExchange [ws_item_sk,ws_order_number] #15 - CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [wr_item_sk,wr_order_number] - CometExchange [wr_item_sk,wr_order_number] #16 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number] - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + Project [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] + SortMergeJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] + InputAdapter + WholeStageCodegen (22) + Sort [ws_item_sk,ws_order_number] + InputAdapter + Exchange [ws_item_sk,ws_order_number] #15 + WholeStageCodegen (21) + ColumnarToRow + InputAdapter + CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + WholeStageCodegen (24) + Sort [wr_item_sk,wr_order_number] + InputAdapter + Exchange [wr_item_sk,wr_order_number] #16 + WholeStageCodegen (23) + ColumnarToRow + InputAdapter + CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometFilter [wr_item_sk,wr_order_number] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 InputAdapter BroadcastExchange #17 - WholeStageCodegen (14) + WholeStageCodegen (26) ColumnarToRow InputAdapter CometFilter [web_site_sk] @@ -157,20 +187,20 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedExchange [i_item_sk] #8 InputAdapter ReusedExchange [p_promo_sk] #9 - WholeStageCodegen (41) + WholeStageCodegen (65) HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel] #18 - WholeStageCodegen (40) + WholeStageCodegen (64) HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (62) + WholeStageCodegen (98) HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange #19 - WholeStageCodegen (61) + WholeStageCodegen (97) HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index c8c7ffd5c..e3390296a 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -1369,8 +1369,10 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { assume(isSpark34Plus) withSQLConf( SQLConf.ANSI_ENABLED.key -> "false", + SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "true", CometConf.COMET_ENABLED.key -> "true", CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_SHUFFLE_ENFORCE_MODE_ENABLED.key -> "true", CometConf.COMET_EXEC_ALL_EXPR_ENABLED.key -> "true", CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true", "spark.sql.extendedExplainProvider" -> "org.apache.comet.ExtendedExplainInfo") { diff --git a/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala index fc6876fd1..64c031eed 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala @@ -549,7 +549,10 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { Seq(1, 100, 10000).foreach { numGroups => Seq(128, 1024, numValues + 1).foreach { batchSize => Seq(true, false).foreach { dictionaryEnabled => - withSQLConf(CometConf.COMET_BATCH_SIZE.key -> batchSize.toString) { + withSQLConf( + SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "true", + CometConf.COMET_SHUFFLE_ENFORCE_MODE_ENABLED.key -> "true", + CometConf.COMET_BATCH_SIZE.key -> batchSize.toString) { withParquetTable( (0 until numValues).map(i => (i, Random.nextInt() % numGroups)), "tbl", @@ -573,7 +576,10 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { Seq(1, 100, numValues).foreach { numGroups => Seq(128, numValues + 100).foreach { batchSize => Seq(true, false).foreach { dictionaryEnabled => - withSQLConf(CometConf.COMET_BATCH_SIZE.key -> batchSize.toString) { + withSQLConf( + SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "true", + CometConf.COMET_SHUFFLE_ENFORCE_MODE_ENABLED.key -> "true", + CometConf.COMET_BATCH_SIZE.key -> batchSize.toString) { withTempPath { dir => val path = new Path(dir.toURI.toString, "test.parquet") makeParquetFile(path, numValues, numGroups, dictionaryEnabled) @@ -611,7 +617,10 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { Seq(1, 100, numValues).foreach { numGroups => Seq(128, numValues + 100).foreach { batchSize => Seq(true, false).foreach { dictionaryEnabled => - withSQLConf(CometConf.COMET_BATCH_SIZE.key -> batchSize.toString) { + withSQLConf( + SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "true", + CometConf.COMET_SHUFFLE_ENFORCE_MODE_ENABLED.key -> "true", + CometConf.COMET_BATCH_SIZE.key -> batchSize.toString) { withTempPath { dir => val path = new Path(dir.toURI.toString, "test.parquet") makeParquetFile(path, numValues, numGroups, dictionaryEnabled) @@ -958,7 +967,9 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("first/last") { withSQLConf( + SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "true", CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", + CometConf.COMET_SHUFFLE_ENFORCE_MODE_ENABLED.key -> "true", CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "true") { Seq(true, false).foreach { dictionary => withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { diff --git a/spark/src/test/scala/org/apache/comet/exec/CometColumnarShuffleSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometColumnarShuffleSuite.scala index 4b4f60a10..bfbd16749 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometColumnarShuffleSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometColumnarShuffleSuite.scala @@ -64,6 +64,27 @@ abstract class CometColumnarShuffleSuite extends CometTestBase with AdaptiveSpar import testImplicits._ + setupTestData() + + test("Disable Comet shuffle with AQE coalesce partitions enabled") { + Seq(true, false).foreach { coalescePartitionsEnabled => + withSQLConf( + CometConf.COMET_EXEC_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.COALESCE_PARTITIONS_ENABLED.key -> coalescePartitionsEnabled.toString, + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val df = sql( + "SELECT * FROM (SELECT * FROM testData WHERE key = 0) t1 FULL JOIN " + + "testData2 t2 ON t1.key = t2.a") + if (coalescePartitionsEnabled) { + checkShuffleAnswer(df, 0) + } else { + checkShuffleAnswer(df, 2) + } + } + } + } + test("columnar shuffle on nested struct including nulls") { Seq(10, 201).foreach { numPartitions => Seq("1.0", "10.0").foreach { ratio => diff --git a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala index bfde14033..47c2c696a 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -339,11 +339,11 @@ class CometExecSuite extends CometTestBase { }.map(_.metrics).get assert(metrics.contains("input_batches")) - assert(metrics("input_batches").value == 2L) + assert(metrics("input_batches").value == 8L) assert(metrics.contains("input_rows")) assert(metrics("input_rows").value == 10L) assert(metrics.contains("output_batches")) - assert(metrics("output_batches").value == 1L) + assert(metrics("output_batches").value == 4L) assert(metrics.contains("output_rows")) assert(metrics("output_rows").value == 5L) assert(metrics.contains("peak_mem_used")) diff --git a/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala b/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala index 8fda13617..1ed447dc3 100644 --- a/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala +++ b/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala @@ -76,6 +76,7 @@ abstract class CometTestBase conf.set(MEMORY_OFFHEAP_SIZE.key, "2g") conf.set(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key, "1g") conf.set(SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key, "1g") + conf.set(SQLConf.COALESCE_PARTITIONS_ENABLED.key, "false") conf.set(CometConf.COMET_ENABLED.key, "true") conf.set(CometConf.COMET_EXEC_ENABLED.key, "true") conf.set(CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key, "true")