diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala index c2fb74a85..9cb7a849c 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala @@ -363,7 +363,11 @@ abstract class CometNativeExec extends CometExec { } override protected def doCanonicalize(): SparkPlan = { - val canonicalizedPlan = super.doCanonicalize().asInstanceOf[CometNativeExec] + val canonicalizedPlan = super + .doCanonicalize() + .asInstanceOf[CometNativeExec] + .canonicalizePlans() + if (serializedPlanOpt.isDefined) { // If the plan is a boundary node, we should remove the serialized plan. canonicalizedPlan.cleanBlock() @@ -371,6 +375,20 @@ abstract class CometNativeExec extends CometExec { canonicalizedPlan } } + + /** + * Canonicalizes the plans of Product parameters in Comet native operators. + */ + protected def canonicalizePlans(): CometNativeExec = { + def transform(arg: Any): AnyRef = arg match { + case sparkPlan: SparkPlan => sparkPlan.canonicalized + case other: AnyRef => other + case null => null + } + + val newArgs = mapProductIterator(transform) + makeCopy(newArgs).asInstanceOf[CometNativeExec] + } } abstract class CometUnaryExec extends CometNativeExec with UnaryExecNode diff --git a/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometScanExec.scala b/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometScanExec.scala index 18871b814..544a67385 100644 --- a/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometScanExec.scala +++ b/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometScanExec.scala @@ -69,6 +69,8 @@ trait ShimCometScanExec { readSchema: StructType, options: ParquetOptions): FileScanRDD = classOf[FileScanRDD].getDeclaredConstructors + // Prevent to pick up incorrect constructors from any custom Spark forks. + .filter(c => List(3, 5, 6).contains(c.getParameterCount()) ) .map { c => c.getParameterCount match { case 3 => c.newInstance(sparkSession, readFunction, filePartitions) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt index 02d9a6ca4..0f1219387 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt @@ -1,16 +1,16 @@ == Physical Plan == -TakeOrderedAndProject (51) -+- * HashAggregate (50) - +- Exchange (49) - +- * HashAggregate (48) - +- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (34) - : : +- * Filter (33) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (32) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (22) +TakeOrderedAndProject (45) ++- * HashAggregate (44) + +- Exchange (43) + +- * HashAggregate (42) + +- * Project (41) + +- * BroadcastHashJoin Inner BuildRight (40) + :- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * Project (28) + : : +- * Filter (27) + : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) + : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) : : : :- * ColumnarToRow (12) : : : : +- CometBroadcastHashJoin (11) : : : : :- CometFilter (2) @@ -23,33 +23,27 @@ TakeOrderedAndProject (51) : : : : +- CometProject (6) : : : : +- CometFilter (5) : : : : +- CometScan parquet spark_catalog.default.date_dim (4) - : : : +- BroadcastExchange (21) - : : : +- * ColumnarToRow (20) - : : : +- CometProject (19) - : : : +- CometBroadcastHashJoin (18) + : : : +- BroadcastExchange (18) + : : : +- * ColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometBroadcastHashJoin (15) : : : :- CometScan parquet spark_catalog.default.web_sales (13) - : : : +- CometBroadcastExchange (17) - : : : +- CometProject (16) - : : : +- CometFilter (15) - : : : +- CometScan parquet spark_catalog.default.date_dim (14) - : : +- BroadcastExchange (31) - : : +- * ColumnarToRow (30) - : : +- CometProject (29) - : : +- CometBroadcastHashJoin (28) - : : :- CometScan parquet spark_catalog.default.catalog_sales (23) - : : +- CometBroadcastExchange (27) - : : +- CometProject (26) - : : +- CometFilter (25) - : : +- CometScan parquet spark_catalog.default.date_dim (24) - : +- BroadcastExchange (39) - : +- * ColumnarToRow (38) - : +- CometProject (37) - : +- CometFilter (36) - : +- CometScan parquet spark_catalog.default.customer_address (35) - +- BroadcastExchange (45) - +- * ColumnarToRow (44) - +- CometFilter (43) - +- CometScan parquet spark_catalog.default.customer_demographics (42) + : : : +- ReusedExchange (14) + : : +- BroadcastExchange (25) + : : +- * ColumnarToRow (24) + : : +- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometScan parquet spark_catalog.default.catalog_sales (20) + : : +- ReusedExchange (21) + : +- BroadcastExchange (33) + : +- * ColumnarToRow (32) + : +- CometProject (31) + : +- CometFilter (30) + : +- CometScan parquet spark_catalog.default.customer_address (29) + +- BroadcastExchange (39) + +- * ColumnarToRow (38) + +- CometFilter (37) + +- CometScan parquet spark_catalog.default.customer_demographics (36) (1) Scan parquet spark_catalog.default.customer @@ -117,219 +111,187 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#14)] ReadSchema: struct -(14) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_year#16, d_moy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [d_date_sk#15, d_year#16, d_moy#17] -Condition : (((((isnotnull(d_year#16) AND isnotnull(d_moy#17)) AND (d_year#16 = 2002)) AND (d_moy#17 >= 1)) AND (d_moy#17 <= 4)) AND isnotnull(d_date_sk#15)) +(14) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#15] -(16) CometProject -Input [3]: [d_date_sk#15, d_year#16, d_moy#17] -Arguments: [d_date_sk#15], [d_date_sk#15] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#15] -Arguments: [d_date_sk#15] - -(18) CometBroadcastHashJoin +(15) CometBroadcastHashJoin Left output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] Right output [1]: [d_date_sk#15] Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner -(19) CometProject +(16) CometProject Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] -(20) ColumnarToRow [codegen id : 1] +(17) ColumnarToRow [codegen id : 1] Input [1]: [ws_bill_customer_sk#12] -(21) BroadcastExchange +(18) BroadcastExchange Input [1]: [ws_bill_customer_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(22) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#12] Join type: ExistenceJoin(exists#2) Join condition: None -(23) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] +(20) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#20)] +PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] ReadSchema: struct -(24) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#21, d_year#22, d_moy#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [3]: [d_date_sk#21, d_year#22, d_moy#23] -Condition : (((((isnotnull(d_year#22) AND isnotnull(d_moy#23)) AND (d_year#22 = 2002)) AND (d_moy#23 >= 1)) AND (d_moy#23 <= 4)) AND isnotnull(d_date_sk#21)) +(21) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#19] -(26) CometProject -Input [3]: [d_date_sk#21, d_year#22, d_moy#23] -Arguments: [d_date_sk#21], [d_date_sk#21] +(22) CometBroadcastHashJoin +Left output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Right output [1]: [d_date_sk#19] +Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner -(27) CometBroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: [d_date_sk#21] +(23) CometProject +Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] +Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] -(28) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] -Right output [1]: [d_date_sk#21] -Arguments: [cs_sold_date_sk#19], [d_date_sk#21], Inner +(24) ColumnarToRow [codegen id : 2] +Input [1]: [cs_ship_customer_sk#16] -(29) CometProject -Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#21] -Arguments: [cs_ship_customer_sk#18], [cs_ship_customer_sk#18] - -(30) ColumnarToRow [codegen id : 2] -Input [1]: [cs_ship_customer_sk#18] - -(31) BroadcastExchange -Input [1]: [cs_ship_customer_sk#18] +(25) BroadcastExchange +Input [1]: [cs_ship_customer_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(32) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#18] +Right keys [1]: [cs_ship_customer_sk#16] Join type: ExistenceJoin(exists#1) Join condition: None -(33) Filter [codegen id : 5] +(27) Filter [codegen id : 5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(34) Project [codegen id : 5] +(28) Project [codegen id : 5] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(35) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#24, ca_county#25] +(29) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#20, ca_county#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Dona Ana County,Jefferson County,La Porte County,Rush County,Toole County]), IsNotNull(ca_address_sk)] ReadSchema: struct -(36) CometFilter -Input [2]: [ca_address_sk#24, ca_county#25] -Condition : (ca_county#25 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#24)) +(30) CometFilter +Input [2]: [ca_address_sk#20, ca_county#21] +Condition : (ca_county#21 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#20)) -(37) CometProject -Input [2]: [ca_address_sk#24, ca_county#25] -Arguments: [ca_address_sk#24], [ca_address_sk#24] +(31) CometProject +Input [2]: [ca_address_sk#20, ca_county#21] +Arguments: [ca_address_sk#20], [ca_address_sk#20] -(38) ColumnarToRow [codegen id : 3] -Input [1]: [ca_address_sk#24] +(32) ColumnarToRow [codegen id : 3] +Input [1]: [ca_address_sk#20] -(39) BroadcastExchange -Input [1]: [ca_address_sk#24] +(33) BroadcastExchange +Input [1]: [ca_address_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(40) BroadcastHashJoin [codegen id : 5] +(34) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#24] +Right keys [1]: [ca_address_sk#20] Join type: Inner Join condition: None -(41) Project [codegen id : 5] +(35) Project [codegen id : 5] Output [1]: [c_current_cdemo_sk#4] -Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#24] +Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20] -(42) Scan parquet spark_catalog.default.customer_demographics -Output [9]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] +(36) Scan parquet spark_catalog.default.customer_demographics +Output [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(43) CometFilter -Input [9]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] -Condition : isnotnull(cd_demo_sk#26) +(37) CometFilter +Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Condition : isnotnull(cd_demo_sk#22) -(44) ColumnarToRow [codegen id : 4] -Input [9]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] +(38) ColumnarToRow [codegen id : 4] +Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(45) BroadcastExchange -Input [9]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] +(39) BroadcastExchange +Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(46) BroadcastHashJoin [codegen id : 5] +(40) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#26] +Right keys [1]: [cd_demo_sk#22] Join type: Inner Join condition: None -(47) Project [codegen id : 5] -Output [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] -Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] +(41) Project [codegen id : 5] +Output [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(48) HashAggregate [codegen id : 5] -Input [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] -Keys [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] +(42) HashAggregate [codegen id : 5] +Input [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Keys [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#35] -Results [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34, count#36] +Aggregate Attributes [1]: [count#31] +Results [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#32] -(49) Exchange -Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34, count#36] -Arguments: hashpartitioning(cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(43) Exchange +Input [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#32] +Arguments: hashpartitioning(cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(50) HashAggregate [codegen id : 6] -Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34, count#36] -Keys [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] +(44) HashAggregate [codegen id : 6] +Input [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#32] +Keys [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#37] -Results [14]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, count(1)#37 AS cnt1#38, cd_purchase_estimate#30, count(1)#37 AS cnt2#39, cd_credit_rating#31, count(1)#37 AS cnt3#40, cd_dep_count#32, count(1)#37 AS cnt4#41, cd_dep_employed_count#33, count(1)#37 AS cnt5#42, cd_dep_college_count#34, count(1)#37 AS cnt6#43] +Aggregate Attributes [1]: [count(1)#33] +Results [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, count(1)#33 AS cnt1#34, cd_purchase_estimate#26, count(1)#33 AS cnt2#35, cd_credit_rating#27, count(1)#33 AS cnt3#36, cd_dep_count#28, count(1)#33 AS cnt4#37, cd_dep_employed_count#29, count(1)#33 AS cnt5#38, cd_dep_college_count#30, count(1)#33 AS cnt6#39] -(51) TakeOrderedAndProject -Input [14]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cnt1#38, cd_purchase_estimate#30, cnt2#39, cd_credit_rating#31, cnt3#40, cd_dep_count#32, cnt4#41, cd_dep_employed_count#33, cnt5#42, cd_dep_college_count#34, cnt6#43] -Arguments: 100, [cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, cd_education_status#29 ASC NULLS FIRST, cd_purchase_estimate#30 ASC NULLS FIRST, cd_credit_rating#31 ASC NULLS FIRST, cd_dep_count#32 ASC NULLS FIRST, cd_dep_employed_count#33 ASC NULLS FIRST, cd_dep_college_count#34 ASC NULLS FIRST], [cd_gender#27, cd_marital_status#28, cd_education_status#29, cnt1#38, cd_purchase_estimate#30, cnt2#39, cd_credit_rating#31, cnt3#40, cd_dep_count#32, cnt4#41, cd_dep_employed_count#33, cnt5#42, cd_dep_college_count#34, cnt6#43] +(45) TakeOrderedAndProject +Input [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#34, cd_purchase_estimate#26, cnt2#35, cd_credit_rating#27, cnt3#36, cd_dep_count#28, cnt4#37, cd_dep_employed_count#29, cnt5#38, cd_dep_college_count#30, cnt6#39] +Arguments: 100, [cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_education_status#25 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#34, cd_purchase_estimate#26, cnt2#35, cd_credit_rating#27, cnt3#36, cd_dep_count#28, cnt4#37, cd_dep_employed_count#29, cnt5#38, cd_dep_college_count#30, cnt6#39] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (56) -+- * ColumnarToRow (55) - +- CometProject (54) - +- CometFilter (53) - +- CometScan parquet spark_catalog.default.date_dim (52) +BroadcastExchange (50) ++- * ColumnarToRow (49) + +- CometProject (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) -(52) Scan parquet spark_catalog.default.date_dim +(46) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#9, d_year#10, d_moy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(53) CometFilter +(47) CometFilter Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Condition : (((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2002)) AND (d_moy#11 >= 1)) AND (d_moy#11 <= 4)) AND isnotnull(d_date_sk#9)) -(54) CometProject +(48) CometProject Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(55) ColumnarToRow [codegen id : 1] +(49) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(56) BroadcastExchange +(50) BroadcastExchange Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt index f262a260d..19243e359 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt @@ -43,12 +43,9 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #6 - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #7 + BroadcastExchange #6 WholeStageCodegen (2) ColumnarToRow InputAdapter @@ -56,12 +53,9 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #8 - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #9 + BroadcastExchange #7 WholeStageCodegen (3) ColumnarToRow InputAdapter @@ -69,7 +63,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha CometFilter [ca_county,ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter - BroadcastExchange #10 + BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt index 08a37f841..e403ab61f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (76) -+- * Project (75) - +- * BroadcastHashJoin Inner BuildRight (74) - :- * Project (56) - : +- * BroadcastHashJoin Inner BuildRight (55) +TakeOrderedAndProject (72) ++- * Project (71) + +- * BroadcastHashJoin Inner BuildRight (70) + :- * Project (54) + : +- * BroadcastHashJoin Inner BuildRight (53) : :- * Project (36) : : +- * BroadcastHashJoin Inner BuildRight (35) : : :- * Filter (17) @@ -40,14 +40,14 @@ TakeOrderedAndProject (76) : : +- CometBroadcastExchange (27) : : +- CometFilter (26) : : +- CometScan parquet spark_catalog.default.date_dim (25) - : +- BroadcastExchange (54) - : +- * Filter (53) - : +- * HashAggregate (52) - : +- Exchange (51) - : +- * ColumnarToRow (50) - : +- CometHashAggregate (49) - : +- CometProject (48) - : +- CometBroadcastHashJoin (47) + : +- BroadcastExchange (52) + : +- * Filter (51) + : +- * HashAggregate (50) + : +- Exchange (49) + : +- * ColumnarToRow (48) + : +- CometHashAggregate (47) + : +- CometProject (46) + : +- CometBroadcastHashJoin (45) : :- CometProject (43) : : +- CometBroadcastHashJoin (42) : : :- CometFilter (38) @@ -55,26 +55,22 @@ TakeOrderedAndProject (76) : : +- CometBroadcastExchange (41) : : +- CometFilter (40) : : +- CometScan parquet spark_catalog.default.web_sales (39) - : +- CometBroadcastExchange (46) - : +- CometFilter (45) - : +- CometScan parquet spark_catalog.default.date_dim (44) - +- BroadcastExchange (73) - +- * HashAggregate (72) - +- Exchange (71) - +- * ColumnarToRow (70) - +- CometHashAggregate (69) - +- CometProject (68) - +- CometBroadcastHashJoin (67) - :- CometProject (63) - : +- CometBroadcastHashJoin (62) - : :- CometFilter (58) - : : +- CometScan parquet spark_catalog.default.customer (57) - : +- CometBroadcastExchange (61) - : +- CometFilter (60) - : +- CometScan parquet spark_catalog.default.web_sales (59) - +- CometBroadcastExchange (66) - +- CometFilter (65) - +- CometScan parquet spark_catalog.default.date_dim (64) + : +- ReusedExchange (44) + +- BroadcastExchange (69) + +- * HashAggregate (68) + +- Exchange (67) + +- * ColumnarToRow (66) + +- CometHashAggregate (65) + +- CometProject (64) + +- CometBroadcastHashJoin (63) + :- CometProject (61) + : +- CometBroadcastHashJoin (60) + : :- CometFilter (56) + : : +- CometScan parquet spark_catalog.default.customer (55) + : +- CometBroadcastExchange (59) + : +- CometFilter (58) + : +- CometScan parquet spark_catalog.default.web_sales (57) + +- ReusedExchange (62) (1) Scan parquet spark_catalog.default.customer @@ -289,79 +285,67 @@ Arguments: [c_customer_sk#39], [ws_bill_customer_sk#47], Inner Input [12]: [c_customer_sk#39, c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, ws_bill_customer_sk#47, ws_ext_discount_amt#48, ws_ext_list_price#49, ws_sold_date_sk#50] Arguments: [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, ws_ext_discount_amt#48, ws_ext_list_price#49, ws_sold_date_sk#50], [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, ws_ext_discount_amt#48, ws_ext_list_price#49, ws_sold_date_sk#50] -(44) Scan parquet spark_catalog.default.date_dim +(44) ReusedExchange [Reuses operator id: 10] Output [2]: [d_date_sk#52, d_year#53] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct -(45) CometFilter -Input [2]: [d_date_sk#52, d_year#53] -Condition : ((isnotnull(d_year#53) AND (d_year#53 = 2001)) AND isnotnull(d_date_sk#52)) - -(46) CometBroadcastExchange -Input [2]: [d_date_sk#52, d_year#53] -Arguments: [d_date_sk#52, d_year#53] - -(47) CometBroadcastHashJoin +(45) CometBroadcastHashJoin Left output [10]: [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, ws_ext_discount_amt#48, ws_ext_list_price#49, ws_sold_date_sk#50] Right output [2]: [d_date_sk#52, d_year#53] Arguments: [ws_sold_date_sk#50], [d_date_sk#52], Inner -(48) CometProject +(46) CometProject Input [12]: [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, ws_ext_discount_amt#48, ws_ext_list_price#49, ws_sold_date_sk#50, d_date_sk#52, d_year#53] Arguments: [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, ws_ext_discount_amt#48, ws_ext_list_price#49, d_year#53], [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, ws_ext_discount_amt#48, ws_ext_list_price#49, d_year#53] -(49) CometHashAggregate +(47) CometHashAggregate Input [10]: [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, ws_ext_discount_amt#48, ws_ext_list_price#49, d_year#53] Keys [8]: [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, d_year#53] Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#49 - ws_ext_discount_amt#48)))] -(50) ColumnarToRow [codegen id : 4] +(48) ColumnarToRow [codegen id : 4] Input [9]: [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, d_year#53, sum#54] -(51) Exchange +(49) Exchange Input [9]: [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, d_year#53, sum#54] Arguments: hashpartitioning(c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, d_year#53, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(52) HashAggregate [codegen id : 5] +(50) HashAggregate [codegen id : 5] Input [9]: [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, d_year#53, sum#54] Keys [8]: [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, d_year#53] Functions [1]: [sum(UnscaledValue((ws_ext_list_price#49 - ws_ext_discount_amt#48)))] Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#49 - ws_ext_discount_amt#48)))#55] Results [2]: [c_customer_id#40 AS customer_id#56, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#49 - ws_ext_discount_amt#48)))#55,18,2) AS year_total#57] -(53) Filter [codegen id : 5] +(51) Filter [codegen id : 5] Input [2]: [customer_id#56, year_total#57] Condition : (isnotnull(year_total#57) AND (year_total#57 > 0.00)) -(54) BroadcastExchange +(52) BroadcastExchange Input [2]: [customer_id#56, year_total#57] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(55) BroadcastHashJoin [codegen id : 8] +(53) BroadcastHashJoin [codegen id : 8] Left keys [1]: [customer_id#18] Right keys [1]: [customer_id#56] Join type: Inner Join condition: None -(56) Project [codegen id : 8] +(54) Project [codegen id : 8] Output [5]: [customer_id#18, year_total#19, customer_preferred_cust_flag#37, year_total#38, year_total#57] Input [6]: [customer_id#18, year_total#19, customer_preferred_cust_flag#37, year_total#38, customer_id#56, year_total#57] -(57) Scan parquet spark_catalog.default.customer +(55) Scan parquet spark_catalog.default.customer Output [8]: [c_customer_sk#58, c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(58) CometFilter +(56) CometFilter Input [8]: [c_customer_sk#58, c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65] Condition : (isnotnull(c_customer_sk#58) AND isnotnull(c_customer_id#59)) -(59) Scan parquet spark_catalog.default.web_sales +(57) Scan parquet spark_catalog.default.web_sales Output [4]: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] Batched: true Location: InMemoryFileIndex [] @@ -369,138 +353,126 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#69), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(60) CometFilter +(58) CometFilter Input [4]: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] Condition : isnotnull(ws_bill_customer_sk#66) -(61) CometBroadcastExchange +(59) CometBroadcastExchange Input [4]: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] Arguments: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] -(62) CometBroadcastHashJoin +(60) CometBroadcastHashJoin Left output [8]: [c_customer_sk#58, c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65] Right output [4]: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] Arguments: [c_customer_sk#58], [ws_bill_customer_sk#66], Inner -(63) CometProject +(61) CometProject Input [12]: [c_customer_sk#58, c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] Arguments: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69], [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] -(64) Scan parquet spark_catalog.default.date_dim +(62) ReusedExchange [Reuses operator id: 27] Output [2]: [d_date_sk#71, d_year#72] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct -(65) CometFilter -Input [2]: [d_date_sk#71, d_year#72] -Condition : ((isnotnull(d_year#72) AND (d_year#72 = 2002)) AND isnotnull(d_date_sk#71)) - -(66) CometBroadcastExchange -Input [2]: [d_date_sk#71, d_year#72] -Arguments: [d_date_sk#71, d_year#72] - -(67) CometBroadcastHashJoin +(63) CometBroadcastHashJoin Left output [10]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] Right output [2]: [d_date_sk#71, d_year#72] Arguments: [ws_sold_date_sk#69], [d_date_sk#71], Inner -(68) CometProject +(64) CometProject Input [12]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69, d_date_sk#71, d_year#72] Arguments: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, d_year#72], [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, d_year#72] -(69) CometHashAggregate +(65) CometHashAggregate Input [10]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, d_year#72] Keys [8]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, d_year#72] Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#68 - ws_ext_discount_amt#67)))] -(70) ColumnarToRow [codegen id : 6] +(66) ColumnarToRow [codegen id : 6] Input [9]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, d_year#72, sum#73] -(71) Exchange +(67) Exchange Input [9]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, d_year#72, sum#73] Arguments: hashpartitioning(c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, d_year#72, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(72) HashAggregate [codegen id : 7] +(68) HashAggregate [codegen id : 7] Input [9]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, d_year#72, sum#73] Keys [8]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, d_year#72] Functions [1]: [sum(UnscaledValue((ws_ext_list_price#68 - ws_ext_discount_amt#67)))] Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#68 - ws_ext_discount_amt#67)))#55] Results [2]: [c_customer_id#59 AS customer_id#74, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#68 - ws_ext_discount_amt#67)))#55,18,2) AS year_total#75] -(73) BroadcastExchange +(69) BroadcastExchange Input [2]: [customer_id#74, year_total#75] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] -(74) BroadcastHashJoin [codegen id : 8] +(70) BroadcastHashJoin [codegen id : 8] Left keys [1]: [customer_id#18] Right keys [1]: [customer_id#74] Join type: Inner Join condition: (CASE WHEN (year_total#57 > 0.00) THEN (year_total#75 / year_total#57) END > CASE WHEN (year_total#19 > 0.00) THEN (year_total#38 / year_total#19) END) -(75) Project [codegen id : 8] +(71) Project [codegen id : 8] Output [1]: [customer_preferred_cust_flag#37] Input [7]: [customer_id#18, year_total#19, customer_preferred_cust_flag#37, year_total#38, year_total#57, customer_id#74, year_total#75] -(76) TakeOrderedAndProject +(72) TakeOrderedAndProject Input [1]: [customer_preferred_cust_flag#37] Arguments: 100, [customer_preferred_cust_flag#37 ASC NULLS FIRST], [customer_preferred_cust_flag#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (80) -+- * ColumnarToRow (79) - +- CometFilter (78) - +- CometScan parquet spark_catalog.default.date_dim (77) +BroadcastExchange (76) ++- * ColumnarToRow (75) + +- CometFilter (74) + +- CometScan parquet spark_catalog.default.date_dim (73) -(77) Scan parquet spark_catalog.default.date_dim +(73) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#14, d_year#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(78) CometFilter +(74) CometFilter Input [2]: [d_date_sk#14, d_year#15] Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2001)) AND isnotnull(d_date_sk#14)) -(79) ColumnarToRow [codegen id : 1] +(75) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#14, d_year#15] -(80) BroadcastExchange +(76) BroadcastExchange Input [2]: [d_date_sk#14, d_year#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#31 IN dynamicpruning#32 -BroadcastExchange (84) -+- * ColumnarToRow (83) - +- CometFilter (82) - +- CometScan parquet spark_catalog.default.date_dim (81) +BroadcastExchange (80) ++- * ColumnarToRow (79) + +- CometFilter (78) + +- CometScan parquet spark_catalog.default.date_dim (77) -(81) Scan parquet spark_catalog.default.date_dim +(77) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#33, d_year#34] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(82) CometFilter +(78) CometFilter Input [2]: [d_date_sk#33, d_year#34] Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2002)) AND isnotnull(d_date_sk#33)) -(83) ColumnarToRow [codegen id : 1] +(79) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#33, d_year#34] -(84) BroadcastExchange +(80) BroadcastExchange Input [2]: [d_date_sk#33, d_year#34] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#50 IN dynamicpruning#13 -Subquery:4 Hosting operator id = 59 Hosting Expression = ws_sold_date_sk#69 IN dynamicpruning#32 +Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#69 IN dynamicpruning#32 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt index 325b901d8..dd1a52206 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt @@ -83,15 +83,13 @@ TakeOrderedAndProject [customer_preferred_cust_flag] CometFilter [ws_bill_customer_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #13 - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk,d_year] #4 InputAdapter - BroadcastExchange #14 + BroadcastExchange #13 WholeStageCodegen (7) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #15 + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 WholeStageCodegen (6) ColumnarToRow InputAdapter @@ -102,10 +100,8 @@ TakeOrderedAndProject [customer_preferred_cust_flag] CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #16 + CometBroadcastExchange #15 CometFilter [ws_bill_customer_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #17 - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt index d9d55eca2..b38a8fb26 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt @@ -1,37 +1,37 @@ == Physical Plan == -TakeOrderedAndProject (113) -+- * HashAggregate (112) - +- Exchange (111) - +- * HashAggregate (110) - +- * Expand (109) - +- Union (108) - :- * Project (75) - : +- * Filter (74) - : +- * HashAggregate (73) - : +- Exchange (72) - : +- * HashAggregate (71) - : +- * Project (70) - : +- * BroadcastHashJoin Inner BuildRight (69) - : :- * Project (67) - : : +- * BroadcastHashJoin Inner BuildRight (66) - : : :- * BroadcastHashJoin LeftSemi BuildRight (59) +TakeOrderedAndProject (105) ++- * HashAggregate (104) + +- Exchange (103) + +- * HashAggregate (102) + +- * Expand (101) + +- Union (100) + :- * Project (67) + : +- * Filter (66) + : +- * HashAggregate (65) + : +- Exchange (64) + : +- * HashAggregate (63) + : +- * Project (62) + : +- * BroadcastHashJoin Inner BuildRight (61) + : :- * Project (59) + : : +- * BroadcastHashJoin Inner BuildRight (58) + : : :- * BroadcastHashJoin LeftSemi BuildRight (51) : : : :- * ColumnarToRow (3) : : : : +- CometFilter (2) : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (58) - : : : +- * Project (57) - : : : +- * BroadcastHashJoin Inner BuildRight (56) + : : : +- BroadcastExchange (50) + : : : +- * Project (49) + : : : +- * BroadcastHashJoin Inner BuildRight (48) : : : :- * ColumnarToRow (6) : : : : +- CometFilter (5) : : : : +- CometScan parquet spark_catalog.default.item (4) - : : : +- BroadcastExchange (55) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (54) - : : : :- * HashAggregate (38) - : : : : +- Exchange (37) - : : : : +- * ColumnarToRow (36) - : : : : +- CometHashAggregate (35) - : : : : +- CometProject (34) - : : : : +- CometBroadcastHashJoin (33) + : : : +- BroadcastExchange (47) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) + : : : :- * HashAggregate (35) + : : : : +- Exchange (34) + : : : : +- * ColumnarToRow (33) + : : : : +- CometHashAggregate (32) + : : : : +- CometProject (31) + : : : : +- CometBroadcastHashJoin (30) : : : : :- CometProject (28) : : : : : +- CometBroadcastHashJoin (27) : : : : : :- CometFilter (8) @@ -54,64 +54,56 @@ TakeOrderedAndProject (113) : : : : : +- CometProject (20) : : : : : +- CometFilter (19) : : : : : +- CometScan parquet spark_catalog.default.date_dim (18) - : : : : +- CometBroadcastExchange (32) - : : : : +- CometProject (31) - : : : : +- CometFilter (30) - : : : : +- CometScan parquet spark_catalog.default.date_dim (29) - : : : +- BroadcastExchange (53) - : : : +- * ColumnarToRow (52) - : : : +- CometProject (51) - : : : +- CometBroadcastHashJoin (50) - : : : :- CometProject (45) - : : : : +- CometBroadcastHashJoin (44) - : : : : :- CometFilter (40) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (39) - : : : : +- CometBroadcastExchange (43) - : : : : +- CometFilter (42) - : : : : +- CometScan parquet spark_catalog.default.item (41) - : : : +- CometBroadcastExchange (49) - : : : +- CometProject (48) - : : : +- CometFilter (47) - : : : +- CometScan parquet spark_catalog.default.date_dim (46) - : : +- BroadcastExchange (65) - : : +- * BroadcastHashJoin LeftSemi BuildRight (64) - : : :- * ColumnarToRow (62) - : : : +- CometFilter (61) - : : : +- CometScan parquet spark_catalog.default.item (60) - : : +- ReusedExchange (63) - : +- ReusedExchange (68) - :- * Project (91) - : +- * Filter (90) - : +- * HashAggregate (89) - : +- Exchange (88) - : +- * HashAggregate (87) - : +- * Project (86) - : +- * BroadcastHashJoin Inner BuildRight (85) - : :- * Project (83) - : : +- * BroadcastHashJoin Inner BuildRight (82) - : : :- * BroadcastHashJoin LeftSemi BuildRight (80) - : : : :- * ColumnarToRow (78) - : : : : +- CometFilter (77) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (76) - : : : +- ReusedExchange (79) - : : +- ReusedExchange (81) - : +- ReusedExchange (84) - +- * Project (107) - +- * Filter (106) - +- * HashAggregate (105) - +- Exchange (104) - +- * HashAggregate (103) - +- * Project (102) - +- * BroadcastHashJoin Inner BuildRight (101) - :- * Project (99) - : +- * BroadcastHashJoin Inner BuildRight (98) - : :- * BroadcastHashJoin LeftSemi BuildRight (96) - : : :- * ColumnarToRow (94) - : : : +- CometFilter (93) - : : : +- CometScan parquet spark_catalog.default.web_sales (92) - : : +- ReusedExchange (95) - : +- ReusedExchange (97) - +- ReusedExchange (100) + : : : : +- ReusedExchange (29) + : : : +- BroadcastExchange (45) + : : : +- * ColumnarToRow (44) + : : : +- CometProject (43) + : : : +- CometBroadcastHashJoin (42) + : : : :- CometProject (40) + : : : : +- CometBroadcastHashJoin (39) + : : : : :- CometFilter (37) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (36) + : : : : +- ReusedExchange (38) + : : : +- ReusedExchange (41) + : : +- BroadcastExchange (57) + : : +- * BroadcastHashJoin LeftSemi BuildRight (56) + : : :- * ColumnarToRow (54) + : : : +- CometFilter (53) + : : : +- CometScan parquet spark_catalog.default.item (52) + : : +- ReusedExchange (55) + : +- ReusedExchange (60) + :- * Project (83) + : +- * Filter (82) + : +- * HashAggregate (81) + : +- Exchange (80) + : +- * HashAggregate (79) + : +- * Project (78) + : +- * BroadcastHashJoin Inner BuildRight (77) + : :- * Project (75) + : : +- * BroadcastHashJoin Inner BuildRight (74) + : : :- * BroadcastHashJoin LeftSemi BuildRight (72) + : : : :- * ColumnarToRow (70) + : : : : +- CometFilter (69) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (68) + : : : +- ReusedExchange (71) + : : +- ReusedExchange (73) + : +- ReusedExchange (76) + +- * Project (99) + +- * Filter (98) + +- * HashAggregate (97) + +- Exchange (96) + +- * HashAggregate (95) + +- * Project (94) + +- * BroadcastHashJoin Inner BuildRight (93) + :- * Project (91) + : +- * BroadcastHashJoin Inner BuildRight (90) + : :- * BroadcastHashJoin LeftSemi BuildRight (88) + : : :- * ColumnarToRow (86) + : : : +- CometFilter (85) + : : : +- CometScan parquet spark_catalog.default.web_sales (84) + : : +- ReusedExchange (87) + : +- ReusedExchange (89) + +- ReusedExchange (92) (1) Scan parquet spark_catalog.default.store_sales @@ -252,646 +244,545 @@ Arguments: [ss_item_sk#10], [i_item_sk#13], Inner Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -(29) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : (((isnotnull(d_year#27) AND (d_year#27 >= 1999)) AND (d_year#27 <= 2001)) AND isnotnull(d_date_sk#26)) - -(31) CometProject -Input [2]: [d_date_sk#26, d_year#27] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(32) CometBroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: [d_date_sk#26] +(29) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#26] -(33) CometBroadcastHashJoin +(30) CometBroadcastHashJoin Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] Right output [1]: [d_date_sk#26] Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner -(34) CometProject +(31) CometProject Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] -Arguments: [brand_id#28, class_id#29, category_id#30], [i_brand_id#14 AS brand_id#28, i_class_id#15 AS class_id#29, i_category_id#16 AS category_id#30] +Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] -(35) CometHashAggregate -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +(32) CometHashAggregate +Input [3]: [brand_id#27, class_id#28, category_id#29] +Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] -(36) ColumnarToRow [codegen id : 1] -Input [3]: [brand_id#28, class_id#29, category_id#30] +(33) ColumnarToRow [codegen id : 1] +Input [3]: [brand_id#27, class_id#28, category_id#29] -(37) Exchange -Input [3]: [brand_id#28, class_id#29, category_id#30] -Arguments: hashpartitioning(brand_id#28, class_id#29, category_id#30, 5), ENSURE_REQUIREMENTS, [plan_id=1] +(34) Exchange +Input [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(38) HashAggregate [codegen id : 3] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +(35) HashAggregate [codegen id : 3] +Input [3]: [brand_id#27, class_id#28, category_id#29] +Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#27, class_id#28, category_id#29] -(39) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#31, ws_sold_date_sk#32] +(36) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#32), dynamicpruningexpression(ws_sold_date_sk#32 IN dynamicpruning#33)] +PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(40) CometFilter -Input [2]: [ws_item_sk#31, ws_sold_date_sk#32] -Condition : isnotnull(ws_item_sk#31) - -(41) Scan parquet spark_catalog.default.item -Output [4]: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(42) CometFilter -Input [4]: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] -Condition : isnotnull(i_item_sk#34) - -(43) CometBroadcastExchange -Input [4]: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] -Arguments: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] +(37) CometFilter +Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] +Condition : isnotnull(ws_item_sk#30) -(44) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#31, ws_sold_date_sk#32] -Right output [4]: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] -Arguments: [ws_item_sk#31], [i_item_sk#34], Inner - -(45) CometProject -Input [6]: [ws_item_sk#31, ws_sold_date_sk#32, i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] -Arguments: [ws_sold_date_sk#32, i_brand_id#35, i_class_id#36, i_category_id#37], [ws_sold_date_sk#32, i_brand_id#35, i_class_id#36, i_category_id#37] - -(46) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#38, d_year#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct +(38) ReusedExchange [Reuses operator id: 15] +Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -(47) CometFilter -Input [2]: [d_date_sk#38, d_year#39] -Condition : (((isnotnull(d_year#39) AND (d_year#39 >= 1999)) AND (d_year#39 <= 2001)) AND isnotnull(d_date_sk#38)) +(39) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] +Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [ws_item_sk#30], [i_item_sk#33], Inner -(48) CometProject -Input [2]: [d_date_sk#38, d_year#39] -Arguments: [d_date_sk#38], [d_date_sk#38] +(40) CometProject +Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -(49) CometBroadcastExchange -Input [1]: [d_date_sk#38] -Arguments: [d_date_sk#38] +(41) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#37] -(50) CometBroadcastHashJoin -Left output [4]: [ws_sold_date_sk#32, i_brand_id#35, i_class_id#36, i_category_id#37] -Right output [1]: [d_date_sk#38] -Arguments: [ws_sold_date_sk#32], [d_date_sk#38], Inner +(42) CometBroadcastHashJoin +Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] +Right output [1]: [d_date_sk#37] +Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner -(51) CometProject -Input [5]: [ws_sold_date_sk#32, i_brand_id#35, i_class_id#36, i_category_id#37, d_date_sk#38] -Arguments: [i_brand_id#35, i_class_id#36, i_category_id#37], [i_brand_id#35, i_class_id#36, i_category_id#37] +(43) CometProject +Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] +Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] -(52) ColumnarToRow [codegen id : 2] -Input [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] +(44) ColumnarToRow [codegen id : 2] +Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -(53) BroadcastExchange -Input [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] +(45) BroadcastExchange +Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] -(54) BroadcastHashJoin [codegen id : 3] -Left keys [6]: [coalesce(brand_id#28, 0), isnull(brand_id#28), coalesce(class_id#29, 0), isnull(class_id#29), coalesce(category_id#30, 0), isnull(category_id#30)] -Right keys [6]: [coalesce(i_brand_id#35, 0), isnull(i_brand_id#35), coalesce(i_class_id#36, 0), isnull(i_class_id#36), coalesce(i_category_id#37, 0), isnull(i_category_id#37)] +(46) BroadcastHashJoin [codegen id : 3] +Left keys [6]: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)] +Right keys [6]: [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)] Join type: LeftSemi Join condition: None -(55) BroadcastExchange -Input [3]: [brand_id#28, class_id#29, category_id#30] +(47) BroadcastExchange +Input [3]: [brand_id#27, class_id#28, category_id#29] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=3] -(56) BroadcastHashJoin [codegen id : 4] +(48) BroadcastHashJoin [codegen id : 4] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#28, class_id#29, category_id#30] +Right keys [3]: [brand_id#27, class_id#28, category_id#29] Join type: Inner Join condition: None -(57) Project [codegen id : 4] -Output [1]: [i_item_sk#6 AS ss_item_sk#40] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#28, class_id#29, category_id#30] +(49) Project [codegen id : 4] +Output [1]: [i_item_sk#6 AS ss_item_sk#38] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] -(58) BroadcastExchange -Input [1]: [ss_item_sk#40] +(50) BroadcastExchange +Input [1]: [ss_item_sk#38] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(59) BroadcastHashJoin [codegen id : 11] +(51) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#40] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None -(60) Scan parquet spark_catalog.default.item -Output [4]: [i_item_sk#41, i_brand_id#42, i_class_id#43, i_category_id#44] +(52) Scan parquet spark_catalog.default.item +Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(61) CometFilter -Input [4]: [i_item_sk#41, i_brand_id#42, i_class_id#43, i_category_id#44] -Condition : isnotnull(i_item_sk#41) +(53) CometFilter +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Condition : isnotnull(i_item_sk#39) -(62) ColumnarToRow [codegen id : 9] -Input [4]: [i_item_sk#41, i_brand_id#42, i_class_id#43, i_category_id#44] +(54) ColumnarToRow [codegen id : 9] +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -(63) ReusedExchange [Reuses operator id: 58] -Output [1]: [ss_item_sk#40] +(55) ReusedExchange [Reuses operator id: 50] +Output [1]: [ss_item_sk#38] -(64) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [i_item_sk#41] -Right keys [1]: [ss_item_sk#40] +(56) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [i_item_sk#39] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None -(65) BroadcastExchange -Input [4]: [i_item_sk#41, i_brand_id#42, i_class_id#43, i_category_id#44] +(57) BroadcastExchange +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(66) BroadcastHashJoin [codegen id : 11] +(58) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#41] +Right keys [1]: [i_item_sk#39] Join type: Inner Join condition: None -(67) Project [codegen id : 11] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#42, i_class_id#43, i_category_id#44] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#41, i_brand_id#42, i_class_id#43, i_category_id#44] +(59) Project [codegen id : 11] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -(68) ReusedExchange [Reuses operator id: 144] -Output [1]: [d_date_sk#45] +(60) ReusedExchange [Reuses operator id: 127] +Output [1]: [d_date_sk#43] -(69) BroadcastHashJoin [codegen id : 11] +(61) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#45] +Right keys [1]: [d_date_sk#43] Join type: Inner Join condition: None -(70) Project [codegen id : 11] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#42, i_class_id#43, i_category_id#44] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#42, i_class_id#43, i_category_id#44, d_date_sk#45] +(62) Project [codegen id : 11] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -(71) HashAggregate [codegen id : 11] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#42, i_class_id#43, i_category_id#44] -Keys [3]: [i_brand_id#42, i_class_id#43, i_category_id#44] +(63) HashAggregate [codegen id : 11] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] +Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] -Aggregate Attributes [3]: [sum#46, isEmpty#47, count#48] -Results [6]: [i_brand_id#42, i_class_id#43, i_category_id#44, sum#49, isEmpty#50, count#51] +Aggregate Attributes [3]: [sum#44, isEmpty#45, count#46] +Results [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] -(72) Exchange -Input [6]: [i_brand_id#42, i_class_id#43, i_category_id#44, sum#49, isEmpty#50, count#51] -Arguments: hashpartitioning(i_brand_id#42, i_class_id#43, i_category_id#44, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(64) Exchange +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] +Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(73) HashAggregate [codegen id : 12] -Input [6]: [i_brand_id#42, i_class_id#43, i_category_id#44, sum#49, isEmpty#50, count#51] -Keys [3]: [i_brand_id#42, i_class_id#43, i_category_id#44] +(65) HashAggregate [codegen id : 12] +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] +Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#52, count(1)#53] -Results [5]: [i_brand_id#42, i_class_id#43, i_category_id#44, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#52 AS sales#54, count(1)#53 AS number_sales#55] +Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#50, count(1)#51] +Results [5]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#50 AS sales#52, count(1)#51 AS number_sales#53] -(74) Filter [codegen id : 12] -Input [5]: [i_brand_id#42, i_class_id#43, i_category_id#44, sales#54, number_sales#55] -Condition : (isnotnull(sales#54) AND (cast(sales#54 as decimal(32,6)) > cast(Subquery scalar-subquery#56, [id=#57] as decimal(32,6)))) +(66) Filter [codegen id : 12] +Input [5]: [i_brand_id#40, i_class_id#41, i_category_id#42, sales#52, number_sales#53] +Condition : (isnotnull(sales#52) AND (cast(sales#52 as decimal(32,6)) > cast(Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) -(75) Project [codegen id : 12] -Output [6]: [sales#54, number_sales#55, store AS channel#58, i_brand_id#42 AS i_brand_id#59, i_class_id#43 AS i_class_id#60, i_category_id#44 AS i_category_id#61] -Input [5]: [i_brand_id#42, i_class_id#43, i_category_id#44, sales#54, number_sales#55] +(67) Project [codegen id : 12] +Output [6]: [sales#52, number_sales#53, store AS channel#56, i_brand_id#40 AS i_brand_id#57, i_class_id#41 AS i_class_id#58, i_category_id#42 AS i_category_id#59] +Input [5]: [i_brand_id#40, i_class_id#41, i_category_id#42, sales#52, number_sales#53] -(76) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#62, cs_quantity#63, cs_list_price#64, cs_sold_date_sk#65] +(68) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_item_sk#60, cs_quantity#61, cs_list_price#62, cs_sold_date_sk#63] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#65), dynamicpruningexpression(cs_sold_date_sk#65 IN dynamicpruning#66)] +PartitionFilters: [isnotnull(cs_sold_date_sk#63), dynamicpruningexpression(cs_sold_date_sk#63 IN dynamicpruning#64)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(77) CometFilter -Input [4]: [cs_item_sk#62, cs_quantity#63, cs_list_price#64, cs_sold_date_sk#65] -Condition : isnotnull(cs_item_sk#62) +(69) CometFilter +Input [4]: [cs_item_sk#60, cs_quantity#61, cs_list_price#62, cs_sold_date_sk#63] +Condition : isnotnull(cs_item_sk#60) -(78) ColumnarToRow [codegen id : 23] -Input [4]: [cs_item_sk#62, cs_quantity#63, cs_list_price#64, cs_sold_date_sk#65] +(70) ColumnarToRow [codegen id : 23] +Input [4]: [cs_item_sk#60, cs_quantity#61, cs_list_price#62, cs_sold_date_sk#63] -(79) ReusedExchange [Reuses operator id: 58] -Output [1]: [ss_item_sk#40] +(71) ReusedExchange [Reuses operator id: 50] +Output [1]: [ss_item_sk#38] -(80) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_item_sk#62] -Right keys [1]: [ss_item_sk#40] +(72) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [cs_item_sk#60] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None -(81) ReusedExchange [Reuses operator id: 65] -Output [4]: [i_item_sk#67, i_brand_id#68, i_class_id#69, i_category_id#70] +(73) ReusedExchange [Reuses operator id: 57] +Output [4]: [i_item_sk#65, i_brand_id#66, i_class_id#67, i_category_id#68] -(82) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_item_sk#62] -Right keys [1]: [i_item_sk#67] +(74) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [cs_item_sk#60] +Right keys [1]: [i_item_sk#65] Join type: Inner Join condition: None -(83) Project [codegen id : 23] -Output [6]: [cs_quantity#63, cs_list_price#64, cs_sold_date_sk#65, i_brand_id#68, i_class_id#69, i_category_id#70] -Input [8]: [cs_item_sk#62, cs_quantity#63, cs_list_price#64, cs_sold_date_sk#65, i_item_sk#67, i_brand_id#68, i_class_id#69, i_category_id#70] +(75) Project [codegen id : 23] +Output [6]: [cs_quantity#61, cs_list_price#62, cs_sold_date_sk#63, i_brand_id#66, i_class_id#67, i_category_id#68] +Input [8]: [cs_item_sk#60, cs_quantity#61, cs_list_price#62, cs_sold_date_sk#63, i_item_sk#65, i_brand_id#66, i_class_id#67, i_category_id#68] -(84) ReusedExchange [Reuses operator id: 144] -Output [1]: [d_date_sk#71] +(76) ReusedExchange [Reuses operator id: 127] +Output [1]: [d_date_sk#69] -(85) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_sold_date_sk#65] -Right keys [1]: [d_date_sk#71] +(77) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [cs_sold_date_sk#63] +Right keys [1]: [d_date_sk#69] Join type: Inner Join condition: None -(86) Project [codegen id : 23] -Output [5]: [cs_quantity#63, cs_list_price#64, i_brand_id#68, i_class_id#69, i_category_id#70] -Input [7]: [cs_quantity#63, cs_list_price#64, cs_sold_date_sk#65, i_brand_id#68, i_class_id#69, i_category_id#70, d_date_sk#71] - -(87) HashAggregate [codegen id : 23] -Input [5]: [cs_quantity#63, cs_list_price#64, i_brand_id#68, i_class_id#69, i_category_id#70] -Keys [3]: [i_brand_id#68, i_class_id#69, i_category_id#70] -Functions [2]: [partial_sum((cast(cs_quantity#63 as decimal(10,0)) * cs_list_price#64)), partial_count(1)] -Aggregate Attributes [3]: [sum#72, isEmpty#73, count#74] -Results [6]: [i_brand_id#68, i_class_id#69, i_category_id#70, sum#75, isEmpty#76, count#77] - -(88) Exchange -Input [6]: [i_brand_id#68, i_class_id#69, i_category_id#70, sum#75, isEmpty#76, count#77] -Arguments: hashpartitioning(i_brand_id#68, i_class_id#69, i_category_id#70, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(89) HashAggregate [codegen id : 24] -Input [6]: [i_brand_id#68, i_class_id#69, i_category_id#70, sum#75, isEmpty#76, count#77] -Keys [3]: [i_brand_id#68, i_class_id#69, i_category_id#70] -Functions [2]: [sum((cast(cs_quantity#63 as decimal(10,0)) * cs_list_price#64)), count(1)] -Aggregate Attributes [2]: [sum((cast(cs_quantity#63 as decimal(10,0)) * cs_list_price#64))#78, count(1)#79] -Results [5]: [i_brand_id#68, i_class_id#69, i_category_id#70, sum((cast(cs_quantity#63 as decimal(10,0)) * cs_list_price#64))#78 AS sales#80, count(1)#79 AS number_sales#81] - -(90) Filter [codegen id : 24] -Input [5]: [i_brand_id#68, i_class_id#69, i_category_id#70, sales#80, number_sales#81] -Condition : (isnotnull(sales#80) AND (cast(sales#80 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#56, [id=#57] as decimal(32,6)))) - -(91) Project [codegen id : 24] -Output [6]: [sales#80, number_sales#81, catalog AS channel#82, i_brand_id#68, i_class_id#69, i_category_id#70] -Input [5]: [i_brand_id#68, i_class_id#69, i_category_id#70, sales#80, number_sales#81] - -(92) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#83, ws_quantity#84, ws_list_price#85, ws_sold_date_sk#86] +(78) Project [codegen id : 23] +Output [5]: [cs_quantity#61, cs_list_price#62, i_brand_id#66, i_class_id#67, i_category_id#68] +Input [7]: [cs_quantity#61, cs_list_price#62, cs_sold_date_sk#63, i_brand_id#66, i_class_id#67, i_category_id#68, d_date_sk#69] + +(79) HashAggregate [codegen id : 23] +Input [5]: [cs_quantity#61, cs_list_price#62, i_brand_id#66, i_class_id#67, i_category_id#68] +Keys [3]: [i_brand_id#66, i_class_id#67, i_category_id#68] +Functions [2]: [partial_sum((cast(cs_quantity#61 as decimal(10,0)) * cs_list_price#62)), partial_count(1)] +Aggregate Attributes [3]: [sum#70, isEmpty#71, count#72] +Results [6]: [i_brand_id#66, i_class_id#67, i_category_id#68, sum#73, isEmpty#74, count#75] + +(80) Exchange +Input [6]: [i_brand_id#66, i_class_id#67, i_category_id#68, sum#73, isEmpty#74, count#75] +Arguments: hashpartitioning(i_brand_id#66, i_class_id#67, i_category_id#68, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(81) HashAggregate [codegen id : 24] +Input [6]: [i_brand_id#66, i_class_id#67, i_category_id#68, sum#73, isEmpty#74, count#75] +Keys [3]: [i_brand_id#66, i_class_id#67, i_category_id#68] +Functions [2]: [sum((cast(cs_quantity#61 as decimal(10,0)) * cs_list_price#62)), count(1)] +Aggregate Attributes [2]: [sum((cast(cs_quantity#61 as decimal(10,0)) * cs_list_price#62))#76, count(1)#77] +Results [5]: [i_brand_id#66, i_class_id#67, i_category_id#68, sum((cast(cs_quantity#61 as decimal(10,0)) * cs_list_price#62))#76 AS sales#78, count(1)#77 AS number_sales#79] + +(82) Filter [codegen id : 24] +Input [5]: [i_brand_id#66, i_class_id#67, i_category_id#68, sales#78, number_sales#79] +Condition : (isnotnull(sales#78) AND (cast(sales#78 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) + +(83) Project [codegen id : 24] +Output [6]: [sales#78, number_sales#79, catalog AS channel#80, i_brand_id#66, i_class_id#67, i_category_id#68] +Input [5]: [i_brand_id#66, i_class_id#67, i_category_id#68, sales#78, number_sales#79] + +(84) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#81, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#84] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#86), dynamicpruningexpression(ws_sold_date_sk#86 IN dynamicpruning#87)] +PartitionFilters: [isnotnull(ws_sold_date_sk#84), dynamicpruningexpression(ws_sold_date_sk#84 IN dynamicpruning#85)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(93) CometFilter -Input [4]: [ws_item_sk#83, ws_quantity#84, ws_list_price#85, ws_sold_date_sk#86] -Condition : isnotnull(ws_item_sk#83) +(85) CometFilter +Input [4]: [ws_item_sk#81, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#84] +Condition : isnotnull(ws_item_sk#81) -(94) ColumnarToRow [codegen id : 35] -Input [4]: [ws_item_sk#83, ws_quantity#84, ws_list_price#85, ws_sold_date_sk#86] +(86) ColumnarToRow [codegen id : 35] +Input [4]: [ws_item_sk#81, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#84] -(95) ReusedExchange [Reuses operator id: 58] -Output [1]: [ss_item_sk#40] +(87) ReusedExchange [Reuses operator id: 50] +Output [1]: [ss_item_sk#38] -(96) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ws_item_sk#83] -Right keys [1]: [ss_item_sk#40] +(88) BroadcastHashJoin [codegen id : 35] +Left keys [1]: [ws_item_sk#81] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None -(97) ReusedExchange [Reuses operator id: 65] -Output [4]: [i_item_sk#88, i_brand_id#89, i_class_id#90, i_category_id#91] +(89) ReusedExchange [Reuses operator id: 57] +Output [4]: [i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89] -(98) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ws_item_sk#83] -Right keys [1]: [i_item_sk#88] +(90) BroadcastHashJoin [codegen id : 35] +Left keys [1]: [ws_item_sk#81] +Right keys [1]: [i_item_sk#86] Join type: Inner Join condition: None -(99) Project [codegen id : 35] -Output [6]: [ws_quantity#84, ws_list_price#85, ws_sold_date_sk#86, i_brand_id#89, i_class_id#90, i_category_id#91] -Input [8]: [ws_item_sk#83, ws_quantity#84, ws_list_price#85, ws_sold_date_sk#86, i_item_sk#88, i_brand_id#89, i_class_id#90, i_category_id#91] +(91) Project [codegen id : 35] +Output [6]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#84, i_brand_id#87, i_class_id#88, i_category_id#89] +Input [8]: [ws_item_sk#81, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#84, i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89] -(100) ReusedExchange [Reuses operator id: 144] -Output [1]: [d_date_sk#92] +(92) ReusedExchange [Reuses operator id: 127] +Output [1]: [d_date_sk#90] -(101) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ws_sold_date_sk#86] -Right keys [1]: [d_date_sk#92] +(93) BroadcastHashJoin [codegen id : 35] +Left keys [1]: [ws_sold_date_sk#84] +Right keys [1]: [d_date_sk#90] Join type: Inner Join condition: None -(102) Project [codegen id : 35] -Output [5]: [ws_quantity#84, ws_list_price#85, i_brand_id#89, i_class_id#90, i_category_id#91] -Input [7]: [ws_quantity#84, ws_list_price#85, ws_sold_date_sk#86, i_brand_id#89, i_class_id#90, i_category_id#91, d_date_sk#92] - -(103) HashAggregate [codegen id : 35] -Input [5]: [ws_quantity#84, ws_list_price#85, i_brand_id#89, i_class_id#90, i_category_id#91] -Keys [3]: [i_brand_id#89, i_class_id#90, i_category_id#91] -Functions [2]: [partial_sum((cast(ws_quantity#84 as decimal(10,0)) * ws_list_price#85)), partial_count(1)] -Aggregate Attributes [3]: [sum#93, isEmpty#94, count#95] -Results [6]: [i_brand_id#89, i_class_id#90, i_category_id#91, sum#96, isEmpty#97, count#98] - -(104) Exchange -Input [6]: [i_brand_id#89, i_class_id#90, i_category_id#91, sum#96, isEmpty#97, count#98] -Arguments: hashpartitioning(i_brand_id#89, i_class_id#90, i_category_id#91, 5), ENSURE_REQUIREMENTS, [plan_id=8] - -(105) HashAggregate [codegen id : 36] -Input [6]: [i_brand_id#89, i_class_id#90, i_category_id#91, sum#96, isEmpty#97, count#98] -Keys [3]: [i_brand_id#89, i_class_id#90, i_category_id#91] -Functions [2]: [sum((cast(ws_quantity#84 as decimal(10,0)) * ws_list_price#85)), count(1)] -Aggregate Attributes [2]: [sum((cast(ws_quantity#84 as decimal(10,0)) * ws_list_price#85))#99, count(1)#100] -Results [5]: [i_brand_id#89, i_class_id#90, i_category_id#91, sum((cast(ws_quantity#84 as decimal(10,0)) * ws_list_price#85))#99 AS sales#101, count(1)#100 AS number_sales#102] - -(106) Filter [codegen id : 36] -Input [5]: [i_brand_id#89, i_class_id#90, i_category_id#91, sales#101, number_sales#102] -Condition : (isnotnull(sales#101) AND (cast(sales#101 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#56, [id=#57] as decimal(32,6)))) - -(107) Project [codegen id : 36] -Output [6]: [sales#101, number_sales#102, web AS channel#103, i_brand_id#89, i_class_id#90, i_category_id#91] -Input [5]: [i_brand_id#89, i_class_id#90, i_category_id#91, sales#101, number_sales#102] - -(108) Union - -(109) Expand [codegen id : 37] -Input [6]: [sales#54, number_sales#55, channel#58, i_brand_id#59, i_class_id#60, i_category_id#61] -Arguments: [[sales#54, number_sales#55, channel#58, i_brand_id#59, i_class_id#60, i_category_id#61, 0], [sales#54, number_sales#55, channel#58, i_brand_id#59, i_class_id#60, null, 1], [sales#54, number_sales#55, channel#58, i_brand_id#59, null, null, 3], [sales#54, number_sales#55, channel#58, null, null, null, 7], [sales#54, number_sales#55, null, null, null, null, 15]], [sales#54, number_sales#55, channel#104, i_brand_id#105, i_class_id#106, i_category_id#107, spark_grouping_id#108] - -(110) HashAggregate [codegen id : 37] -Input [7]: [sales#54, number_sales#55, channel#104, i_brand_id#105, i_class_id#106, i_category_id#107, spark_grouping_id#108] -Keys [5]: [channel#104, i_brand_id#105, i_class_id#106, i_category_id#107, spark_grouping_id#108] -Functions [2]: [partial_sum(sales#54), partial_sum(number_sales#55)] -Aggregate Attributes [3]: [sum#109, isEmpty#110, sum#111] -Results [8]: [channel#104, i_brand_id#105, i_class_id#106, i_category_id#107, spark_grouping_id#108, sum#112, isEmpty#113, sum#114] - -(111) Exchange -Input [8]: [channel#104, i_brand_id#105, i_class_id#106, i_category_id#107, spark_grouping_id#108, sum#112, isEmpty#113, sum#114] -Arguments: hashpartitioning(channel#104, i_brand_id#105, i_class_id#106, i_category_id#107, spark_grouping_id#108, 5), ENSURE_REQUIREMENTS, [plan_id=9] - -(112) HashAggregate [codegen id : 38] -Input [8]: [channel#104, i_brand_id#105, i_class_id#106, i_category_id#107, spark_grouping_id#108, sum#112, isEmpty#113, sum#114] -Keys [5]: [channel#104, i_brand_id#105, i_class_id#106, i_category_id#107, spark_grouping_id#108] -Functions [2]: [sum(sales#54), sum(number_sales#55)] -Aggregate Attributes [2]: [sum(sales#54)#115, sum(number_sales#55)#116] -Results [6]: [channel#104, i_brand_id#105, i_class_id#106, i_category_id#107, sum(sales#54)#115 AS sum(sales)#117, sum(number_sales#55)#116 AS sum(number_sales)#118] - -(113) TakeOrderedAndProject -Input [6]: [channel#104, i_brand_id#105, i_class_id#106, i_category_id#107, sum(sales)#117, sum(number_sales)#118] -Arguments: 100, [channel#104 ASC NULLS FIRST, i_brand_id#105 ASC NULLS FIRST, i_class_id#106 ASC NULLS FIRST, i_category_id#107 ASC NULLS FIRST], [channel#104, i_brand_id#105, i_class_id#106, i_category_id#107, sum(sales)#117, sum(number_sales)#118] +(94) Project [codegen id : 35] +Output [5]: [ws_quantity#82, ws_list_price#83, i_brand_id#87, i_class_id#88, i_category_id#89] +Input [7]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#84, i_brand_id#87, i_class_id#88, i_category_id#89, d_date_sk#90] + +(95) HashAggregate [codegen id : 35] +Input [5]: [ws_quantity#82, ws_list_price#83, i_brand_id#87, i_class_id#88, i_category_id#89] +Keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] +Functions [2]: [partial_sum((cast(ws_quantity#82 as decimal(10,0)) * ws_list_price#83)), partial_count(1)] +Aggregate Attributes [3]: [sum#91, isEmpty#92, count#93] +Results [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#94, isEmpty#95, count#96] + +(96) Exchange +Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#94, isEmpty#95, count#96] +Arguments: hashpartitioning(i_brand_id#87, i_class_id#88, i_category_id#89, 5), ENSURE_REQUIREMENTS, [plan_id=8] + +(97) HashAggregate [codegen id : 36] +Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#94, isEmpty#95, count#96] +Keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] +Functions [2]: [sum((cast(ws_quantity#82 as decimal(10,0)) * ws_list_price#83)), count(1)] +Aggregate Attributes [2]: [sum((cast(ws_quantity#82 as decimal(10,0)) * ws_list_price#83))#97, count(1)#98] +Results [5]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum((cast(ws_quantity#82 as decimal(10,0)) * ws_list_price#83))#97 AS sales#99, count(1)#98 AS number_sales#100] + +(98) Filter [codegen id : 36] +Input [5]: [i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] +Condition : (isnotnull(sales#99) AND (cast(sales#99 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) + +(99) Project [codegen id : 36] +Output [6]: [sales#99, number_sales#100, web AS channel#101, i_brand_id#87, i_class_id#88, i_category_id#89] +Input [5]: [i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] + +(100) Union + +(101) Expand [codegen id : 37] +Input [6]: [sales#52, number_sales#53, channel#56, i_brand_id#57, i_class_id#58, i_category_id#59] +Arguments: [[sales#52, number_sales#53, channel#56, i_brand_id#57, i_class_id#58, i_category_id#59, 0], [sales#52, number_sales#53, channel#56, i_brand_id#57, i_class_id#58, null, 1], [sales#52, number_sales#53, channel#56, i_brand_id#57, null, null, 3], [sales#52, number_sales#53, channel#56, null, null, null, 7], [sales#52, number_sales#53, null, null, null, null, 15]], [sales#52, number_sales#53, channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106] + +(102) HashAggregate [codegen id : 37] +Input [7]: [sales#52, number_sales#53, channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106] +Keys [5]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106] +Functions [2]: [partial_sum(sales#52), partial_sum(number_sales#53)] +Aggregate Attributes [3]: [sum#107, isEmpty#108, sum#109] +Results [8]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106, sum#110, isEmpty#111, sum#112] + +(103) Exchange +Input [8]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106, sum#110, isEmpty#111, sum#112] +Arguments: hashpartitioning(channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106, 5), ENSURE_REQUIREMENTS, [plan_id=9] + +(104) HashAggregate [codegen id : 38] +Input [8]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106, sum#110, isEmpty#111, sum#112] +Keys [5]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106] +Functions [2]: [sum(sales#52), sum(number_sales#53)] +Aggregate Attributes [2]: [sum(sales#52)#113, sum(number_sales#53)#114] +Results [6]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, sum(sales#52)#113 AS sum(sales)#115, sum(number_sales#53)#114 AS sum(number_sales)#116] + +(105) TakeOrderedAndProject +Input [6]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, sum(sales)#115, sum(number_sales)#116] +Arguments: 100, [channel#102 ASC NULLS FIRST, i_brand_id#103 ASC NULLS FIRST, i_class_id#104 ASC NULLS FIRST, i_category_id#105 ASC NULLS FIRST], [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, sum(sales)#115, sum(number_sales)#116] ===== Subqueries ===== -Subquery:1 Hosting operator id = 74 Hosting Expression = Subquery scalar-subquery#56, [id=#57] -* HashAggregate (139) -+- Exchange (138) - +- * ColumnarToRow (137) - +- CometHashAggregate (136) - +- CometUnion (135) - :- CometProject (120) - : +- CometBroadcastHashJoin (119) - : :- CometScan parquet spark_catalog.default.store_sales (114) - : +- CometBroadcastExchange (118) - : +- CometProject (117) - : +- CometFilter (116) - : +- CometScan parquet spark_catalog.default.date_dim (115) - :- CometProject (127) - : +- CometBroadcastHashJoin (126) - : :- CometScan parquet spark_catalog.default.catalog_sales (121) - : +- CometBroadcastExchange (125) - : +- CometProject (124) - : +- CometFilter (123) - : +- CometScan parquet spark_catalog.default.date_dim (122) - +- CometProject (134) - +- CometBroadcastHashJoin (133) - :- CometScan parquet spark_catalog.default.web_sales (128) - +- CometBroadcastExchange (132) - +- CometProject (131) - +- CometFilter (130) - +- CometScan parquet spark_catalog.default.date_dim (129) - - -(114) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#119, ss_list_price#120, ss_sold_date_sk#121] +Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#54, [id=#55] +* HashAggregate (122) ++- Exchange (121) + +- * ColumnarToRow (120) + +- CometHashAggregate (119) + +- CometUnion (118) + :- CometProject (109) + : +- CometBroadcastHashJoin (108) + : :- CometScan parquet spark_catalog.default.store_sales (106) + : +- ReusedExchange (107) + :- CometProject (113) + : +- CometBroadcastHashJoin (112) + : :- CometScan parquet spark_catalog.default.catalog_sales (110) + : +- ReusedExchange (111) + +- CometProject (117) + +- CometBroadcastHashJoin (116) + :- CometScan parquet spark_catalog.default.web_sales (114) + +- ReusedExchange (115) + + +(106) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_quantity#117, ss_list_price#118, ss_sold_date_sk#119] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#121), dynamicpruningexpression(ss_sold_date_sk#121 IN dynamicpruning#122)] +PartitionFilters: [isnotnull(ss_sold_date_sk#119), dynamicpruningexpression(ss_sold_date_sk#119 IN dynamicpruning#120)] ReadSchema: struct -(115) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#123, d_year#124] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(116) CometFilter -Input [2]: [d_date_sk#123, d_year#124] -Condition : (((isnotnull(d_year#124) AND (d_year#124 >= 1999)) AND (d_year#124 <= 2001)) AND isnotnull(d_date_sk#123)) - -(117) CometProject -Input [2]: [d_date_sk#123, d_year#124] -Arguments: [d_date_sk#123], [d_date_sk#123] - -(118) CometBroadcastExchange -Input [1]: [d_date_sk#123] -Arguments: [d_date_sk#123] +(107) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#121] -(119) CometBroadcastHashJoin -Left output [3]: [ss_quantity#119, ss_list_price#120, ss_sold_date_sk#121] -Right output [1]: [d_date_sk#123] -Arguments: [ss_sold_date_sk#121], [d_date_sk#123], Inner +(108) CometBroadcastHashJoin +Left output [3]: [ss_quantity#117, ss_list_price#118, ss_sold_date_sk#119] +Right output [1]: [d_date_sk#121] +Arguments: [ss_sold_date_sk#119], [d_date_sk#121], Inner -(120) CometProject -Input [4]: [ss_quantity#119, ss_list_price#120, ss_sold_date_sk#121, d_date_sk#123] -Arguments: [quantity#125, list_price#126], [ss_quantity#119 AS quantity#125, ss_list_price#120 AS list_price#126] +(109) CometProject +Input [4]: [ss_quantity#117, ss_list_price#118, ss_sold_date_sk#119, d_date_sk#121] +Arguments: [quantity#122, list_price#123], [ss_quantity#117 AS quantity#122, ss_list_price#118 AS list_price#123] -(121) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#127, cs_list_price#128, cs_sold_date_sk#129] +(110) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#124, cs_list_price#125, cs_sold_date_sk#126] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#129), dynamicpruningexpression(cs_sold_date_sk#129 IN dynamicpruning#130)] +PartitionFilters: [isnotnull(cs_sold_date_sk#126), dynamicpruningexpression(cs_sold_date_sk#126 IN dynamicpruning#127)] ReadSchema: struct -(122) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#131, d_year#132] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct +(111) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#128] -(123) CometFilter -Input [2]: [d_date_sk#131, d_year#132] -Condition : (((isnotnull(d_year#132) AND (d_year#132 >= 1999)) AND (d_year#132 <= 2001)) AND isnotnull(d_date_sk#131)) +(112) CometBroadcastHashJoin +Left output [3]: [cs_quantity#124, cs_list_price#125, cs_sold_date_sk#126] +Right output [1]: [d_date_sk#128] +Arguments: [cs_sold_date_sk#126], [d_date_sk#128], Inner -(124) CometProject -Input [2]: [d_date_sk#131, d_year#132] -Arguments: [d_date_sk#131], [d_date_sk#131] +(113) CometProject +Input [4]: [cs_quantity#124, cs_list_price#125, cs_sold_date_sk#126, d_date_sk#128] +Arguments: [quantity#129, list_price#130], [cs_quantity#124 AS quantity#129, cs_list_price#125 AS list_price#130] -(125) CometBroadcastExchange -Input [1]: [d_date_sk#131] -Arguments: [d_date_sk#131] - -(126) CometBroadcastHashJoin -Left output [3]: [cs_quantity#127, cs_list_price#128, cs_sold_date_sk#129] -Right output [1]: [d_date_sk#131] -Arguments: [cs_sold_date_sk#129], [d_date_sk#131], Inner - -(127) CometProject -Input [4]: [cs_quantity#127, cs_list_price#128, cs_sold_date_sk#129, d_date_sk#131] -Arguments: [quantity#133, list_price#134], [cs_quantity#127 AS quantity#133, cs_list_price#128 AS list_price#134] - -(128) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#135, ws_list_price#136, ws_sold_date_sk#137] +(114) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#131, ws_list_price#132, ws_sold_date_sk#133] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#137), dynamicpruningexpression(ws_sold_date_sk#137 IN dynamicpruning#138)] +PartitionFilters: [isnotnull(ws_sold_date_sk#133), dynamicpruningexpression(ws_sold_date_sk#133 IN dynamicpruning#134)] ReadSchema: struct -(129) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#139, d_year#140] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(130) CometFilter -Input [2]: [d_date_sk#139, d_year#140] -Condition : (((isnotnull(d_year#140) AND (d_year#140 >= 1999)) AND (d_year#140 <= 2001)) AND isnotnull(d_date_sk#139)) - -(131) CometProject -Input [2]: [d_date_sk#139, d_year#140] -Arguments: [d_date_sk#139], [d_date_sk#139] +(115) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#135] -(132) CometBroadcastExchange -Input [1]: [d_date_sk#139] -Arguments: [d_date_sk#139] +(116) CometBroadcastHashJoin +Left output [3]: [ws_quantity#131, ws_list_price#132, ws_sold_date_sk#133] +Right output [1]: [d_date_sk#135] +Arguments: [ws_sold_date_sk#133], [d_date_sk#135], Inner -(133) CometBroadcastHashJoin -Left output [3]: [ws_quantity#135, ws_list_price#136, ws_sold_date_sk#137] -Right output [1]: [d_date_sk#139] -Arguments: [ws_sold_date_sk#137], [d_date_sk#139], Inner - -(134) CometProject -Input [4]: [ws_quantity#135, ws_list_price#136, ws_sold_date_sk#137, d_date_sk#139] -Arguments: [quantity#141, list_price#142], [ws_quantity#135 AS quantity#141, ws_list_price#136 AS list_price#142] +(117) CometProject +Input [4]: [ws_quantity#131, ws_list_price#132, ws_sold_date_sk#133, d_date_sk#135] +Arguments: [quantity#136, list_price#137], [ws_quantity#131 AS quantity#136, ws_list_price#132 AS list_price#137] -(135) CometUnion -Child 0 Input [2]: [quantity#125, list_price#126] -Child 1 Input [2]: [quantity#133, list_price#134] -Child 2 Input [2]: [quantity#141, list_price#142] +(118) CometUnion +Child 0 Input [2]: [quantity#122, list_price#123] +Child 1 Input [2]: [quantity#129, list_price#130] +Child 2 Input [2]: [quantity#136, list_price#137] -(136) CometHashAggregate -Input [2]: [quantity#125, list_price#126] +(119) CometHashAggregate +Input [2]: [quantity#122, list_price#123] Keys: [] -Functions [1]: [partial_avg((cast(quantity#125 as decimal(10,0)) * list_price#126))] +Functions [1]: [partial_avg((cast(quantity#122 as decimal(10,0)) * list_price#123))] -(137) ColumnarToRow [codegen id : 1] -Input [2]: [sum#143, count#144] +(120) ColumnarToRow [codegen id : 1] +Input [2]: [sum#138, count#139] -(138) Exchange -Input [2]: [sum#143, count#144] +(121) Exchange +Input [2]: [sum#138, count#139] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] -(139) HashAggregate [codegen id : 2] -Input [2]: [sum#143, count#144] +(122) HashAggregate [codegen id : 2] +Input [2]: [sum#138, count#139] Keys: [] -Functions [1]: [avg((cast(quantity#125 as decimal(10,0)) * list_price#126))] -Aggregate Attributes [1]: [avg((cast(quantity#125 as decimal(10,0)) * list_price#126))#145] -Results [1]: [avg((cast(quantity#125 as decimal(10,0)) * list_price#126))#145 AS average_sales#146] +Functions [1]: [avg((cast(quantity#122 as decimal(10,0)) * list_price#123))] +Aggregate Attributes [1]: [avg((cast(quantity#122 as decimal(10,0)) * list_price#123))#140] +Results [1]: [avg((cast(quantity#122 as decimal(10,0)) * list_price#123))#140 AS average_sales#141] -Subquery:2 Hosting operator id = 114 Hosting Expression = ss_sold_date_sk#121 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#119 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 121 Hosting Expression = cs_sold_date_sk#129 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#126 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 128 Hosting Expression = ws_sold_date_sk#137 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 114 Hosting Expression = ws_sold_date_sk#133 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (144) -+- * ColumnarToRow (143) - +- CometProject (142) - +- CometFilter (141) - +- CometScan parquet spark_catalog.default.date_dim (140) +BroadcastExchange (127) ++- * ColumnarToRow (126) + +- CometProject (125) + +- CometFilter (124) + +- CometScan parquet spark_catalog.default.date_dim (123) -(140) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#45, d_year#147, d_moy#148] +(123) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#43, d_year#142, d_moy#143] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(141) CometFilter -Input [3]: [d_date_sk#45, d_year#147, d_moy#148] -Condition : ((((isnotnull(d_year#147) AND isnotnull(d_moy#148)) AND (d_year#147 = 2001)) AND (d_moy#148 = 11)) AND isnotnull(d_date_sk#45)) +(124) CometFilter +Input [3]: [d_date_sk#43, d_year#142, d_moy#143] +Condition : ((((isnotnull(d_year#142) AND isnotnull(d_moy#143)) AND (d_year#142 = 2001)) AND (d_moy#143 = 11)) AND isnotnull(d_date_sk#43)) -(142) CometProject -Input [3]: [d_date_sk#45, d_year#147, d_moy#148] -Arguments: [d_date_sk#45], [d_date_sk#45] +(125) CometProject +Input [3]: [d_date_sk#43, d_year#142, d_moy#143] +Arguments: [d_date_sk#43], [d_date_sk#43] -(143) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#45] +(126) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#43] -(144) BroadcastExchange -Input [1]: [d_date_sk#45] +(127) BroadcastExchange +Input [1]: [d_date_sk#43] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (149) -+- * ColumnarToRow (148) - +- CometProject (147) - +- CometFilter (146) - +- CometScan parquet spark_catalog.default.date_dim (145) +BroadcastExchange (132) ++- * ColumnarToRow (131) + +- CometProject (130) + +- CometFilter (129) + +- CometScan parquet spark_catalog.default.date_dim (128) -(145) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] +(128) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_year#144] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(146) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : (((isnotnull(d_year#27) AND (d_year#27 >= 1999)) AND (d_year#27 <= 2001)) AND isnotnull(d_date_sk#26)) +(129) CometFilter +Input [2]: [d_date_sk#26, d_year#144] +Condition : (((isnotnull(d_year#144) AND (d_year#144 >= 1999)) AND (d_year#144 <= 2001)) AND isnotnull(d_date_sk#26)) -(147) CometProject -Input [2]: [d_date_sk#26, d_year#27] +(130) CometProject +Input [2]: [d_date_sk#26, d_year#144] Arguments: [d_date_sk#26], [d_date_sk#26] -(148) ColumnarToRow [codegen id : 1] +(131) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] -(149) BroadcastExchange +(132) BroadcastExchange Input [1]: [d_date_sk#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] Subquery:7 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#32 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 90 Hosting Expression = ReusedSubquery Subquery scalar-subquery#56, [id=#57] +Subquery:9 Hosting operator id = 82 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] -Subquery:10 Hosting operator id = 76 Hosting Expression = cs_sold_date_sk#65 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 68 Hosting Expression = cs_sold_date_sk#63 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 106 Hosting Expression = ReusedSubquery Subquery scalar-subquery#56, [id=#57] +Subquery:11 Hosting operator id = 98 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] -Subquery:12 Hosting operator id = 92 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 84 Hosting Expression = ws_sold_date_sk#84 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt index 9dcb99a7a..e3dfa631b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su WholeStageCodegen (2) HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] InputAdapter - Exchange #17 + Exchange #14 WholeStageCodegen (1) ColumnarToRow InputAdapter @@ -25,26 +25,17 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #18 - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #11 CometProject [cs_quantity,cs_list_price] [quantity,list_price] CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #19 - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #11 CometProject [ws_quantity,ws_list_price] [quantity,list_price] CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #20 - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #11 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),sales,number_sales,sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #2 @@ -120,12 +111,9 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #12 - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #11 InputAdapter - BroadcastExchange #13 + BroadcastExchange #12 WholeStageCodegen (2) ColumnarToRow InputAdapter @@ -136,15 +124,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #14 - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #15 - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + ReusedExchange [d_date_sk] #11 InputAdapter - BroadcastExchange #16 + BroadcastExchange #13 WholeStageCodegen (9) BroadcastHashJoin [i_item_sk,ss_item_sk] ColumnarToRow @@ -161,7 +144,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),sales,number_sales,sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #21 + Exchange [i_brand_id,i_class_id,i_category_id] #15 WholeStageCodegen (23) HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] @@ -177,7 +160,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter ReusedExchange [ss_item_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #16 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 InputAdapter ReusedExchange [d_date_sk] #3 WholeStageCodegen (36) @@ -186,7 +169,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),sales,number_sales,sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #22 + Exchange [i_brand_id,i_class_id,i_category_id] #16 WholeStageCodegen (35) HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] @@ -202,6 +185,6 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter ReusedExchange [ss_item_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #16 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 InputAdapter ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt index 259b341a1..72879a2a9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt @@ -1,32 +1,32 @@ == Physical Plan == -TakeOrderedAndProject (92) -+- * BroadcastHashJoin Inner BuildRight (91) - :- * Filter (74) - : +- * HashAggregate (73) - : +- Exchange (72) - : +- * HashAggregate (71) - : +- * Project (70) - : +- * BroadcastHashJoin Inner BuildRight (69) - : :- * Project (67) - : : +- * BroadcastHashJoin Inner BuildRight (66) - : : :- * BroadcastHashJoin LeftSemi BuildRight (59) +TakeOrderedAndProject (84) ++- * BroadcastHashJoin Inner BuildRight (83) + :- * Filter (66) + : +- * HashAggregate (65) + : +- Exchange (64) + : +- * HashAggregate (63) + : +- * Project (62) + : +- * BroadcastHashJoin Inner BuildRight (61) + : :- * Project (59) + : : +- * BroadcastHashJoin Inner BuildRight (58) + : : :- * BroadcastHashJoin LeftSemi BuildRight (51) : : : :- * ColumnarToRow (3) : : : : +- CometFilter (2) : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (58) - : : : +- * Project (57) - : : : +- * BroadcastHashJoin Inner BuildRight (56) + : : : +- BroadcastExchange (50) + : : : +- * Project (49) + : : : +- * BroadcastHashJoin Inner BuildRight (48) : : : :- * ColumnarToRow (6) : : : : +- CometFilter (5) : : : : +- CometScan parquet spark_catalog.default.item (4) - : : : +- BroadcastExchange (55) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (54) - : : : :- * HashAggregate (38) - : : : : +- Exchange (37) - : : : : +- * ColumnarToRow (36) - : : : : +- CometHashAggregate (35) - : : : : +- CometProject (34) - : : : : +- CometBroadcastHashJoin (33) + : : : +- BroadcastExchange (47) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) + : : : :- * HashAggregate (35) + : : : : +- Exchange (34) + : : : : +- * ColumnarToRow (33) + : : : : +- CometHashAggregate (32) + : : : : +- CometProject (31) + : : : : +- CometBroadcastHashJoin (30) : : : : :- CometProject (28) : : : : : +- CometBroadcastHashJoin (27) : : : : : :- CometFilter (8) @@ -49,48 +49,40 @@ TakeOrderedAndProject (92) : : : : : +- CometProject (20) : : : : : +- CometFilter (19) : : : : : +- CometScan parquet spark_catalog.default.date_dim (18) - : : : : +- CometBroadcastExchange (32) - : : : : +- CometProject (31) - : : : : +- CometFilter (30) - : : : : +- CometScan parquet spark_catalog.default.date_dim (29) - : : : +- BroadcastExchange (53) - : : : +- * ColumnarToRow (52) - : : : +- CometProject (51) - : : : +- CometBroadcastHashJoin (50) - : : : :- CometProject (45) - : : : : +- CometBroadcastHashJoin (44) - : : : : :- CometFilter (40) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (39) - : : : : +- CometBroadcastExchange (43) - : : : : +- CometFilter (42) - : : : : +- CometScan parquet spark_catalog.default.item (41) - : : : +- CometBroadcastExchange (49) - : : : +- CometProject (48) - : : : +- CometFilter (47) - : : : +- CometScan parquet spark_catalog.default.date_dim (46) - : : +- BroadcastExchange (65) - : : +- * BroadcastHashJoin LeftSemi BuildRight (64) - : : :- * ColumnarToRow (62) - : : : +- CometFilter (61) - : : : +- CometScan parquet spark_catalog.default.item (60) - : : +- ReusedExchange (63) - : +- ReusedExchange (68) - +- BroadcastExchange (90) - +- * Filter (89) - +- * HashAggregate (88) - +- Exchange (87) - +- * HashAggregate (86) - +- * Project (85) - +- * BroadcastHashJoin Inner BuildRight (84) - :- * Project (82) - : +- * BroadcastHashJoin Inner BuildRight (81) - : :- * BroadcastHashJoin LeftSemi BuildRight (79) - : : :- * ColumnarToRow (77) - : : : +- CometFilter (76) - : : : +- CometScan parquet spark_catalog.default.store_sales (75) - : : +- ReusedExchange (78) - : +- ReusedExchange (80) - +- ReusedExchange (83) + : : : : +- ReusedExchange (29) + : : : +- BroadcastExchange (45) + : : : +- * ColumnarToRow (44) + : : : +- CometProject (43) + : : : +- CometBroadcastHashJoin (42) + : : : :- CometProject (40) + : : : : +- CometBroadcastHashJoin (39) + : : : : :- CometFilter (37) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (36) + : : : : +- ReusedExchange (38) + : : : +- ReusedExchange (41) + : : +- BroadcastExchange (57) + : : +- * BroadcastHashJoin LeftSemi BuildRight (56) + : : :- * ColumnarToRow (54) + : : : +- CometFilter (53) + : : : +- CometScan parquet spark_catalog.default.item (52) + : : +- ReusedExchange (55) + : +- ReusedExchange (60) + +- BroadcastExchange (82) + +- * Filter (81) + +- * HashAggregate (80) + +- Exchange (79) + +- * HashAggregate (78) + +- * Project (77) + +- * BroadcastHashJoin Inner BuildRight (76) + :- * Project (74) + : +- * BroadcastHashJoin Inner BuildRight (73) + : :- * BroadcastHashJoin LeftSemi BuildRight (71) + : : :- * ColumnarToRow (69) + : : : +- CometFilter (68) + : : : +- CometScan parquet spark_catalog.default.store_sales (67) + : : +- ReusedExchange (70) + : +- ReusedExchange (72) + +- ReusedExchange (75) (1) Scan parquet spark_catalog.default.store_sales @@ -231,622 +223,521 @@ Arguments: [ss_item_sk#10], [i_item_sk#13], Inner Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -(29) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct +(29) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#26] -(30) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : (((isnotnull(d_year#27) AND (d_year#27 >= 1999)) AND (d_year#27 <= 2001)) AND isnotnull(d_date_sk#26)) - -(31) CometProject -Input [2]: [d_date_sk#26, d_year#27] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(32) CometBroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: [d_date_sk#26] - -(33) CometBroadcastHashJoin +(30) CometBroadcastHashJoin Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] Right output [1]: [d_date_sk#26] Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner -(34) CometProject +(31) CometProject Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] -Arguments: [brand_id#28, class_id#29, category_id#30], [i_brand_id#14 AS brand_id#28, i_class_id#15 AS class_id#29, i_category_id#16 AS category_id#30] +Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] -(35) CometHashAggregate -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +(32) CometHashAggregate +Input [3]: [brand_id#27, class_id#28, category_id#29] +Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] -(36) ColumnarToRow [codegen id : 1] -Input [3]: [brand_id#28, class_id#29, category_id#30] +(33) ColumnarToRow [codegen id : 1] +Input [3]: [brand_id#27, class_id#28, category_id#29] -(37) Exchange -Input [3]: [brand_id#28, class_id#29, category_id#30] -Arguments: hashpartitioning(brand_id#28, class_id#29, category_id#30, 5), ENSURE_REQUIREMENTS, [plan_id=1] +(34) Exchange +Input [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(38) HashAggregate [codegen id : 3] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +(35) HashAggregate [codegen id : 3] +Input [3]: [brand_id#27, class_id#28, category_id#29] +Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#27, class_id#28, category_id#29] -(39) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#31, ws_sold_date_sk#32] +(36) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#32), dynamicpruningexpression(ws_sold_date_sk#32 IN dynamicpruning#33)] +PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(40) CometFilter -Input [2]: [ws_item_sk#31, ws_sold_date_sk#32] -Condition : isnotnull(ws_item_sk#31) +(37) CometFilter +Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] +Condition : isnotnull(ws_item_sk#30) -(41) Scan parquet spark_catalog.default.item -Output [4]: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(42) CometFilter -Input [4]: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] -Condition : isnotnull(i_item_sk#34) - -(43) CometBroadcastExchange -Input [4]: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] -Arguments: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] - -(44) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#31, ws_sold_date_sk#32] -Right output [4]: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] -Arguments: [ws_item_sk#31], [i_item_sk#34], Inner - -(45) CometProject -Input [6]: [ws_item_sk#31, ws_sold_date_sk#32, i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] -Arguments: [ws_sold_date_sk#32, i_brand_id#35, i_class_id#36, i_category_id#37], [ws_sold_date_sk#32, i_brand_id#35, i_class_id#36, i_category_id#37] +(38) ReusedExchange [Reuses operator id: 15] +Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -(46) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#38, d_year#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(47) CometFilter -Input [2]: [d_date_sk#38, d_year#39] -Condition : (((isnotnull(d_year#39) AND (d_year#39 >= 1999)) AND (d_year#39 <= 2001)) AND isnotnull(d_date_sk#38)) +(39) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] +Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [ws_item_sk#30], [i_item_sk#33], Inner -(48) CometProject -Input [2]: [d_date_sk#38, d_year#39] -Arguments: [d_date_sk#38], [d_date_sk#38] +(40) CometProject +Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -(49) CometBroadcastExchange -Input [1]: [d_date_sk#38] -Arguments: [d_date_sk#38] +(41) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#37] -(50) CometBroadcastHashJoin -Left output [4]: [ws_sold_date_sk#32, i_brand_id#35, i_class_id#36, i_category_id#37] -Right output [1]: [d_date_sk#38] -Arguments: [ws_sold_date_sk#32], [d_date_sk#38], Inner +(42) CometBroadcastHashJoin +Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] +Right output [1]: [d_date_sk#37] +Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner -(51) CometProject -Input [5]: [ws_sold_date_sk#32, i_brand_id#35, i_class_id#36, i_category_id#37, d_date_sk#38] -Arguments: [i_brand_id#35, i_class_id#36, i_category_id#37], [i_brand_id#35, i_class_id#36, i_category_id#37] +(43) CometProject +Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] +Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] -(52) ColumnarToRow [codegen id : 2] -Input [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] +(44) ColumnarToRow [codegen id : 2] +Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -(53) BroadcastExchange -Input [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] +(45) BroadcastExchange +Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] -(54) BroadcastHashJoin [codegen id : 3] -Left keys [6]: [coalesce(brand_id#28, 0), isnull(brand_id#28), coalesce(class_id#29, 0), isnull(class_id#29), coalesce(category_id#30, 0), isnull(category_id#30)] -Right keys [6]: [coalesce(i_brand_id#35, 0), isnull(i_brand_id#35), coalesce(i_class_id#36, 0), isnull(i_class_id#36), coalesce(i_category_id#37, 0), isnull(i_category_id#37)] +(46) BroadcastHashJoin [codegen id : 3] +Left keys [6]: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)] +Right keys [6]: [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)] Join type: LeftSemi Join condition: None -(55) BroadcastExchange -Input [3]: [brand_id#28, class_id#29, category_id#30] +(47) BroadcastExchange +Input [3]: [brand_id#27, class_id#28, category_id#29] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=3] -(56) BroadcastHashJoin [codegen id : 4] +(48) BroadcastHashJoin [codegen id : 4] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#28, class_id#29, category_id#30] +Right keys [3]: [brand_id#27, class_id#28, category_id#29] Join type: Inner Join condition: None -(57) Project [codegen id : 4] -Output [1]: [i_item_sk#6 AS ss_item_sk#40] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#28, class_id#29, category_id#30] +(49) Project [codegen id : 4] +Output [1]: [i_item_sk#6 AS ss_item_sk#38] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] -(58) BroadcastExchange -Input [1]: [ss_item_sk#40] +(50) BroadcastExchange +Input [1]: [ss_item_sk#38] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(59) BroadcastHashJoin [codegen id : 11] +(51) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#40] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None -(60) Scan parquet spark_catalog.default.item -Output [4]: [i_item_sk#41, i_brand_id#42, i_class_id#43, i_category_id#44] +(52) Scan parquet spark_catalog.default.item +Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(61) CometFilter -Input [4]: [i_item_sk#41, i_brand_id#42, i_class_id#43, i_category_id#44] -Condition : (((isnotnull(i_item_sk#41) AND isnotnull(i_brand_id#42)) AND isnotnull(i_class_id#43)) AND isnotnull(i_category_id#44)) +(53) CometFilter +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Condition : (((isnotnull(i_item_sk#39) AND isnotnull(i_brand_id#40)) AND isnotnull(i_class_id#41)) AND isnotnull(i_category_id#42)) -(62) ColumnarToRow [codegen id : 9] -Input [4]: [i_item_sk#41, i_brand_id#42, i_class_id#43, i_category_id#44] +(54) ColumnarToRow [codegen id : 9] +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -(63) ReusedExchange [Reuses operator id: 58] -Output [1]: [ss_item_sk#40] +(55) ReusedExchange [Reuses operator id: 50] +Output [1]: [ss_item_sk#38] -(64) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [i_item_sk#41] -Right keys [1]: [ss_item_sk#40] +(56) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [i_item_sk#39] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None -(65) BroadcastExchange -Input [4]: [i_item_sk#41, i_brand_id#42, i_class_id#43, i_category_id#44] +(57) BroadcastExchange +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(66) BroadcastHashJoin [codegen id : 11] +(58) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#41] +Right keys [1]: [i_item_sk#39] Join type: Inner Join condition: None -(67) Project [codegen id : 11] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#42, i_class_id#43, i_category_id#44] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#41, i_brand_id#42, i_class_id#43, i_category_id#44] +(59) Project [codegen id : 11] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -(68) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#45] +(60) ReusedExchange [Reuses operator id: 106] +Output [1]: [d_date_sk#43] -(69) BroadcastHashJoin [codegen id : 11] +(61) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#45] +Right keys [1]: [d_date_sk#43] Join type: Inner Join condition: None -(70) Project [codegen id : 11] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#42, i_class_id#43, i_category_id#44] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#42, i_class_id#43, i_category_id#44, d_date_sk#45] +(62) Project [codegen id : 11] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -(71) HashAggregate [codegen id : 11] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#42, i_class_id#43, i_category_id#44] -Keys [3]: [i_brand_id#42, i_class_id#43, i_category_id#44] +(63) HashAggregate [codegen id : 11] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] +Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] -Aggregate Attributes [3]: [sum#46, isEmpty#47, count#48] -Results [6]: [i_brand_id#42, i_class_id#43, i_category_id#44, sum#49, isEmpty#50, count#51] +Aggregate Attributes [3]: [sum#44, isEmpty#45, count#46] +Results [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] -(72) Exchange -Input [6]: [i_brand_id#42, i_class_id#43, i_category_id#44, sum#49, isEmpty#50, count#51] -Arguments: hashpartitioning(i_brand_id#42, i_class_id#43, i_category_id#44, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(64) Exchange +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] +Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(73) HashAggregate [codegen id : 24] -Input [6]: [i_brand_id#42, i_class_id#43, i_category_id#44, sum#49, isEmpty#50, count#51] -Keys [3]: [i_brand_id#42, i_class_id#43, i_category_id#44] +(65) HashAggregate [codegen id : 24] +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] +Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#52, count(1)#53] -Results [6]: [store AS channel#54, i_brand_id#42, i_class_id#43, i_category_id#44, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#52 AS sales#55, count(1)#53 AS number_sales#56] +Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#50, count(1)#51] +Results [6]: [store AS channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#50 AS sales#53, count(1)#51 AS number_sales#54] -(74) Filter [codegen id : 24] -Input [6]: [channel#54, i_brand_id#42, i_class_id#43, i_category_id#44, sales#55, number_sales#56] -Condition : (isnotnull(sales#55) AND (cast(sales#55 as decimal(32,6)) > cast(Subquery scalar-subquery#57, [id=#58] as decimal(32,6)))) +(66) Filter [codegen id : 24] +Input [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sales#53, number_sales#54] +Condition : (isnotnull(sales#53) AND (cast(sales#53 as decimal(32,6)) > cast(Subquery scalar-subquery#55, [id=#56] as decimal(32,6)))) -(75) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#59, ss_quantity#60, ss_list_price#61, ss_sold_date_sk#62] +(67) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#62), dynamicpruningexpression(ss_sold_date_sk#62 IN dynamicpruning#63)] +PartitionFilters: [isnotnull(ss_sold_date_sk#60), dynamicpruningexpression(ss_sold_date_sk#60 IN dynamicpruning#61)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(76) CometFilter -Input [4]: [ss_item_sk#59, ss_quantity#60, ss_list_price#61, ss_sold_date_sk#62] -Condition : isnotnull(ss_item_sk#59) +(68) CometFilter +Input [4]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60] +Condition : isnotnull(ss_item_sk#57) -(77) ColumnarToRow [codegen id : 22] -Input [4]: [ss_item_sk#59, ss_quantity#60, ss_list_price#61, ss_sold_date_sk#62] +(69) ColumnarToRow [codegen id : 22] +Input [4]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60] -(78) ReusedExchange [Reuses operator id: 58] -Output [1]: [ss_item_sk#40] +(70) ReusedExchange [Reuses operator id: 50] +Output [1]: [ss_item_sk#38] -(79) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ss_item_sk#59] -Right keys [1]: [ss_item_sk#40] +(71) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [ss_item_sk#57] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None -(80) ReusedExchange [Reuses operator id: 65] -Output [4]: [i_item_sk#64, i_brand_id#65, i_class_id#66, i_category_id#67] +(72) ReusedExchange [Reuses operator id: 57] +Output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] -(81) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ss_item_sk#59] -Right keys [1]: [i_item_sk#64] +(73) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [ss_item_sk#57] +Right keys [1]: [i_item_sk#62] Join type: Inner Join condition: None -(82) Project [codegen id : 22] -Output [6]: [ss_quantity#60, ss_list_price#61, ss_sold_date_sk#62, i_brand_id#65, i_class_id#66, i_category_id#67] -Input [8]: [ss_item_sk#59, ss_quantity#60, ss_list_price#61, ss_sold_date_sk#62, i_item_sk#64, i_brand_id#65, i_class_id#66, i_category_id#67] +(74) Project [codegen id : 22] +Output [6]: [ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60, i_brand_id#63, i_class_id#64, i_category_id#65] +Input [8]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] -(83) ReusedExchange [Reuses operator id: 137] -Output [1]: [d_date_sk#68] +(75) ReusedExchange [Reuses operator id: 120] +Output [1]: [d_date_sk#66] -(84) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ss_sold_date_sk#62] -Right keys [1]: [d_date_sk#68] +(76) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [ss_sold_date_sk#60] +Right keys [1]: [d_date_sk#66] Join type: Inner Join condition: None -(85) Project [codegen id : 22] -Output [5]: [ss_quantity#60, ss_list_price#61, i_brand_id#65, i_class_id#66, i_category_id#67] -Input [7]: [ss_quantity#60, ss_list_price#61, ss_sold_date_sk#62, i_brand_id#65, i_class_id#66, i_category_id#67, d_date_sk#68] - -(86) HashAggregate [codegen id : 22] -Input [5]: [ss_quantity#60, ss_list_price#61, i_brand_id#65, i_class_id#66, i_category_id#67] -Keys [3]: [i_brand_id#65, i_class_id#66, i_category_id#67] -Functions [2]: [partial_sum((cast(ss_quantity#60 as decimal(10,0)) * ss_list_price#61)), partial_count(1)] -Aggregate Attributes [3]: [sum#69, isEmpty#70, count#71] -Results [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] - -(87) Exchange -Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] -Arguments: hashpartitioning(i_brand_id#65, i_class_id#66, i_category_id#67, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(88) HashAggregate [codegen id : 23] -Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] -Keys [3]: [i_brand_id#65, i_class_id#66, i_category_id#67] -Functions [2]: [sum((cast(ss_quantity#60 as decimal(10,0)) * ss_list_price#61)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#60 as decimal(10,0)) * ss_list_price#61))#75, count(1)#76] -Results [6]: [store AS channel#77, i_brand_id#65, i_class_id#66, i_category_id#67, sum((cast(ss_quantity#60 as decimal(10,0)) * ss_list_price#61))#75 AS sales#78, count(1)#76 AS number_sales#79] - -(89) Filter [codegen id : 23] -Input [6]: [channel#77, i_brand_id#65, i_class_id#66, i_category_id#67, sales#78, number_sales#79] -Condition : (isnotnull(sales#78) AND (cast(sales#78 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#57, [id=#58] as decimal(32,6)))) - -(90) BroadcastExchange -Input [6]: [channel#77, i_brand_id#65, i_class_id#66, i_category_id#67, sales#78, number_sales#79] +(77) Project [codegen id : 22] +Output [5]: [ss_quantity#58, ss_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65] +Input [7]: [ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60, i_brand_id#63, i_class_id#64, i_category_id#65, d_date_sk#66] + +(78) HashAggregate [codegen id : 22] +Input [5]: [ss_quantity#58, ss_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65] +Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] +Functions [2]: [partial_sum((cast(ss_quantity#58 as decimal(10,0)) * ss_list_price#59)), partial_count(1)] +Aggregate Attributes [3]: [sum#67, isEmpty#68, count#69] +Results [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] + +(79) Exchange +Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] +Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(80) HashAggregate [codegen id : 23] +Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] +Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] +Functions [2]: [sum((cast(ss_quantity#58 as decimal(10,0)) * ss_list_price#59)), count(1)] +Aggregate Attributes [2]: [sum((cast(ss_quantity#58 as decimal(10,0)) * ss_list_price#59))#73, count(1)#74] +Results [6]: [store AS channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sum((cast(ss_quantity#58 as decimal(10,0)) * ss_list_price#59))#73 AS sales#76, count(1)#74 AS number_sales#77] + +(81) Filter [codegen id : 23] +Input [6]: [channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] +Condition : (isnotnull(sales#76) AND (cast(sales#76 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#55, [id=#56] as decimal(32,6)))) + +(82) BroadcastExchange +Input [6]: [channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=8] -(91) BroadcastHashJoin [codegen id : 24] -Left keys [3]: [i_brand_id#42, i_class_id#43, i_category_id#44] -Right keys [3]: [i_brand_id#65, i_class_id#66, i_category_id#67] +(83) BroadcastHashJoin [codegen id : 24] +Left keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] +Right keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] Join type: Inner Join condition: None -(92) TakeOrderedAndProject -Input [12]: [channel#54, i_brand_id#42, i_class_id#43, i_category_id#44, sales#55, number_sales#56, channel#77, i_brand_id#65, i_class_id#66, i_category_id#67, sales#78, number_sales#79] -Arguments: 100, [i_brand_id#42 ASC NULLS FIRST, i_class_id#43 ASC NULLS FIRST, i_category_id#44 ASC NULLS FIRST], [channel#54, i_brand_id#42, i_class_id#43, i_category_id#44, sales#55, number_sales#56, channel#77, i_brand_id#65, i_class_id#66, i_category_id#67, sales#78, number_sales#79] +(84) TakeOrderedAndProject +Input [12]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sales#53, number_sales#54, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] +Arguments: 100, [i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sales#53, number_sales#54, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] ===== Subqueries ===== -Subquery:1 Hosting operator id = 74 Hosting Expression = Subquery scalar-subquery#57, [id=#58] -* HashAggregate (118) -+- Exchange (117) - +- * ColumnarToRow (116) - +- CometHashAggregate (115) - +- CometUnion (114) - :- CometProject (99) - : +- CometBroadcastHashJoin (98) - : :- CometScan parquet spark_catalog.default.store_sales (93) - : +- CometBroadcastExchange (97) - : +- CometProject (96) - : +- CometFilter (95) - : +- CometScan parquet spark_catalog.default.date_dim (94) - :- CometProject (106) - : +- CometBroadcastHashJoin (105) - : :- CometScan parquet spark_catalog.default.catalog_sales (100) - : +- CometBroadcastExchange (104) - : +- CometProject (103) - : +- CometFilter (102) - : +- CometScan parquet spark_catalog.default.date_dim (101) - +- CometProject (113) - +- CometBroadcastHashJoin (112) - :- CometScan parquet spark_catalog.default.web_sales (107) - +- CometBroadcastExchange (111) - +- CometProject (110) - +- CometFilter (109) - +- CometScan parquet spark_catalog.default.date_dim (108) - - -(93) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#80, ss_list_price#81, ss_sold_date_sk#82] +Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#55, [id=#56] +* HashAggregate (101) ++- Exchange (100) + +- * ColumnarToRow (99) + +- CometHashAggregate (98) + +- CometUnion (97) + :- CometProject (88) + : +- CometBroadcastHashJoin (87) + : :- CometScan parquet spark_catalog.default.store_sales (85) + : +- ReusedExchange (86) + :- CometProject (92) + : +- CometBroadcastHashJoin (91) + : :- CometScan parquet spark_catalog.default.catalog_sales (89) + : +- ReusedExchange (90) + +- CometProject (96) + +- CometBroadcastHashJoin (95) + :- CometScan parquet spark_catalog.default.web_sales (93) + +- ReusedExchange (94) + + +(85) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#82), dynamicpruningexpression(ss_sold_date_sk#82 IN dynamicpruning#83)] +PartitionFilters: [isnotnull(ss_sold_date_sk#80), dynamicpruningexpression(ss_sold_date_sk#80 IN dynamicpruning#81)] ReadSchema: struct -(94) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#84, d_year#85] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(95) CometFilter -Input [2]: [d_date_sk#84, d_year#85] -Condition : (((isnotnull(d_year#85) AND (d_year#85 >= 1999)) AND (d_year#85 <= 2001)) AND isnotnull(d_date_sk#84)) - -(96) CometProject -Input [2]: [d_date_sk#84, d_year#85] -Arguments: [d_date_sk#84], [d_date_sk#84] +(86) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#82] -(97) CometBroadcastExchange -Input [1]: [d_date_sk#84] -Arguments: [d_date_sk#84] +(87) CometBroadcastHashJoin +Left output [3]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80] +Right output [1]: [d_date_sk#82] +Arguments: [ss_sold_date_sk#80], [d_date_sk#82], Inner -(98) CometBroadcastHashJoin -Left output [3]: [ss_quantity#80, ss_list_price#81, ss_sold_date_sk#82] -Right output [1]: [d_date_sk#84] -Arguments: [ss_sold_date_sk#82], [d_date_sk#84], Inner +(88) CometProject +Input [4]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80, d_date_sk#82] +Arguments: [quantity#83, list_price#84], [ss_quantity#78 AS quantity#83, ss_list_price#79 AS list_price#84] -(99) CometProject -Input [4]: [ss_quantity#80, ss_list_price#81, ss_sold_date_sk#82, d_date_sk#84] -Arguments: [quantity#86, list_price#87], [ss_quantity#80 AS quantity#86, ss_list_price#81 AS list_price#87] - -(100) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#88, cs_list_price#89, cs_sold_date_sk#90] +(89) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#90), dynamicpruningexpression(cs_sold_date_sk#90 IN dynamicpruning#91)] +PartitionFilters: [isnotnull(cs_sold_date_sk#87), dynamicpruningexpression(cs_sold_date_sk#87 IN dynamicpruning#88)] ReadSchema: struct -(101) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#92, d_year#93] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(102) CometFilter -Input [2]: [d_date_sk#92, d_year#93] -Condition : (((isnotnull(d_year#93) AND (d_year#93 >= 1999)) AND (d_year#93 <= 2001)) AND isnotnull(d_date_sk#92)) +(90) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#89] -(103) CometProject -Input [2]: [d_date_sk#92, d_year#93] -Arguments: [d_date_sk#92], [d_date_sk#92] +(91) CometBroadcastHashJoin +Left output [3]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87] +Right output [1]: [d_date_sk#89] +Arguments: [cs_sold_date_sk#87], [d_date_sk#89], Inner -(104) CometBroadcastExchange -Input [1]: [d_date_sk#92] -Arguments: [d_date_sk#92] +(92) CometProject +Input [4]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87, d_date_sk#89] +Arguments: [quantity#90, list_price#91], [cs_quantity#85 AS quantity#90, cs_list_price#86 AS list_price#91] -(105) CometBroadcastHashJoin -Left output [3]: [cs_quantity#88, cs_list_price#89, cs_sold_date_sk#90] -Right output [1]: [d_date_sk#92] -Arguments: [cs_sold_date_sk#90], [d_date_sk#92], Inner - -(106) CometProject -Input [4]: [cs_quantity#88, cs_list_price#89, cs_sold_date_sk#90, d_date_sk#92] -Arguments: [quantity#94, list_price#95], [cs_quantity#88 AS quantity#94, cs_list_price#89 AS list_price#95] - -(107) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#96, ws_list_price#97, ws_sold_date_sk#98] +(93) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#98), dynamicpruningexpression(ws_sold_date_sk#98 IN dynamicpruning#99)] +PartitionFilters: [isnotnull(ws_sold_date_sk#94), dynamicpruningexpression(ws_sold_date_sk#94 IN dynamicpruning#95)] ReadSchema: struct -(108) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#100, d_year#101] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(109) CometFilter -Input [2]: [d_date_sk#100, d_year#101] -Condition : (((isnotnull(d_year#101) AND (d_year#101 >= 1999)) AND (d_year#101 <= 2001)) AND isnotnull(d_date_sk#100)) - -(110) CometProject -Input [2]: [d_date_sk#100, d_year#101] -Arguments: [d_date_sk#100], [d_date_sk#100] - -(111) CometBroadcastExchange -Input [1]: [d_date_sk#100] -Arguments: [d_date_sk#100] +(94) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#96] -(112) CometBroadcastHashJoin -Left output [3]: [ws_quantity#96, ws_list_price#97, ws_sold_date_sk#98] -Right output [1]: [d_date_sk#100] -Arguments: [ws_sold_date_sk#98], [d_date_sk#100], Inner +(95) CometBroadcastHashJoin +Left output [3]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] +Right output [1]: [d_date_sk#96] +Arguments: [ws_sold_date_sk#94], [d_date_sk#96], Inner -(113) CometProject -Input [4]: [ws_quantity#96, ws_list_price#97, ws_sold_date_sk#98, d_date_sk#100] -Arguments: [quantity#102, list_price#103], [ws_quantity#96 AS quantity#102, ws_list_price#97 AS list_price#103] +(96) CometProject +Input [4]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, d_date_sk#96] +Arguments: [quantity#97, list_price#98], [ws_quantity#92 AS quantity#97, ws_list_price#93 AS list_price#98] -(114) CometUnion -Child 0 Input [2]: [quantity#86, list_price#87] -Child 1 Input [2]: [quantity#94, list_price#95] -Child 2 Input [2]: [quantity#102, list_price#103] +(97) CometUnion +Child 0 Input [2]: [quantity#83, list_price#84] +Child 1 Input [2]: [quantity#90, list_price#91] +Child 2 Input [2]: [quantity#97, list_price#98] -(115) CometHashAggregate -Input [2]: [quantity#86, list_price#87] +(98) CometHashAggregate +Input [2]: [quantity#83, list_price#84] Keys: [] -Functions [1]: [partial_avg((cast(quantity#86 as decimal(10,0)) * list_price#87))] +Functions [1]: [partial_avg((cast(quantity#83 as decimal(10,0)) * list_price#84))] -(116) ColumnarToRow [codegen id : 1] -Input [2]: [sum#104, count#105] +(99) ColumnarToRow [codegen id : 1] +Input [2]: [sum#99, count#100] -(117) Exchange -Input [2]: [sum#104, count#105] +(100) Exchange +Input [2]: [sum#99, count#100] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] -(118) HashAggregate [codegen id : 2] -Input [2]: [sum#104, count#105] +(101) HashAggregate [codegen id : 2] +Input [2]: [sum#99, count#100] Keys: [] -Functions [1]: [avg((cast(quantity#86 as decimal(10,0)) * list_price#87))] -Aggregate Attributes [1]: [avg((cast(quantity#86 as decimal(10,0)) * list_price#87))#106] -Results [1]: [avg((cast(quantity#86 as decimal(10,0)) * list_price#87))#106 AS average_sales#107] +Functions [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))] +Aggregate Attributes [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))#101] +Results [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))#101 AS average_sales#102] -Subquery:2 Hosting operator id = 93 Hosting Expression = ss_sold_date_sk#82 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 85 Hosting Expression = ss_sold_date_sk#80 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 100 Hosting Expression = cs_sold_date_sk#90 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 89 Hosting Expression = cs_sold_date_sk#87 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#98 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 93 Hosting Expression = ws_sold_date_sk#94 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (123) -+- * ColumnarToRow (122) - +- CometProject (121) - +- CometFilter (120) - +- CometScan parquet spark_catalog.default.date_dim (119) +BroadcastExchange (106) ++- * ColumnarToRow (105) + +- CometProject (104) + +- CometFilter (103) + +- CometScan parquet spark_catalog.default.date_dim (102) -(119) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#45, d_week_seq#108] +(102) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#43, d_week_seq#103] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(120) CometFilter -Input [2]: [d_date_sk#45, d_week_seq#108] -Condition : ((isnotnull(d_week_seq#108) AND (d_week_seq#108 = Subquery scalar-subquery#109, [id=#110])) AND isnotnull(d_date_sk#45)) +(103) CometFilter +Input [2]: [d_date_sk#43, d_week_seq#103] +Condition : ((isnotnull(d_week_seq#103) AND (d_week_seq#103 = Subquery scalar-subquery#104, [id=#105])) AND isnotnull(d_date_sk#43)) -(121) CometProject -Input [2]: [d_date_sk#45, d_week_seq#108] -Arguments: [d_date_sk#45], [d_date_sk#45] +(104) CometProject +Input [2]: [d_date_sk#43, d_week_seq#103] +Arguments: [d_date_sk#43], [d_date_sk#43] -(122) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#45] +(105) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#43] -(123) BroadcastExchange -Input [1]: [d_date_sk#45] +(106) BroadcastExchange +Input [1]: [d_date_sk#43] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] -Subquery:6 Hosting operator id = 120 Hosting Expression = Subquery scalar-subquery#109, [id=#110] -* ColumnarToRow (127) -+- CometProject (126) - +- CometFilter (125) - +- CometScan parquet spark_catalog.default.date_dim (124) +Subquery:6 Hosting operator id = 103 Hosting Expression = Subquery scalar-subquery#104, [id=#105] +* ColumnarToRow (110) ++- CometProject (109) + +- CometFilter (108) + +- CometScan parquet spark_catalog.default.date_dim (107) -(124) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#111, d_year#112, d_moy#113, d_dom#114] +(107) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_week_seq#106, d_year#107, d_moy#108, d_dom#109] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct -(125) CometFilter -Input [4]: [d_week_seq#111, d_year#112, d_moy#113, d_dom#114] -Condition : (((((isnotnull(d_year#112) AND isnotnull(d_moy#113)) AND isnotnull(d_dom#114)) AND (d_year#112 = 2000)) AND (d_moy#113 = 12)) AND (d_dom#114 = 11)) +(108) CometFilter +Input [4]: [d_week_seq#106, d_year#107, d_moy#108, d_dom#109] +Condition : (((((isnotnull(d_year#107) AND isnotnull(d_moy#108)) AND isnotnull(d_dom#109)) AND (d_year#107 = 2000)) AND (d_moy#108 = 12)) AND (d_dom#109 = 11)) -(126) CometProject -Input [4]: [d_week_seq#111, d_year#112, d_moy#113, d_dom#114] -Arguments: [d_week_seq#111], [d_week_seq#111] +(109) CometProject +Input [4]: [d_week_seq#106, d_year#107, d_moy#108, d_dom#109] +Arguments: [d_week_seq#106], [d_week_seq#106] -(127) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#111] +(110) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#106] Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (132) -+- * ColumnarToRow (131) - +- CometProject (130) - +- CometFilter (129) - +- CometScan parquet spark_catalog.default.date_dim (128) +BroadcastExchange (115) ++- * ColumnarToRow (114) + +- CometProject (113) + +- CometFilter (112) + +- CometScan parquet spark_catalog.default.date_dim (111) -(128) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] +(111) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_year#110] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(129) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : (((isnotnull(d_year#27) AND (d_year#27 >= 1999)) AND (d_year#27 <= 2001)) AND isnotnull(d_date_sk#26)) +(112) CometFilter +Input [2]: [d_date_sk#26, d_year#110] +Condition : (((isnotnull(d_year#110) AND (d_year#110 >= 1999)) AND (d_year#110 <= 2001)) AND isnotnull(d_date_sk#26)) -(130) CometProject -Input [2]: [d_date_sk#26, d_year#27] +(113) CometProject +Input [2]: [d_date_sk#26, d_year#110] Arguments: [d_date_sk#26], [d_date_sk#26] -(131) ColumnarToRow [codegen id : 1] +(114) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] -(132) BroadcastExchange +(115) BroadcastExchange Input [1]: [d_date_sk#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] Subquery:8 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#32 IN dynamicpruning#12 +Subquery:9 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 -Subquery:10 Hosting operator id = 89 Hosting Expression = ReusedSubquery Subquery scalar-subquery#57, [id=#58] +Subquery:10 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#55, [id=#56] -Subquery:11 Hosting operator id = 75 Hosting Expression = ss_sold_date_sk#62 IN dynamicpruning#63 -BroadcastExchange (137) -+- * ColumnarToRow (136) - +- CometProject (135) - +- CometFilter (134) - +- CometScan parquet spark_catalog.default.date_dim (133) +Subquery:11 Hosting operator id = 67 Hosting Expression = ss_sold_date_sk#60 IN dynamicpruning#61 +BroadcastExchange (120) ++- * ColumnarToRow (119) + +- CometProject (118) + +- CometFilter (117) + +- CometScan parquet spark_catalog.default.date_dim (116) -(133) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#68, d_week_seq#115] +(116) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#66, d_week_seq#111] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(134) CometFilter -Input [2]: [d_date_sk#68, d_week_seq#115] -Condition : ((isnotnull(d_week_seq#115) AND (d_week_seq#115 = Subquery scalar-subquery#116, [id=#117])) AND isnotnull(d_date_sk#68)) +(117) CometFilter +Input [2]: [d_date_sk#66, d_week_seq#111] +Condition : ((isnotnull(d_week_seq#111) AND (d_week_seq#111 = Subquery scalar-subquery#112, [id=#113])) AND isnotnull(d_date_sk#66)) -(135) CometProject -Input [2]: [d_date_sk#68, d_week_seq#115] -Arguments: [d_date_sk#68], [d_date_sk#68] +(118) CometProject +Input [2]: [d_date_sk#66, d_week_seq#111] +Arguments: [d_date_sk#66], [d_date_sk#66] -(136) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#68] +(119) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#66] -(137) BroadcastExchange -Input [1]: [d_date_sk#68] +(120) BroadcastExchange +Input [1]: [d_date_sk#66] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -Subquery:12 Hosting operator id = 134 Hosting Expression = Subquery scalar-subquery#116, [id=#117] -* ColumnarToRow (141) -+- CometProject (140) - +- CometFilter (139) - +- CometScan parquet spark_catalog.default.date_dim (138) +Subquery:12 Hosting operator id = 117 Hosting Expression = Subquery scalar-subquery#112, [id=#113] +* ColumnarToRow (124) ++- CometProject (123) + +- CometFilter (122) + +- CometScan parquet spark_catalog.default.date_dim (121) -(138) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#118, d_year#119, d_moy#120, d_dom#121] +(121) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_week_seq#114, d_year#115, d_moy#116, d_dom#117] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct -(139) CometFilter -Input [4]: [d_week_seq#118, d_year#119, d_moy#120, d_dom#121] -Condition : (((((isnotnull(d_year#119) AND isnotnull(d_moy#120)) AND isnotnull(d_dom#121)) AND (d_year#119 = 1999)) AND (d_moy#120 = 12)) AND (d_dom#121 = 11)) +(122) CometFilter +Input [4]: [d_week_seq#114, d_year#115, d_moy#116, d_dom#117] +Condition : (((((isnotnull(d_year#115) AND isnotnull(d_moy#116)) AND isnotnull(d_dom#117)) AND (d_year#115 = 1999)) AND (d_moy#116 = 12)) AND (d_dom#117 = 11)) -(140) CometProject -Input [4]: [d_week_seq#118, d_year#119, d_moy#120, d_dom#121] -Arguments: [d_week_seq#118], [d_week_seq#118] +(123) CometProject +Input [4]: [d_week_seq#114, d_year#115, d_moy#116, d_dom#117] +Arguments: [d_week_seq#114], [d_week_seq#114] -(141) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#118] +(124) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#114] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt index e4517e1cf..799f74a36 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt @@ -6,7 +6,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (2) HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] InputAdapter - Exchange #16 + Exchange #13 WholeStageCodegen (1) ColumnarToRow InputAdapter @@ -16,26 +16,17 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange #17 - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #10 CometProject [cs_quantity,cs_list_price] [quantity,list_price] CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange #18 - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #10 CometProject [ws_quantity,ws_list_price] [quantity,list_price] CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange #19 - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #10 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #1 @@ -118,12 +109,9 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #11 - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #10 InputAdapter - BroadcastExchange #12 + BroadcastExchange #11 WholeStageCodegen (2) ColumnarToRow InputAdapter @@ -134,15 +122,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange #13 - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #14 - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + ReusedExchange [d_date_sk] #10 InputAdapter - BroadcastExchange #15 + BroadcastExchange #12 WholeStageCodegen (9) BroadcastHashJoin [i_item_sk,ss_item_sk] ColumnarToRow @@ -154,13 +137,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter - BroadcastExchange #20 + BroadcastExchange #14 WholeStageCodegen (23) Filter [sales] ReusedSubquery [average_sales] #4 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #21 + Exchange [i_brand_id,i_class_id,i_category_id] #15 WholeStageCodegen (22) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] @@ -173,7 +156,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 - BroadcastExchange #22 + BroadcastExchange #16 WholeStageCodegen (1) ColumnarToRow InputAdapter @@ -190,6 +173,6 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter ReusedExchange [ss_item_sk] #3 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 InputAdapter - ReusedExchange [d_date_sk] #22 + ReusedExchange [d_date_sk] #16 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt index 1caccba47..c19fde749 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt @@ -1,15 +1,15 @@ == Physical Plan == -TakeOrderedAndProject (45) -+- * HashAggregate (44) - +- Exchange (43) - +- * ColumnarToRow (42) - +- CometHashAggregate (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (35) - : +- CometBroadcastHashJoin (34) - : :- CometProject (30) - : : +- CometBroadcastHashJoin (29) +TakeOrderedAndProject (42) ++- * HashAggregate (41) + +- Exchange (40) + +- * ColumnarToRow (39) + +- CometHashAggregate (38) + +- CometProject (37) + +- CometBroadcastHashJoin (36) + :- CometProject (32) + : +- CometBroadcastHashJoin (31) + : :- CometProject (27) + : : +- CometBroadcastHashJoin (26) : : :- CometProject (24) : : : +- CometBroadcastHashJoin (23) : : : :- CometProject (18) @@ -34,16 +34,13 @@ TakeOrderedAndProject (45) : : : +- CometProject (21) : : : +- CometFilter (20) : : : +- CometScan parquet spark_catalog.default.date_dim (19) - : : +- CometBroadcastExchange (28) - : : +- CometProject (27) - : : +- CometFilter (26) - : : +- CometScan parquet spark_catalog.default.date_dim (25) - : +- CometBroadcastExchange (33) - : +- CometFilter (32) - : +- CometScan parquet spark_catalog.default.store (31) - +- CometBroadcastExchange (38) - +- CometFilter (37) - +- CometScan parquet spark_catalog.default.item (36) + : : +- ReusedExchange (25) + : +- CometBroadcastExchange (30) + : +- CometFilter (29) + : +- CometScan parquet spark_catalog.default.store (28) + +- CometBroadcastExchange (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.item (33) (1) Scan parquet spark_catalog.default.store_sales @@ -164,164 +161,148 @@ Arguments: [sr_returned_date_sk#12], [d_date_sk#21], Inner Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#21] Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] -(25) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#23, d_quarter_name#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_quarter_name, [2001Q1,2001Q2,2001Q3]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [d_date_sk#23, d_quarter_name#24] -Condition : (d_quarter_name#24 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#23)) +(25) ReusedExchange [Reuses operator id: 22] +Output [1]: [d_date_sk#23] -(27) CometProject -Input [2]: [d_date_sk#23, d_quarter_name#24] -Arguments: [d_date_sk#23], [d_date_sk#23] - -(28) CometBroadcastExchange -Input [1]: [d_date_sk#23] -Arguments: [d_date_sk#23] - -(29) CometBroadcastHashJoin +(26) CometBroadcastHashJoin Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] Right output [1]: [d_date_sk#23] Arguments: [cs_sold_date_sk#17], [d_date_sk#23], Inner -(30) CometProject +(27) CometProject Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#23] Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] -(31) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#25, s_state#26] +(28) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#24, s_state#25] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(32) CometFilter -Input [2]: [s_store_sk#25, s_state#26] -Condition : isnotnull(s_store_sk#25) +(29) CometFilter +Input [2]: [s_store_sk#24, s_state#25] +Condition : isnotnull(s_store_sk#24) -(33) CometBroadcastExchange -Input [2]: [s_store_sk#25, s_state#26] -Arguments: [s_store_sk#25, s_state#26] +(30) CometBroadcastExchange +Input [2]: [s_store_sk#24, s_state#25] +Arguments: [s_store_sk#24, s_state#25] -(34) CometBroadcastHashJoin +(31) CometBroadcastHashJoin Left output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] -Right output [2]: [s_store_sk#25, s_state#26] -Arguments: [ss_store_sk#3], [s_store_sk#25], Inner +Right output [2]: [s_store_sk#24, s_state#25] +Arguments: [ss_store_sk#3], [s_store_sk#24], Inner -(35) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#25, s_state#26] -Arguments: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26], [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26] +(32) CometProject +Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#24, s_state#25] +Arguments: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25], [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25] -(36) Scan parquet spark_catalog.default.item -Output [3]: [i_item_sk#27, i_item_id#28, i_item_desc#29] +(33) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#26, i_item_id#27, i_item_desc#28] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(37) CometFilter -Input [3]: [i_item_sk#27, i_item_id#28, i_item_desc#29] -Condition : isnotnull(i_item_sk#27) +(34) CometFilter +Input [3]: [i_item_sk#26, i_item_id#27, i_item_desc#28] +Condition : isnotnull(i_item_sk#26) -(38) CometBroadcastExchange -Input [3]: [i_item_sk#27, i_item_id#28, i_item_desc#29] -Arguments: [i_item_sk#27, i_item_id#28, i_item_desc#29] +(35) CometBroadcastExchange +Input [3]: [i_item_sk#26, i_item_id#27, i_item_desc#28] +Arguments: [i_item_sk#26, i_item_id#27, i_item_desc#28] -(39) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26] -Right output [3]: [i_item_sk#27, i_item_id#28, i_item_desc#29] -Arguments: [ss_item_sk#1], [i_item_sk#27], Inner +(36) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25] +Right output [3]: [i_item_sk#26, i_item_id#27, i_item_desc#28] +Arguments: [ss_item_sk#1], [i_item_sk#26], Inner -(40) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26, i_item_sk#27, i_item_id#28, i_item_desc#29] -Arguments: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26, i_item_id#28, i_item_desc#29], [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26, i_item_id#28, i_item_desc#29] +(37) CometProject +Input [8]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25, i_item_sk#26, i_item_id#27, i_item_desc#28] +Arguments: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25, i_item_id#27, i_item_desc#28], [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25, i_item_id#27, i_item_desc#28] -(41) CometHashAggregate -Input [6]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26, i_item_id#28, i_item_desc#29] -Keys [3]: [i_item_id#28, i_item_desc#29, s_state#26] +(38) CometHashAggregate +Input [6]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25, i_item_id#27, i_item_desc#28] +Keys [3]: [i_item_id#27, i_item_desc#28, s_state#25] Functions [9]: [partial_count(ss_quantity#5), partial_avg(ss_quantity#5), partial_stddev_samp(cast(ss_quantity#5 as double)), partial_count(sr_return_quantity#11), partial_avg(sr_return_quantity#11), partial_stddev_samp(cast(sr_return_quantity#11 as double)), partial_count(cs_quantity#16), partial_avg(cs_quantity#16), partial_stddev_samp(cast(cs_quantity#16 as double))] -(42) ColumnarToRow [codegen id : 1] -Input [21]: [i_item_id#28, i_item_desc#29, s_state#26, count#30, sum#31, count#32, n#33, avg#34, m2#35, count#36, sum#37, count#38, n#39, avg#40, m2#41, count#42, sum#43, count#44, n#45, avg#46, m2#47] +(39) ColumnarToRow [codegen id : 1] +Input [21]: [i_item_id#27, i_item_desc#28, s_state#25, count#29, sum#30, count#31, n#32, avg#33, m2#34, count#35, sum#36, count#37, n#38, avg#39, m2#40, count#41, sum#42, count#43, n#44, avg#45, m2#46] -(43) Exchange -Input [21]: [i_item_id#28, i_item_desc#29, s_state#26, count#30, sum#31, count#32, n#33, avg#34, m2#35, count#36, sum#37, count#38, n#39, avg#40, m2#41, count#42, sum#43, count#44, n#45, avg#46, m2#47] -Arguments: hashpartitioning(i_item_id#28, i_item_desc#29, s_state#26, 5), ENSURE_REQUIREMENTS, [plan_id=1] +(40) Exchange +Input [21]: [i_item_id#27, i_item_desc#28, s_state#25, count#29, sum#30, count#31, n#32, avg#33, m2#34, count#35, sum#36, count#37, n#38, avg#39, m2#40, count#41, sum#42, count#43, n#44, avg#45, m2#46] +Arguments: hashpartitioning(i_item_id#27, i_item_desc#28, s_state#25, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(44) HashAggregate [codegen id : 2] -Input [21]: [i_item_id#28, i_item_desc#29, s_state#26, count#30, sum#31, count#32, n#33, avg#34, m2#35, count#36, sum#37, count#38, n#39, avg#40, m2#41, count#42, sum#43, count#44, n#45, avg#46, m2#47] -Keys [3]: [i_item_id#28, i_item_desc#29, s_state#26] +(41) HashAggregate [codegen id : 2] +Input [21]: [i_item_id#27, i_item_desc#28, s_state#25, count#29, sum#30, count#31, n#32, avg#33, m2#34, count#35, sum#36, count#37, n#38, avg#39, m2#40, count#41, sum#42, count#43, n#44, avg#45, m2#46] +Keys [3]: [i_item_id#27, i_item_desc#28, s_state#25] Functions [9]: [count(ss_quantity#5), avg(ss_quantity#5), stddev_samp(cast(ss_quantity#5 as double)), count(sr_return_quantity#11), avg(sr_return_quantity#11), stddev_samp(cast(sr_return_quantity#11 as double)), count(cs_quantity#16), avg(cs_quantity#16), stddev_samp(cast(cs_quantity#16 as double))] -Aggregate Attributes [9]: [count(ss_quantity#5)#48, avg(ss_quantity#5)#49, stddev_samp(cast(ss_quantity#5 as double))#50, count(sr_return_quantity#11)#51, avg(sr_return_quantity#11)#52, stddev_samp(cast(sr_return_quantity#11 as double))#53, count(cs_quantity#16)#54, avg(cs_quantity#16)#55, stddev_samp(cast(cs_quantity#16 as double))#56] -Results [15]: [i_item_id#28, i_item_desc#29, s_state#26, count(ss_quantity#5)#48 AS store_sales_quantitycount#57, avg(ss_quantity#5)#49 AS store_sales_quantityave#58, stddev_samp(cast(ss_quantity#5 as double))#50 AS store_sales_quantitystdev#59, (stddev_samp(cast(ss_quantity#5 as double))#50 / avg(ss_quantity#5)#49) AS store_sales_quantitycov#60, count(sr_return_quantity#11)#51 AS as_store_returns_quantitycount#61, avg(sr_return_quantity#11)#52 AS as_store_returns_quantityave#62, stddev_samp(cast(sr_return_quantity#11 as double))#53 AS as_store_returns_quantitystdev#63, (stddev_samp(cast(sr_return_quantity#11 as double))#53 / avg(sr_return_quantity#11)#52) AS store_returns_quantitycov#64, count(cs_quantity#16)#54 AS catalog_sales_quantitycount#65, avg(cs_quantity#16)#55 AS catalog_sales_quantityave#66, (stddev_samp(cast(cs_quantity#16 as double))#56 / avg(cs_quantity#16)#55) AS catalog_sales_quantitystdev#67, (stddev_samp(cast(cs_quantity#16 as double))#56 / avg(cs_quantity#16)#55) AS catalog_sales_quantitycov#68] +Aggregate Attributes [9]: [count(ss_quantity#5)#47, avg(ss_quantity#5)#48, stddev_samp(cast(ss_quantity#5 as double))#49, count(sr_return_quantity#11)#50, avg(sr_return_quantity#11)#51, stddev_samp(cast(sr_return_quantity#11 as double))#52, count(cs_quantity#16)#53, avg(cs_quantity#16)#54, stddev_samp(cast(cs_quantity#16 as double))#55] +Results [15]: [i_item_id#27, i_item_desc#28, s_state#25, count(ss_quantity#5)#47 AS store_sales_quantitycount#56, avg(ss_quantity#5)#48 AS store_sales_quantityave#57, stddev_samp(cast(ss_quantity#5 as double))#49 AS store_sales_quantitystdev#58, (stddev_samp(cast(ss_quantity#5 as double))#49 / avg(ss_quantity#5)#48) AS store_sales_quantitycov#59, count(sr_return_quantity#11)#50 AS as_store_returns_quantitycount#60, avg(sr_return_quantity#11)#51 AS as_store_returns_quantityave#61, stddev_samp(cast(sr_return_quantity#11 as double))#52 AS as_store_returns_quantitystdev#62, (stddev_samp(cast(sr_return_quantity#11 as double))#52 / avg(sr_return_quantity#11)#51) AS store_returns_quantitycov#63, count(cs_quantity#16)#53 AS catalog_sales_quantitycount#64, avg(cs_quantity#16)#54 AS catalog_sales_quantityave#65, (stddev_samp(cast(cs_quantity#16 as double))#55 / avg(cs_quantity#16)#54) AS catalog_sales_quantitystdev#66, (stddev_samp(cast(cs_quantity#16 as double))#55 / avg(cs_quantity#16)#54) AS catalog_sales_quantitycov#67] -(45) TakeOrderedAndProject -Input [15]: [i_item_id#28, i_item_desc#29, s_state#26, store_sales_quantitycount#57, store_sales_quantityave#58, store_sales_quantitystdev#59, store_sales_quantitycov#60, as_store_returns_quantitycount#61, as_store_returns_quantityave#62, as_store_returns_quantitystdev#63, store_returns_quantitycov#64, catalog_sales_quantitycount#65, catalog_sales_quantityave#66, catalog_sales_quantitystdev#67, catalog_sales_quantitycov#68] -Arguments: 100, [i_item_id#28 ASC NULLS FIRST, i_item_desc#29 ASC NULLS FIRST, s_state#26 ASC NULLS FIRST], [i_item_id#28, i_item_desc#29, s_state#26, store_sales_quantitycount#57, store_sales_quantityave#58, store_sales_quantitystdev#59, store_sales_quantitycov#60, as_store_returns_quantitycount#61, as_store_returns_quantityave#62, as_store_returns_quantitystdev#63, store_returns_quantitycov#64, catalog_sales_quantitycount#65, catalog_sales_quantityave#66, catalog_sales_quantitystdev#67, catalog_sales_quantitycov#68] +(42) TakeOrderedAndProject +Input [15]: [i_item_id#27, i_item_desc#28, s_state#25, store_sales_quantitycount#56, store_sales_quantityave#57, store_sales_quantitystdev#58, store_sales_quantitycov#59, as_store_returns_quantitycount#60, as_store_returns_quantityave#61, as_store_returns_quantitystdev#62, store_returns_quantitycov#63, catalog_sales_quantitycount#64, catalog_sales_quantityave#65, catalog_sales_quantitystdev#66, catalog_sales_quantitycov#67] +Arguments: 100, [i_item_id#27 ASC NULLS FIRST, i_item_desc#28 ASC NULLS FIRST, s_state#25 ASC NULLS FIRST], [i_item_id#27, i_item_desc#28, s_state#25, store_sales_quantitycount#56, store_sales_quantityave#57, store_sales_quantitystdev#58, store_sales_quantitycov#59, as_store_returns_quantitycount#60, as_store_returns_quantityave#61, as_store_returns_quantitystdev#62, store_returns_quantitycov#63, catalog_sales_quantitycount#64, catalog_sales_quantityave#65, catalog_sales_quantitystdev#66, catalog_sales_quantitycov#67] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (50) -+- * ColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan parquet spark_catalog.default.date_dim (46) +BroadcastExchange (47) ++- * ColumnarToRow (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) -(46) Scan parquet spark_catalog.default.date_dim +(43) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#19, d_quarter_name#20] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_quarter_name), EqualTo(d_quarter_name,2001Q1), IsNotNull(d_date_sk)] ReadSchema: struct -(47) CometFilter +(44) CometFilter Input [2]: [d_date_sk#19, d_quarter_name#20] Condition : ((isnotnull(d_quarter_name#20) AND (d_quarter_name#20 = 2001Q1)) AND isnotnull(d_date_sk#19)) -(48) CometProject +(45) CometProject Input [2]: [d_date_sk#19, d_quarter_name#20] Arguments: [d_date_sk#19], [d_date_sk#19] -(49) ColumnarToRow [codegen id : 1] +(46) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] -(50) BroadcastExchange +(47) BroadcastExchange Input [1]: [d_date_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (55) -+- * ColumnarToRow (54) - +- CometProject (53) - +- CometFilter (52) - +- CometScan parquet spark_catalog.default.date_dim (51) +BroadcastExchange (52) ++- * ColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) -(51) Scan parquet spark_catalog.default.date_dim +(48) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#21, d_quarter_name#22] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_quarter_name, [2001Q1,2001Q2,2001Q3]), IsNotNull(d_date_sk)] ReadSchema: struct -(52) CometFilter +(49) CometFilter Input [2]: [d_date_sk#21, d_quarter_name#22] Condition : (d_quarter_name#22 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#21)) -(53) CometProject +(50) CometProject Input [2]: [d_date_sk#21, d_quarter_name#22] Arguments: [d_date_sk#21], [d_date_sk#21] -(54) ColumnarToRow [codegen id : 1] +(51) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#21] -(55) BroadcastExchange +(52) BroadcastExchange Input [1]: [d_date_sk#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt index b522843d6..77aba376e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt @@ -54,13 +54,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s CometProject [d_date_sk] CometFilter [d_quarter_name,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange #8 - CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange #9 + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange #8 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange #10 + CometBroadcastExchange #9 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt index 814b158c4..9c5d52d55 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt @@ -443,26 +443,24 @@ Input [2]: [d_date_sk#10, d_date#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] Subquery:3 Hosting operator id = 39 Hosting Expression = Subquery scalar-subquery#29, [id=#30] -* HashAggregate (97) -+- Exchange (96) - +- * HashAggregate (95) - +- * HashAggregate (94) - +- Exchange (93) - +- * ColumnarToRow (92) - +- CometHashAggregate (91) - +- CometProject (90) - +- CometBroadcastHashJoin (89) - :- CometProject (84) - : +- CometBroadcastHashJoin (83) +* HashAggregate (95) ++- Exchange (94) + +- * HashAggregate (93) + +- * HashAggregate (92) + +- Exchange (91) + +- * ColumnarToRow (90) + +- CometHashAggregate (89) + +- CometProject (88) + +- CometBroadcastHashJoin (87) + :- CometProject (82) + : +- CometBroadcastHashJoin (81) : :- CometFilter (79) : : +- CometScan parquet spark_catalog.default.store_sales (78) - : +- CometBroadcastExchange (82) - : +- CometFilter (81) - : +- CometScan parquet spark_catalog.default.customer (80) - +- CometBroadcastExchange (88) - +- CometProject (87) - +- CometFilter (86) - +- CometScan parquet spark_catalog.default.date_dim (85) + : +- ReusedExchange (80) + +- CometBroadcastExchange (86) + +- CometProject (85) + +- CometFilter (84) + +- CometScan parquet spark_catalog.default.date_dim (83) (78) Scan parquet spark_catalog.default.store_sales @@ -477,89 +475,77 @@ ReadSchema: struct - -(81) CometFilter -Input [1]: [c_customer_sk#54] -Condition : isnotnull(c_customer_sk#54) - -(82) CometBroadcastExchange -Input [1]: [c_customer_sk#54] -Arguments: [c_customer_sk#54] -(83) CometBroadcastHashJoin +(81) CometBroadcastHashJoin Left output [4]: [ss_customer_sk#49, ss_quantity#50, ss_sales_price#51, ss_sold_date_sk#52] Right output [1]: [c_customer_sk#54] Arguments: [ss_customer_sk#49], [c_customer_sk#54], Inner -(84) CometProject +(82) CometProject Input [5]: [ss_customer_sk#49, ss_quantity#50, ss_sales_price#51, ss_sold_date_sk#52, c_customer_sk#54] Arguments: [ss_quantity#50, ss_sales_price#51, ss_sold_date_sk#52, c_customer_sk#54], [ss_quantity#50, ss_sales_price#51, ss_sold_date_sk#52, c_customer_sk#54] -(85) Scan parquet spark_catalog.default.date_dim +(83) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#55, d_year#56] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(86) CometFilter +(84) CometFilter Input [2]: [d_date_sk#55, d_year#56] Condition : (d_year#56 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#55)) -(87) CometProject +(85) CometProject Input [2]: [d_date_sk#55, d_year#56] Arguments: [d_date_sk#55], [d_date_sk#55] -(88) CometBroadcastExchange +(86) CometBroadcastExchange Input [1]: [d_date_sk#55] Arguments: [d_date_sk#55] -(89) CometBroadcastHashJoin +(87) CometBroadcastHashJoin Left output [4]: [ss_quantity#50, ss_sales_price#51, ss_sold_date_sk#52, c_customer_sk#54] Right output [1]: [d_date_sk#55] Arguments: [ss_sold_date_sk#52], [d_date_sk#55], Inner -(90) CometProject +(88) CometProject Input [5]: [ss_quantity#50, ss_sales_price#51, ss_sold_date_sk#52, c_customer_sk#54, d_date_sk#55] Arguments: [ss_quantity#50, ss_sales_price#51, c_customer_sk#54], [ss_quantity#50, ss_sales_price#51, c_customer_sk#54] -(91) CometHashAggregate +(89) CometHashAggregate Input [3]: [ss_quantity#50, ss_sales_price#51, c_customer_sk#54] Keys [1]: [c_customer_sk#54] Functions [1]: [partial_sum((cast(ss_quantity#50 as decimal(10,0)) * ss_sales_price#51))] -(92) ColumnarToRow [codegen id : 1] +(90) ColumnarToRow [codegen id : 1] Input [3]: [c_customer_sk#54, sum#57, isEmpty#58] -(93) Exchange +(91) Exchange Input [3]: [c_customer_sk#54, sum#57, isEmpty#58] Arguments: hashpartitioning(c_customer_sk#54, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(94) HashAggregate [codegen id : 2] +(92) HashAggregate [codegen id : 2] Input [3]: [c_customer_sk#54, sum#57, isEmpty#58] Keys [1]: [c_customer_sk#54] Functions [1]: [sum((cast(ss_quantity#50 as decimal(10,0)) * ss_sales_price#51))] Aggregate Attributes [1]: [sum((cast(ss_quantity#50 as decimal(10,0)) * ss_sales_price#51))#59] Results [1]: [sum((cast(ss_quantity#50 as decimal(10,0)) * ss_sales_price#51))#59 AS csales#60] -(95) HashAggregate [codegen id : 2] +(93) HashAggregate [codegen id : 2] Input [1]: [csales#60] Keys: [] Functions [1]: [partial_max(csales#60)] Aggregate Attributes [1]: [max#61] Results [1]: [max#62] -(96) Exchange +(94) Exchange Input [1]: [max#62] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] -(97) HashAggregate [codegen id : 3] +(95) HashAggregate [codegen id : 3] Input [1]: [max#62] Keys: [] Functions [1]: [max(csales#60)] @@ -567,32 +553,32 @@ Aggregate Attributes [1]: [max(csales#60)#63] Results [1]: [max(csales#60)#63 AS tpcds_cmax#64] Subquery:4 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#52 IN dynamicpruning#53 -BroadcastExchange (102) -+- * ColumnarToRow (101) - +- CometProject (100) - +- CometFilter (99) - +- CometScan parquet spark_catalog.default.date_dim (98) +BroadcastExchange (100) ++- * ColumnarToRow (99) + +- CometProject (98) + +- CometFilter (97) + +- CometScan parquet spark_catalog.default.date_dim (96) -(98) Scan parquet spark_catalog.default.date_dim +(96) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#55, d_year#56] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(99) CometFilter +(97) CometFilter Input [2]: [d_date_sk#55, d_year#56] Condition : (d_year#56 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#55)) -(100) CometProject +(98) CometProject Input [2]: [d_date_sk#55, d_year#56] Arguments: [d_date_sk#55], [d_date_sk#55] -(101) ColumnarToRow [codegen id : 1] +(99) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#55] -(102) BroadcastExchange +(100) BroadcastExchange Input [1]: [d_date_sk#55] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt index 226170e4a..f818fd25f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt @@ -96,10 +96,8 @@ WholeStageCodegen (18) CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #14 - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk] - CometBroadcastExchange #15 + ReusedExchange [c_customer_sk] #10 + CometBroadcastExchange #14 CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -129,7 +127,7 @@ WholeStageCodegen (18) WholeStageCodegen (12) Sort [ws_bill_customer_sk] InputAdapter - Exchange [ws_bill_customer_sk] #16 + Exchange [ws_bill_customer_sk] #15 WholeStageCodegen (11) Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] BroadcastHashJoin [ws_item_sk,item_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 678e2d980..6411f9eab 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 @@ -565,26 +565,24 @@ Input [2]: [d_date_sk#10, d_date#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] Subquery:3 Hosting operator id = 40 Hosting Expression = Subquery scalar-subquery#29, [id=#30] -* HashAggregate (118) -+- Exchange (117) - +- * HashAggregate (116) - +- * HashAggregate (115) - +- Exchange (114) - +- * ColumnarToRow (113) - +- CometHashAggregate (112) - +- CometProject (111) - +- CometBroadcastHashJoin (110) - :- CometProject (105) - : +- CometBroadcastHashJoin (104) +* HashAggregate (116) ++- Exchange (115) + +- * HashAggregate (114) + +- * HashAggregate (113) + +- Exchange (112) + +- * ColumnarToRow (111) + +- CometHashAggregate (110) + +- CometProject (109) + +- CometBroadcastHashJoin (108) + :- CometProject (103) + : +- CometBroadcastHashJoin (102) : :- CometFilter (100) : : +- CometScan parquet spark_catalog.default.store_sales (99) - : +- CometBroadcastExchange (103) - : +- CometFilter (102) - : +- CometScan parquet spark_catalog.default.customer (101) - +- CometBroadcastExchange (109) - +- CometProject (108) - +- CometFilter (107) - +- CometScan parquet spark_catalog.default.date_dim (106) + : +- ReusedExchange (101) + +- CometBroadcastExchange (107) + +- CometProject (106) + +- CometFilter (105) + +- CometScan parquet spark_catalog.default.date_dim (104) (99) Scan parquet spark_catalog.default.store_sales @@ -599,89 +597,77 @@ ReadSchema: struct - -(102) CometFilter -Input [1]: [c_customer_sk#64] -Condition : isnotnull(c_customer_sk#64) - -(103) CometBroadcastExchange -Input [1]: [c_customer_sk#64] -Arguments: [c_customer_sk#64] -(104) CometBroadcastHashJoin +(102) CometBroadcastHashJoin Left output [4]: [ss_customer_sk#59, ss_quantity#60, ss_sales_price#61, ss_sold_date_sk#62] Right output [1]: [c_customer_sk#64] Arguments: [ss_customer_sk#59], [c_customer_sk#64], Inner -(105) CometProject +(103) CometProject Input [5]: [ss_customer_sk#59, ss_quantity#60, ss_sales_price#61, ss_sold_date_sk#62, c_customer_sk#64] Arguments: [ss_quantity#60, ss_sales_price#61, ss_sold_date_sk#62, c_customer_sk#64], [ss_quantity#60, ss_sales_price#61, ss_sold_date_sk#62, c_customer_sk#64] -(106) Scan parquet spark_catalog.default.date_dim +(104) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#65, d_year#66] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(107) CometFilter +(105) CometFilter Input [2]: [d_date_sk#65, d_year#66] Condition : (d_year#66 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#65)) -(108) CometProject +(106) CometProject Input [2]: [d_date_sk#65, d_year#66] Arguments: [d_date_sk#65], [d_date_sk#65] -(109) CometBroadcastExchange +(107) CometBroadcastExchange Input [1]: [d_date_sk#65] Arguments: [d_date_sk#65] -(110) CometBroadcastHashJoin +(108) CometBroadcastHashJoin Left output [4]: [ss_quantity#60, ss_sales_price#61, ss_sold_date_sk#62, c_customer_sk#64] Right output [1]: [d_date_sk#65] Arguments: [ss_sold_date_sk#62], [d_date_sk#65], Inner -(111) CometProject +(109) CometProject Input [5]: [ss_quantity#60, ss_sales_price#61, ss_sold_date_sk#62, c_customer_sk#64, d_date_sk#65] Arguments: [ss_quantity#60, ss_sales_price#61, c_customer_sk#64], [ss_quantity#60, ss_sales_price#61, c_customer_sk#64] -(112) CometHashAggregate +(110) CometHashAggregate Input [3]: [ss_quantity#60, ss_sales_price#61, c_customer_sk#64] Keys [1]: [c_customer_sk#64] Functions [1]: [partial_sum((cast(ss_quantity#60 as decimal(10,0)) * ss_sales_price#61))] -(113) ColumnarToRow [codegen id : 1] +(111) ColumnarToRow [codegen id : 1] Input [3]: [c_customer_sk#64, sum#67, isEmpty#68] -(114) Exchange +(112) Exchange Input [3]: [c_customer_sk#64, sum#67, isEmpty#68] Arguments: hashpartitioning(c_customer_sk#64, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(115) HashAggregate [codegen id : 2] +(113) HashAggregate [codegen id : 2] Input [3]: [c_customer_sk#64, sum#67, isEmpty#68] Keys [1]: [c_customer_sk#64] Functions [1]: [sum((cast(ss_quantity#60 as decimal(10,0)) * ss_sales_price#61))] Aggregate Attributes [1]: [sum((cast(ss_quantity#60 as decimal(10,0)) * ss_sales_price#61))#69] Results [1]: [sum((cast(ss_quantity#60 as decimal(10,0)) * ss_sales_price#61))#69 AS csales#70] -(116) HashAggregate [codegen id : 2] +(114) HashAggregate [codegen id : 2] Input [1]: [csales#70] Keys: [] Functions [1]: [partial_max(csales#70)] Aggregate Attributes [1]: [max#71] Results [1]: [max#72] -(117) Exchange +(115) Exchange Input [1]: [max#72] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] -(118) HashAggregate [codegen id : 3] +(116) HashAggregate [codegen id : 3] Input [1]: [max#72] Keys: [] Functions [1]: [max(csales#70)] @@ -689,32 +675,32 @@ Aggregate Attributes [1]: [max(csales#70)#73] Results [1]: [max(csales#70)#73 AS tpcds_cmax#74] Subquery:4 Hosting operator id = 99 Hosting Expression = ss_sold_date_sk#62 IN dynamicpruning#63 -BroadcastExchange (123) -+- * ColumnarToRow (122) - +- CometProject (121) - +- CometFilter (120) - +- CometScan parquet spark_catalog.default.date_dim (119) +BroadcastExchange (121) ++- * ColumnarToRow (120) + +- CometProject (119) + +- CometFilter (118) + +- CometScan parquet spark_catalog.default.date_dim (117) -(119) Scan parquet spark_catalog.default.date_dim +(117) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#65, d_year#66] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(120) CometFilter +(118) CometFilter Input [2]: [d_date_sk#65, d_year#66] Condition : (d_year#66 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#65)) -(121) CometProject +(119) CometProject Input [2]: [d_date_sk#65, d_year#66] Arguments: [d_date_sk#65], [d_date_sk#65] -(122) ColumnarToRow [codegen id : 1] +(120) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#65] -(123) BroadcastExchange +(121) BroadcastExchange Input [1]: [d_date_sk#65] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] 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 2c8399b3a..54ee3dbde 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 @@ -97,10 +97,8 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #14 - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk] - CometBroadcastExchange #15 + ReusedExchange [c_customer_sk] #10 + CometBroadcastExchange #14 CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -120,14 +118,14 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] CometFilter [c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk] InputAdapter - BroadcastExchange #16 + BroadcastExchange #15 WholeStageCodegen (11) SortMergeJoin [c_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (8) Sort [c_customer_sk] InputAdapter - Exchange [c_customer_sk] #17 + Exchange [c_customer_sk] #16 WholeStageCodegen (7) ColumnarToRow InputAdapter @@ -147,7 +145,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (28) 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] #18 + Exchange [c_last_name,c_first_name] #17 WholeStageCodegen (27) 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] @@ -159,7 +157,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (18) Sort [ws_bill_customer_sk] InputAdapter - Exchange [ws_bill_customer_sk] #19 + Exchange [ws_bill_customer_sk] #18 WholeStageCodegen (17) Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] BroadcastHashJoin [ws_item_sk,item_sk] @@ -180,6 +178,6 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] InputAdapter ReusedExchange [c_customer_sk,sum,isEmpty] #9 InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #16 + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #15 InputAdapter ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt index 57f509acd..bba2d61d9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt @@ -1,15 +1,15 @@ == Physical Plan == -TakeOrderedAndProject (45) -+- * HashAggregate (44) - +- Exchange (43) - +- * ColumnarToRow (42) - +- CometHashAggregate (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (35) - : +- CometBroadcastHashJoin (34) - : :- CometProject (30) - : : +- CometBroadcastHashJoin (29) +TakeOrderedAndProject (42) ++- * HashAggregate (41) + +- Exchange (40) + +- * ColumnarToRow (39) + +- CometHashAggregate (38) + +- CometProject (37) + +- CometBroadcastHashJoin (36) + :- CometProject (32) + : +- CometBroadcastHashJoin (31) + : :- CometProject (27) + : : +- CometBroadcastHashJoin (26) : : :- CometProject (24) : : : +- CometBroadcastHashJoin (23) : : : :- CometProject (18) @@ -34,16 +34,13 @@ TakeOrderedAndProject (45) : : : +- CometProject (21) : : : +- CometFilter (20) : : : +- CometScan parquet spark_catalog.default.date_dim (19) - : : +- CometBroadcastExchange (28) - : : +- CometProject (27) - : : +- CometFilter (26) - : : +- CometScan parquet spark_catalog.default.date_dim (25) - : +- CometBroadcastExchange (33) - : +- CometFilter (32) - : +- CometScan parquet spark_catalog.default.store (31) - +- CometBroadcastExchange (38) - +- CometFilter (37) - +- CometScan parquet spark_catalog.default.item (36) + : : +- ReusedExchange (25) + : +- CometBroadcastExchange (30) + : +- CometFilter (29) + : +- CometScan parquet spark_catalog.default.store (28) + +- CometBroadcastExchange (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.item (33) (1) Scan parquet spark_catalog.default.store_sales @@ -164,164 +161,148 @@ Arguments: [sr_returned_date_sk#12], [d_date_sk#22], Inner Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#22] Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17] -(25) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#25, d_year#26, d_moy#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [3]: [d_date_sk#25, d_year#26, d_moy#27] -Condition : (((((isnotnull(d_moy#27) AND isnotnull(d_year#26)) AND (d_moy#27 >= 4)) AND (d_moy#27 <= 10)) AND (d_year#26 = 2001)) AND isnotnull(d_date_sk#25)) +(25) ReusedExchange [Reuses operator id: 22] +Output [1]: [d_date_sk#25] -(27) CometProject -Input [3]: [d_date_sk#25, d_year#26, d_moy#27] -Arguments: [d_date_sk#25], [d_date_sk#25] - -(28) CometBroadcastExchange -Input [1]: [d_date_sk#25] -Arguments: [d_date_sk#25] - -(29) CometBroadcastHashJoin +(26) CometBroadcastHashJoin Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17] Right output [1]: [d_date_sk#25] Arguments: [cs_sold_date_sk#17], [d_date_sk#25], Inner -(30) CometProject +(27) CometProject Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#25] Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16] -(31) Scan parquet spark_catalog.default.store -Output [3]: [s_store_sk#28, s_store_id#29, s_store_name#30] +(28) Scan parquet spark_catalog.default.store +Output [3]: [s_store_sk#26, s_store_id#27, s_store_name#28] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(32) CometFilter -Input [3]: [s_store_sk#28, s_store_id#29, s_store_name#30] -Condition : isnotnull(s_store_sk#28) +(29) CometFilter +Input [3]: [s_store_sk#26, s_store_id#27, s_store_name#28] +Condition : isnotnull(s_store_sk#26) -(33) CometBroadcastExchange -Input [3]: [s_store_sk#28, s_store_id#29, s_store_name#30] -Arguments: [s_store_sk#28, s_store_id#29, s_store_name#30] +(30) CometBroadcastExchange +Input [3]: [s_store_sk#26, s_store_id#27, s_store_name#28] +Arguments: [s_store_sk#26, s_store_id#27, s_store_name#28] -(34) CometBroadcastHashJoin +(31) CometBroadcastHashJoin Left output [5]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16] -Right output [3]: [s_store_sk#28, s_store_id#29, s_store_name#30] -Arguments: [ss_store_sk#3], [s_store_sk#28], Inner +Right output [3]: [s_store_sk#26, s_store_id#27, s_store_name#28] +Arguments: [ss_store_sk#3], [s_store_sk#26], Inner -(35) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_sk#28, s_store_id#29, s_store_name#30] -Arguments: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#30], [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#30] +(32) CometProject +Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_sk#26, s_store_id#27, s_store_name#28] +Arguments: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28], [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28] -(36) Scan parquet spark_catalog.default.item -Output [3]: [i_item_sk#31, i_item_id#32, i_item_desc#33] +(33) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#29, i_item_id#30, i_item_desc#31] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(37) CometFilter -Input [3]: [i_item_sk#31, i_item_id#32, i_item_desc#33] -Condition : isnotnull(i_item_sk#31) +(34) CometFilter +Input [3]: [i_item_sk#29, i_item_id#30, i_item_desc#31] +Condition : isnotnull(i_item_sk#29) -(38) CometBroadcastExchange -Input [3]: [i_item_sk#31, i_item_id#32, i_item_desc#33] -Arguments: [i_item_sk#31, i_item_id#32, i_item_desc#33] +(35) CometBroadcastExchange +Input [3]: [i_item_sk#29, i_item_id#30, i_item_desc#31] +Arguments: [i_item_sk#29, i_item_id#30, i_item_desc#31] -(39) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#30] -Right output [3]: [i_item_sk#31, i_item_id#32, i_item_desc#33] -Arguments: [ss_item_sk#1], [i_item_sk#31], Inner +(36) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28] +Right output [3]: [i_item_sk#29, i_item_id#30, i_item_desc#31] +Arguments: [ss_item_sk#1], [i_item_sk#29], Inner -(40) CometProject -Input [9]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#30, i_item_sk#31, i_item_id#32, i_item_desc#33] -Arguments: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#30, i_item_id#32, i_item_desc#33], [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#30, i_item_id#32, i_item_desc#33] +(37) CometProject +Input [9]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28, i_item_sk#29, i_item_id#30, i_item_desc#31] +Arguments: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28, i_item_id#30, i_item_desc#31], [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28, i_item_id#30, i_item_desc#31] -(41) CometHashAggregate -Input [7]: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#30, i_item_id#32, i_item_desc#33] -Keys [4]: [i_item_id#32, i_item_desc#33, s_store_id#29, s_store_name#30] +(38) CometHashAggregate +Input [7]: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28, i_item_id#30, i_item_desc#31] +Keys [4]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28] Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#5)), partial_sum(UnscaledValue(sr_net_loss#11)), partial_sum(UnscaledValue(cs_net_profit#16))] -(42) ColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#32, i_item_desc#33, s_store_id#29, s_store_name#30, sum#34, sum#35, sum#36] +(39) ColumnarToRow [codegen id : 1] +Input [7]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, sum#32, sum#33, sum#34] -(43) Exchange -Input [7]: [i_item_id#32, i_item_desc#33, s_store_id#29, s_store_name#30, sum#34, sum#35, sum#36] -Arguments: hashpartitioning(i_item_id#32, i_item_desc#33, s_store_id#29, s_store_name#30, 5), ENSURE_REQUIREMENTS, [plan_id=1] +(40) Exchange +Input [7]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, sum#32, sum#33, sum#34] +Arguments: hashpartitioning(i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(44) HashAggregate [codegen id : 2] -Input [7]: [i_item_id#32, i_item_desc#33, s_store_id#29, s_store_name#30, sum#34, sum#35, sum#36] -Keys [4]: [i_item_id#32, i_item_desc#33, s_store_id#29, s_store_name#30] +(41) HashAggregate [codegen id : 2] +Input [7]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, sum#32, sum#33, sum#34] +Keys [4]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28] Functions [3]: [sum(UnscaledValue(ss_net_profit#5)), sum(UnscaledValue(sr_net_loss#11)), sum(UnscaledValue(cs_net_profit#16))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#5))#37, sum(UnscaledValue(sr_net_loss#11))#38, sum(UnscaledValue(cs_net_profit#16))#39] -Results [7]: [i_item_id#32, i_item_desc#33, s_store_id#29, s_store_name#30, MakeDecimal(sum(UnscaledValue(ss_net_profit#5))#37,17,2) AS store_sales_profit#40, MakeDecimal(sum(UnscaledValue(sr_net_loss#11))#38,17,2) AS store_returns_loss#41, MakeDecimal(sum(UnscaledValue(cs_net_profit#16))#39,17,2) AS catalog_sales_profit#42] +Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#5))#35, sum(UnscaledValue(sr_net_loss#11))#36, sum(UnscaledValue(cs_net_profit#16))#37] +Results [7]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, MakeDecimal(sum(UnscaledValue(ss_net_profit#5))#35,17,2) AS store_sales_profit#38, MakeDecimal(sum(UnscaledValue(sr_net_loss#11))#36,17,2) AS store_returns_loss#39, MakeDecimal(sum(UnscaledValue(cs_net_profit#16))#37,17,2) AS catalog_sales_profit#40] -(45) TakeOrderedAndProject -Input [7]: [i_item_id#32, i_item_desc#33, s_store_id#29, s_store_name#30, store_sales_profit#40, store_returns_loss#41, catalog_sales_profit#42] -Arguments: 100, [i_item_id#32 ASC NULLS FIRST, i_item_desc#33 ASC NULLS FIRST, s_store_id#29 ASC NULLS FIRST, s_store_name#30 ASC NULLS FIRST], [i_item_id#32, i_item_desc#33, s_store_id#29, s_store_name#30, store_sales_profit#40, store_returns_loss#41, catalog_sales_profit#42] +(42) TakeOrderedAndProject +Input [7]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, store_sales_profit#38, store_returns_loss#39, catalog_sales_profit#40] +Arguments: 100, [i_item_id#30 ASC NULLS FIRST, i_item_desc#31 ASC NULLS FIRST, s_store_id#27 ASC NULLS FIRST, s_store_name#28 ASC NULLS FIRST], [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, store_sales_profit#38, store_returns_loss#39, catalog_sales_profit#40] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (50) -+- * ColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan parquet spark_catalog.default.date_dim (46) +BroadcastExchange (47) ++- * ColumnarToRow (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) -(46) Scan parquet spark_catalog.default.date_dim +(43) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#19, d_year#20, d_moy#21] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,4), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(47) CometFilter +(44) CometFilter Input [3]: [d_date_sk#19, d_year#20, d_moy#21] Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 4)) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#19)) -(48) CometProject +(45) CometProject Input [3]: [d_date_sk#19, d_year#20, d_moy#21] Arguments: [d_date_sk#19], [d_date_sk#19] -(49) ColumnarToRow [codegen id : 1] +(46) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] -(50) BroadcastExchange +(47) BroadcastExchange Input [1]: [d_date_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (55) -+- * ColumnarToRow (54) - +- CometProject (53) - +- CometFilter (52) - +- CometScan parquet spark_catalog.default.date_dim (51) +BroadcastExchange (52) ++- * ColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) -(51) Scan parquet spark_catalog.default.date_dim +(48) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#22, d_year#23, d_moy#24] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(52) CometFilter +(49) CometFilter Input [3]: [d_date_sk#22, d_year#23, d_moy#24] Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 4)) AND (d_moy#24 <= 10)) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) -(53) CometProject +(50) CometProject Input [3]: [d_date_sk#22, d_year#23, d_moy#24] Arguments: [d_date_sk#22], [d_date_sk#22] -(54) ColumnarToRow [codegen id : 1] +(51) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(55) BroadcastExchange +(52) BroadcastExchange Input [1]: [d_date_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt index 8ed29f6e6..74a5e8777 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt @@ -54,13 +54,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales CometProject [d_date_sk] CometFilter [d_moy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #8 - CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #9 + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange #8 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange #10 + CometBroadcastExchange #9 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt index addad5559..83b71232a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt @@ -1,14 +1,14 @@ == Physical Plan == -* Sort (111) -+- Exchange (110) - +- * Project (109) - +- * BroadcastHashJoin Inner BuildRight (108) - :- * Project (90) - : +- * BroadcastHashJoin Inner BuildRight (89) - : :- * BroadcastHashJoin Inner BuildRight (71) - : : :- * Project (53) - : : : +- * BroadcastHashJoin Inner BuildRight (52) - : : : :- * BroadcastHashJoin Inner BuildRight (34) +* Sort (95) ++- Exchange (94) + +- * Project (93) + +- * BroadcastHashJoin Inner BuildRight (92) + :- * Project (78) + : +- * BroadcastHashJoin Inner BuildRight (77) + : :- * BroadcastHashJoin Inner BuildRight (63) + : : :- * Project (49) + : : : +- * BroadcastHashJoin Inner BuildRight (48) + : : : :- * BroadcastHashJoin Inner BuildRight (32) : : : : :- * HashAggregate (16) : : : : : +- Exchange (15) : : : : : +- * ColumnarToRow (14) @@ -25,13 +25,13 @@ : : : : : +- CometBroadcastExchange (10) : : : : : +- CometFilter (9) : : : : : +- CometScan parquet spark_catalog.default.customer_address (8) - : : : : +- BroadcastExchange (33) - : : : : +- * HashAggregate (32) - : : : : +- Exchange (31) - : : : : +- * ColumnarToRow (30) - : : : : +- CometHashAggregate (29) - : : : : +- CometProject (28) - : : : : +- CometBroadcastHashJoin (27) + : : : : +- BroadcastExchange (31) + : : : : +- * HashAggregate (30) + : : : : +- Exchange (29) + : : : : +- * ColumnarToRow (28) + : : : : +- CometHashAggregate (27) + : : : : +- CometProject (26) + : : : : +- CometBroadcastHashJoin (25) : : : : :- CometProject (23) : : : : : +- CometBroadcastHashJoin (22) : : : : : :- CometFilter (18) @@ -39,77 +39,61 @@ : : : : : +- CometBroadcastExchange (21) : : : : : +- CometFilter (20) : : : : : +- CometScan parquet spark_catalog.default.date_dim (19) - : : : : +- CometBroadcastExchange (26) - : : : : +- CometFilter (25) - : : : : +- CometScan parquet spark_catalog.default.customer_address (24) - : : : +- BroadcastExchange (51) - : : : +- * HashAggregate (50) - : : : +- Exchange (49) - : : : +- * ColumnarToRow (48) - : : : +- CometHashAggregate (47) - : : : +- CometProject (46) - : : : +- CometBroadcastHashJoin (45) - : : : :- CometProject (41) - : : : : +- CometBroadcastHashJoin (40) - : : : : :- CometFilter (36) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (35) - : : : : +- CometBroadcastExchange (39) - : : : : +- CometFilter (38) - : : : : +- CometScan parquet spark_catalog.default.date_dim (37) - : : : +- CometBroadcastExchange (44) - : : : +- CometFilter (43) - : : : +- CometScan parquet spark_catalog.default.customer_address (42) - : : +- BroadcastExchange (70) - : : +- * HashAggregate (69) - : : +- Exchange (68) - : : +- * ColumnarToRow (67) - : : +- CometHashAggregate (66) - : : +- CometProject (65) - : : +- CometBroadcastHashJoin (64) - : : :- CometProject (60) - : : : +- CometBroadcastHashJoin (59) - : : : :- CometFilter (55) - : : : : +- CometScan parquet spark_catalog.default.web_sales (54) - : : : +- CometBroadcastExchange (58) - : : : +- CometFilter (57) - : : : +- CometScan parquet spark_catalog.default.date_dim (56) - : : +- CometBroadcastExchange (63) - : : +- CometFilter (62) - : : +- CometScan parquet spark_catalog.default.customer_address (61) - : +- BroadcastExchange (88) - : +- * HashAggregate (87) - : +- Exchange (86) - : +- * ColumnarToRow (85) - : +- CometHashAggregate (84) - : +- CometProject (83) - : +- CometBroadcastHashJoin (82) - : :- CometProject (78) - : : +- CometBroadcastHashJoin (77) - : : :- CometFilter (73) - : : : +- CometScan parquet spark_catalog.default.web_sales (72) - : : +- CometBroadcastExchange (76) - : : +- CometFilter (75) - : : +- CometScan parquet spark_catalog.default.date_dim (74) - : +- CometBroadcastExchange (81) - : +- CometFilter (80) - : +- CometScan parquet spark_catalog.default.customer_address (79) - +- BroadcastExchange (107) - +- * HashAggregate (106) - +- Exchange (105) - +- * ColumnarToRow (104) - +- CometHashAggregate (103) - +- CometProject (102) - +- CometBroadcastHashJoin (101) - :- CometProject (97) - : +- CometBroadcastHashJoin (96) - : :- CometFilter (92) - : : +- CometScan parquet spark_catalog.default.web_sales (91) - : +- CometBroadcastExchange (95) - : +- CometFilter (94) - : +- CometScan parquet spark_catalog.default.date_dim (93) - +- CometBroadcastExchange (100) - +- CometFilter (99) - +- CometScan parquet spark_catalog.default.customer_address (98) + : : : : +- ReusedExchange (24) + : : : +- BroadcastExchange (47) + : : : +- * HashAggregate (46) + : : : +- Exchange (45) + : : : +- * ColumnarToRow (44) + : : : +- CometHashAggregate (43) + : : : +- CometProject (42) + : : : +- CometBroadcastHashJoin (41) + : : : :- CometProject (39) + : : : : +- CometBroadcastHashJoin (38) + : : : : :- CometFilter (34) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (33) + : : : : +- CometBroadcastExchange (37) + : : : : +- CometFilter (36) + : : : : +- CometScan parquet spark_catalog.default.date_dim (35) + : : : +- ReusedExchange (40) + : : +- BroadcastExchange (62) + : : +- * HashAggregate (61) + : : +- Exchange (60) + : : +- * ColumnarToRow (59) + : : +- CometHashAggregate (58) + : : +- CometProject (57) + : : +- CometBroadcastHashJoin (56) + : : :- CometProject (54) + : : : +- CometBroadcastHashJoin (53) + : : : :- CometFilter (51) + : : : : +- CometScan parquet spark_catalog.default.web_sales (50) + : : : +- ReusedExchange (52) + : : +- ReusedExchange (55) + : +- BroadcastExchange (76) + : +- * HashAggregate (75) + : +- Exchange (74) + : +- * ColumnarToRow (73) + : +- CometHashAggregate (72) + : +- CometProject (71) + : +- CometBroadcastHashJoin (70) + : :- CometProject (68) + : : +- CometBroadcastHashJoin (67) + : : :- CometFilter (65) + : : : +- CometScan parquet spark_catalog.default.web_sales (64) + : : +- ReusedExchange (66) + : +- ReusedExchange (69) + +- BroadcastExchange (91) + +- * HashAggregate (90) + +- Exchange (89) + +- * ColumnarToRow (88) + +- CometHashAggregate (87) + +- CometProject (86) + +- CometBroadcastHashJoin (85) + :- CometProject (83) + : +- CometBroadcastHashJoin (82) + : :- CometFilter (80) + : : +- CometScan parquet spark_catalog.default.web_sales (79) + : +- ReusedExchange (81) + +- ReusedExchange (84) (1) Scan parquet spark_catalog.default.store_sales @@ -227,60 +211,48 @@ Arguments: [ss_sold_date_sk#15], [d_date_sk#17], Inner Input [6]: [ss_addr_sk#13, ss_ext_sales_price#14, ss_sold_date_sk#15, d_date_sk#17, d_year#18, d_qoy#19] Arguments: [ss_addr_sk#13, ss_ext_sales_price#14, d_year#18, d_qoy#19], [ss_addr_sk#13, ss_ext_sales_price#14, d_year#18, d_qoy#19] -(24) Scan parquet spark_catalog.default.customer_address +(24) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#20, ca_county#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county)] -ReadSchema: struct -(25) CometFilter -Input [2]: [ca_address_sk#20, ca_county#21] -Condition : (isnotnull(ca_address_sk#20) AND isnotnull(ca_county#21)) - -(26) CometBroadcastExchange -Input [2]: [ca_address_sk#20, ca_county#21] -Arguments: [ca_address_sk#20, ca_county#21] - -(27) CometBroadcastHashJoin +(25) CometBroadcastHashJoin Left output [4]: [ss_addr_sk#13, ss_ext_sales_price#14, d_year#18, d_qoy#19] Right output [2]: [ca_address_sk#20, ca_county#21] Arguments: [ss_addr_sk#13], [ca_address_sk#20], Inner -(28) CometProject +(26) CometProject Input [6]: [ss_addr_sk#13, ss_ext_sales_price#14, d_year#18, d_qoy#19, ca_address_sk#20, ca_county#21] Arguments: [ss_ext_sales_price#14, d_year#18, d_qoy#19, ca_county#21], [ss_ext_sales_price#14, d_year#18, d_qoy#19, ca_county#21] -(29) CometHashAggregate +(27) CometHashAggregate Input [4]: [ss_ext_sales_price#14, d_year#18, d_qoy#19, ca_county#21] Keys [3]: [ca_county#21, d_qoy#19, d_year#18] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#14))] -(30) ColumnarToRow [codegen id : 2] +(28) ColumnarToRow [codegen id : 2] Input [4]: [ca_county#21, d_qoy#19, d_year#18, sum#22] -(31) Exchange +(29) Exchange Input [4]: [ca_county#21, d_qoy#19, d_year#18, sum#22] Arguments: hashpartitioning(ca_county#21, d_qoy#19, d_year#18, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(32) HashAggregate [codegen id : 3] +(30) HashAggregate [codegen id : 3] Input [4]: [ca_county#21, d_qoy#19, d_year#18, sum#22] Keys [3]: [ca_county#21, d_qoy#19, d_year#18] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#14))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#14))#11] Results [2]: [ca_county#21, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#14))#11,17,2) AS store_sales#23] -(33) BroadcastExchange +(31) BroadcastExchange Input [2]: [ca_county#21, store_sales#23] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] -(34) BroadcastHashJoin [codegen id : 12] +(32) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ca_county#9] Right keys [1]: [ca_county#21] Join type: Inner Join condition: None -(35) Scan parquet spark_catalog.default.store_sales +(33) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_addr_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] Batched: true Location: InMemoryFileIndex [] @@ -288,92 +260,80 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct -(36) CometFilter +(34) CometFilter Input [3]: [ss_addr_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] Condition : isnotnull(ss_addr_sk#24) -(37) Scan parquet spark_catalog.default.date_dim +(35) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#28, d_year#29, d_qoy#30] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(38) CometFilter +(36) CometFilter Input [3]: [d_date_sk#28, d_year#29, d_qoy#30] Condition : ((((isnotnull(d_qoy#30) AND isnotnull(d_year#29)) AND (d_qoy#30 = 3)) AND (d_year#29 = 2000)) AND isnotnull(d_date_sk#28)) -(39) CometBroadcastExchange +(37) CometBroadcastExchange Input [3]: [d_date_sk#28, d_year#29, d_qoy#30] Arguments: [d_date_sk#28, d_year#29, d_qoy#30] -(40) CometBroadcastHashJoin +(38) CometBroadcastHashJoin Left output [3]: [ss_addr_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] Right output [3]: [d_date_sk#28, d_year#29, d_qoy#30] Arguments: [ss_sold_date_sk#26], [d_date_sk#28], Inner -(41) CometProject +(39) CometProject Input [6]: [ss_addr_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26, d_date_sk#28, d_year#29, d_qoy#30] Arguments: [ss_addr_sk#24, ss_ext_sales_price#25, d_year#29, d_qoy#30], [ss_addr_sk#24, ss_ext_sales_price#25, d_year#29, d_qoy#30] -(42) Scan parquet spark_catalog.default.customer_address +(40) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#31, ca_county#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county)] -ReadSchema: struct -(43) CometFilter -Input [2]: [ca_address_sk#31, ca_county#32] -Condition : (isnotnull(ca_address_sk#31) AND isnotnull(ca_county#32)) - -(44) CometBroadcastExchange -Input [2]: [ca_address_sk#31, ca_county#32] -Arguments: [ca_address_sk#31, ca_county#32] - -(45) CometBroadcastHashJoin +(41) CometBroadcastHashJoin Left output [4]: [ss_addr_sk#24, ss_ext_sales_price#25, d_year#29, d_qoy#30] Right output [2]: [ca_address_sk#31, ca_county#32] Arguments: [ss_addr_sk#24], [ca_address_sk#31], Inner -(46) CometProject +(42) CometProject Input [6]: [ss_addr_sk#24, ss_ext_sales_price#25, d_year#29, d_qoy#30, ca_address_sk#31, ca_county#32] Arguments: [ss_ext_sales_price#25, d_year#29, d_qoy#30, ca_county#32], [ss_ext_sales_price#25, d_year#29, d_qoy#30, ca_county#32] -(47) CometHashAggregate +(43) CometHashAggregate Input [4]: [ss_ext_sales_price#25, d_year#29, d_qoy#30, ca_county#32] Keys [3]: [ca_county#32, d_qoy#30, d_year#29] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#25))] -(48) ColumnarToRow [codegen id : 4] +(44) ColumnarToRow [codegen id : 4] Input [4]: [ca_county#32, d_qoy#30, d_year#29, sum#33] -(49) Exchange +(45) Exchange Input [4]: [ca_county#32, d_qoy#30, d_year#29, sum#33] Arguments: hashpartitioning(ca_county#32, d_qoy#30, d_year#29, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(50) HashAggregate [codegen id : 5] +(46) HashAggregate [codegen id : 5] Input [4]: [ca_county#32, d_qoy#30, d_year#29, sum#33] Keys [3]: [ca_county#32, d_qoy#30, d_year#29] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#25))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#25))#11] Results [2]: [ca_county#32, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#25))#11,17,2) AS store_sales#34] -(51) BroadcastExchange +(47) BroadcastExchange Input [2]: [ca_county#32, store_sales#34] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(52) BroadcastHashJoin [codegen id : 12] +(48) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ca_county#21] Right keys [1]: [ca_county#32] Join type: Inner Join condition: None -(53) Project [codegen id : 12] +(49) Project [codegen id : 12] Output [5]: [ca_county#9, d_year#6, store_sales#12, store_sales#23, store_sales#34] Input [7]: [ca_county#9, d_year#6, store_sales#12, ca_county#21, store_sales#23, ca_county#32, store_sales#34] -(54) Scan parquet spark_catalog.default.web_sales +(50) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_bill_addr_sk#35, ws_ext_sales_price#36, ws_sold_date_sk#37] Batched: true Location: InMemoryFileIndex [] @@ -381,88 +341,64 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#37), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(55) CometFilter +(51) CometFilter Input [3]: [ws_bill_addr_sk#35, ws_ext_sales_price#36, ws_sold_date_sk#37] Condition : isnotnull(ws_bill_addr_sk#35) -(56) Scan parquet spark_catalog.default.date_dim +(52) ReusedExchange [Reuses operator id: 5] Output [3]: [d_date_sk#39, d_year#40, d_qoy#41] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,1), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(57) CometFilter -Input [3]: [d_date_sk#39, d_year#40, d_qoy#41] -Condition : ((((isnotnull(d_qoy#41) AND isnotnull(d_year#40)) AND (d_qoy#41 = 1)) AND (d_year#40 = 2000)) AND isnotnull(d_date_sk#39)) -(58) CometBroadcastExchange -Input [3]: [d_date_sk#39, d_year#40, d_qoy#41] -Arguments: [d_date_sk#39, d_year#40, d_qoy#41] - -(59) CometBroadcastHashJoin +(53) CometBroadcastHashJoin Left output [3]: [ws_bill_addr_sk#35, ws_ext_sales_price#36, ws_sold_date_sk#37] Right output [3]: [d_date_sk#39, d_year#40, d_qoy#41] Arguments: [ws_sold_date_sk#37], [d_date_sk#39], Inner -(60) CometProject +(54) CometProject Input [6]: [ws_bill_addr_sk#35, ws_ext_sales_price#36, ws_sold_date_sk#37, d_date_sk#39, d_year#40, d_qoy#41] Arguments: [ws_bill_addr_sk#35, ws_ext_sales_price#36, d_year#40, d_qoy#41], [ws_bill_addr_sk#35, ws_ext_sales_price#36, d_year#40, d_qoy#41] -(61) Scan parquet spark_catalog.default.customer_address +(55) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#42, ca_county#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county)] -ReadSchema: struct - -(62) CometFilter -Input [2]: [ca_address_sk#42, ca_county#43] -Condition : (isnotnull(ca_address_sk#42) AND isnotnull(ca_county#43)) - -(63) CometBroadcastExchange -Input [2]: [ca_address_sk#42, ca_county#43] -Arguments: [ca_address_sk#42, ca_county#43] -(64) CometBroadcastHashJoin +(56) CometBroadcastHashJoin Left output [4]: [ws_bill_addr_sk#35, ws_ext_sales_price#36, d_year#40, d_qoy#41] Right output [2]: [ca_address_sk#42, ca_county#43] Arguments: [ws_bill_addr_sk#35], [ca_address_sk#42], Inner -(65) CometProject +(57) CometProject Input [6]: [ws_bill_addr_sk#35, ws_ext_sales_price#36, d_year#40, d_qoy#41, ca_address_sk#42, ca_county#43] Arguments: [ws_ext_sales_price#36, d_year#40, d_qoy#41, ca_county#43], [ws_ext_sales_price#36, d_year#40, d_qoy#41, ca_county#43] -(66) CometHashAggregate +(58) CometHashAggregate Input [4]: [ws_ext_sales_price#36, d_year#40, d_qoy#41, ca_county#43] Keys [3]: [ca_county#43, d_qoy#41, d_year#40] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#36))] -(67) ColumnarToRow [codegen id : 6] +(59) ColumnarToRow [codegen id : 6] Input [4]: [ca_county#43, d_qoy#41, d_year#40, sum#44] -(68) Exchange +(60) Exchange Input [4]: [ca_county#43, d_qoy#41, d_year#40, sum#44] Arguments: hashpartitioning(ca_county#43, d_qoy#41, d_year#40, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(69) HashAggregate [codegen id : 7] +(61) HashAggregate [codegen id : 7] Input [4]: [ca_county#43, d_qoy#41, d_year#40, sum#44] Keys [3]: [ca_county#43, d_qoy#41, d_year#40] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#36))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#36))#45] Results [2]: [ca_county#43, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#36))#45,17,2) AS web_sales#46] -(70) BroadcastExchange +(62) BroadcastExchange Input [2]: [ca_county#43, web_sales#46] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] -(71) BroadcastHashJoin [codegen id : 12] +(63) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ca_county#9] Right keys [1]: [ca_county#43] Join type: Inner Join condition: None -(72) Scan parquet spark_catalog.default.web_sales +(64) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, ws_sold_date_sk#49] Batched: true Location: InMemoryFileIndex [] @@ -470,92 +406,68 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#49), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(73) CometFilter +(65) CometFilter Input [3]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, ws_sold_date_sk#49] Condition : isnotnull(ws_bill_addr_sk#47) -(74) Scan parquet spark_catalog.default.date_dim +(66) ReusedExchange [Reuses operator id: 21] Output [3]: [d_date_sk#51, d_year#52, d_qoy#53] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(75) CometFilter -Input [3]: [d_date_sk#51, d_year#52, d_qoy#53] -Condition : ((((isnotnull(d_qoy#53) AND isnotnull(d_year#52)) AND (d_qoy#53 = 2)) AND (d_year#52 = 2000)) AND isnotnull(d_date_sk#51)) -(76) CometBroadcastExchange -Input [3]: [d_date_sk#51, d_year#52, d_qoy#53] -Arguments: [d_date_sk#51, d_year#52, d_qoy#53] - -(77) CometBroadcastHashJoin +(67) CometBroadcastHashJoin Left output [3]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, ws_sold_date_sk#49] Right output [3]: [d_date_sk#51, d_year#52, d_qoy#53] Arguments: [ws_sold_date_sk#49], [d_date_sk#51], Inner -(78) CometProject +(68) CometProject Input [6]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, ws_sold_date_sk#49, d_date_sk#51, d_year#52, d_qoy#53] Arguments: [ws_bill_addr_sk#47, ws_ext_sales_price#48, d_year#52, d_qoy#53], [ws_bill_addr_sk#47, ws_ext_sales_price#48, d_year#52, d_qoy#53] -(79) Scan parquet spark_catalog.default.customer_address +(69) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#54, ca_county#55] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county)] -ReadSchema: struct -(80) CometFilter -Input [2]: [ca_address_sk#54, ca_county#55] -Condition : (isnotnull(ca_address_sk#54) AND isnotnull(ca_county#55)) - -(81) CometBroadcastExchange -Input [2]: [ca_address_sk#54, ca_county#55] -Arguments: [ca_address_sk#54, ca_county#55] - -(82) CometBroadcastHashJoin +(70) CometBroadcastHashJoin Left output [4]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, d_year#52, d_qoy#53] Right output [2]: [ca_address_sk#54, ca_county#55] Arguments: [ws_bill_addr_sk#47], [ca_address_sk#54], Inner -(83) CometProject +(71) CometProject Input [6]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, d_year#52, d_qoy#53, ca_address_sk#54, ca_county#55] Arguments: [ws_ext_sales_price#48, d_year#52, d_qoy#53, ca_county#55], [ws_ext_sales_price#48, d_year#52, d_qoy#53, ca_county#55] -(84) CometHashAggregate +(72) CometHashAggregate Input [4]: [ws_ext_sales_price#48, d_year#52, d_qoy#53, ca_county#55] Keys [3]: [ca_county#55, d_qoy#53, d_year#52] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#48))] -(85) ColumnarToRow [codegen id : 8] +(73) ColumnarToRow [codegen id : 8] Input [4]: [ca_county#55, d_qoy#53, d_year#52, sum#56] -(86) Exchange +(74) Exchange Input [4]: [ca_county#55, d_qoy#53, d_year#52, sum#56] Arguments: hashpartitioning(ca_county#55, d_qoy#53, d_year#52, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(87) HashAggregate [codegen id : 9] +(75) HashAggregate [codegen id : 9] Input [4]: [ca_county#55, d_qoy#53, d_year#52, sum#56] Keys [3]: [ca_county#55, d_qoy#53, d_year#52] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#48))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#48))#45] Results [2]: [ca_county#55, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#48))#45,17,2) AS web_sales#57] -(88) BroadcastExchange +(76) BroadcastExchange Input [2]: [ca_county#55, web_sales#57] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=9] -(89) BroadcastHashJoin [codegen id : 12] +(77) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ca_county#43] Right keys [1]: [ca_county#55] Join type: Inner Join condition: (CASE WHEN (web_sales#46 > 0.00) THEN (web_sales#57 / web_sales#46) END > CASE WHEN (store_sales#12 > 0.00) THEN (store_sales#23 / store_sales#12) END) -(90) Project [codegen id : 12] +(78) Project [codegen id : 12] Output [8]: [ca_county#9, d_year#6, store_sales#12, store_sales#23, store_sales#34, ca_county#43, web_sales#46, web_sales#57] Input [9]: [ca_county#9, d_year#6, store_sales#12, store_sales#23, store_sales#34, ca_county#43, web_sales#46, ca_county#55, web_sales#57] -(91) Scan parquet spark_catalog.default.web_sales +(79) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_bill_addr_sk#58, ws_ext_sales_price#59, ws_sold_date_sk#60] Batched: true Location: InMemoryFileIndex [] @@ -563,180 +475,156 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#60), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(92) CometFilter +(80) CometFilter Input [3]: [ws_bill_addr_sk#58, ws_ext_sales_price#59, ws_sold_date_sk#60] Condition : isnotnull(ws_bill_addr_sk#58) -(93) Scan parquet spark_catalog.default.date_dim +(81) ReusedExchange [Reuses operator id: 37] Output [3]: [d_date_sk#62, d_year#63, d_qoy#64] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(94) CometFilter -Input [3]: [d_date_sk#62, d_year#63, d_qoy#64] -Condition : ((((isnotnull(d_qoy#64) AND isnotnull(d_year#63)) AND (d_qoy#64 = 3)) AND (d_year#63 = 2000)) AND isnotnull(d_date_sk#62)) - -(95) CometBroadcastExchange -Input [3]: [d_date_sk#62, d_year#63, d_qoy#64] -Arguments: [d_date_sk#62, d_year#63, d_qoy#64] -(96) CometBroadcastHashJoin +(82) CometBroadcastHashJoin Left output [3]: [ws_bill_addr_sk#58, ws_ext_sales_price#59, ws_sold_date_sk#60] Right output [3]: [d_date_sk#62, d_year#63, d_qoy#64] Arguments: [ws_sold_date_sk#60], [d_date_sk#62], Inner -(97) CometProject +(83) CometProject Input [6]: [ws_bill_addr_sk#58, ws_ext_sales_price#59, ws_sold_date_sk#60, d_date_sk#62, d_year#63, d_qoy#64] Arguments: [ws_bill_addr_sk#58, ws_ext_sales_price#59, d_year#63, d_qoy#64], [ws_bill_addr_sk#58, ws_ext_sales_price#59, d_year#63, d_qoy#64] -(98) Scan parquet spark_catalog.default.customer_address +(84) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#65, ca_county#66] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county)] -ReadSchema: struct - -(99) CometFilter -Input [2]: [ca_address_sk#65, ca_county#66] -Condition : (isnotnull(ca_address_sk#65) AND isnotnull(ca_county#66)) -(100) CometBroadcastExchange -Input [2]: [ca_address_sk#65, ca_county#66] -Arguments: [ca_address_sk#65, ca_county#66] - -(101) CometBroadcastHashJoin +(85) CometBroadcastHashJoin Left output [4]: [ws_bill_addr_sk#58, ws_ext_sales_price#59, d_year#63, d_qoy#64] Right output [2]: [ca_address_sk#65, ca_county#66] Arguments: [ws_bill_addr_sk#58], [ca_address_sk#65], Inner -(102) CometProject +(86) CometProject Input [6]: [ws_bill_addr_sk#58, ws_ext_sales_price#59, d_year#63, d_qoy#64, ca_address_sk#65, ca_county#66] Arguments: [ws_ext_sales_price#59, d_year#63, d_qoy#64, ca_county#66], [ws_ext_sales_price#59, d_year#63, d_qoy#64, ca_county#66] -(103) CometHashAggregate +(87) CometHashAggregate Input [4]: [ws_ext_sales_price#59, d_year#63, d_qoy#64, ca_county#66] Keys [3]: [ca_county#66, d_qoy#64, d_year#63] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#59))] -(104) ColumnarToRow [codegen id : 10] +(88) ColumnarToRow [codegen id : 10] Input [4]: [ca_county#66, d_qoy#64, d_year#63, sum#67] -(105) Exchange +(89) Exchange Input [4]: [ca_county#66, d_qoy#64, d_year#63, sum#67] Arguments: hashpartitioning(ca_county#66, d_qoy#64, d_year#63, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(106) HashAggregate [codegen id : 11] +(90) HashAggregate [codegen id : 11] Input [4]: [ca_county#66, d_qoy#64, d_year#63, sum#67] Keys [3]: [ca_county#66, d_qoy#64, d_year#63] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#59))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#59))#45] Results [2]: [ca_county#66, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#59))#45,17,2) AS web_sales#68] -(107) BroadcastExchange +(91) BroadcastExchange Input [2]: [ca_county#66, web_sales#68] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(108) BroadcastHashJoin [codegen id : 12] +(92) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ca_county#43] Right keys [1]: [ca_county#66] Join type: Inner Join condition: (CASE WHEN (web_sales#57 > 0.00) THEN (web_sales#68 / web_sales#57) END > CASE WHEN (store_sales#23 > 0.00) THEN (store_sales#34 / store_sales#23) END) -(109) Project [codegen id : 12] +(93) Project [codegen id : 12] Output [6]: [ca_county#9, d_year#6, (web_sales#57 / web_sales#46) AS web_q1_q2_increase#69, (store_sales#23 / store_sales#12) AS store_q1_q2_increase#70, (web_sales#68 / web_sales#57) AS web_q2_q3_increase#71, (store_sales#34 / store_sales#23) AS store_q2_q3_increase#72] Input [10]: [ca_county#9, d_year#6, store_sales#12, store_sales#23, store_sales#34, ca_county#43, web_sales#46, web_sales#57, ca_county#66, web_sales#68] -(110) Exchange +(94) Exchange Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#69, store_q1_q2_increase#70, web_q2_q3_increase#71, store_q2_q3_increase#72] Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(111) Sort [codegen id : 13] +(95) Sort [codegen id : 13] Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#69, store_q1_q2_increase#70, web_q2_q3_increase#71, store_q2_q3_increase#72] Arguments: [ca_county#9 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (115) -+- * ColumnarToRow (114) - +- CometFilter (113) - +- CometScan parquet spark_catalog.default.date_dim (112) +BroadcastExchange (99) ++- * ColumnarToRow (98) + +- CometFilter (97) + +- CometScan parquet spark_catalog.default.date_dim (96) -(112) Scan parquet spark_catalog.default.date_dim +(96) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,1), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(113) CometFilter +(97) CometFilter Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) -(114) ColumnarToRow [codegen id : 1] +(98) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -(115) BroadcastExchange +(99) BroadcastExchange Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] Subquery:2 Hosting operator id = 17 Hosting Expression = ss_sold_date_sk#15 IN dynamicpruning#16 -BroadcastExchange (119) -+- * ColumnarToRow (118) - +- CometFilter (117) - +- CometScan parquet spark_catalog.default.date_dim (116) +BroadcastExchange (103) ++- * ColumnarToRow (102) + +- CometFilter (101) + +- CometScan parquet spark_catalog.default.date_dim (100) -(116) Scan parquet spark_catalog.default.date_dim +(100) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#17, d_year#18, d_qoy#19] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(117) CometFilter +(101) CometFilter Input [3]: [d_date_sk#17, d_year#18, d_qoy#19] Condition : ((((isnotnull(d_qoy#19) AND isnotnull(d_year#18)) AND (d_qoy#19 = 2)) AND (d_year#18 = 2000)) AND isnotnull(d_date_sk#17)) -(118) ColumnarToRow [codegen id : 1] +(102) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#17, d_year#18, d_qoy#19] -(119) BroadcastExchange +(103) BroadcastExchange Input [3]: [d_date_sk#17, d_year#18, d_qoy#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=14] -Subquery:3 Hosting operator id = 35 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#27 -BroadcastExchange (123) -+- * ColumnarToRow (122) - +- CometFilter (121) - +- CometScan parquet spark_catalog.default.date_dim (120) +Subquery:3 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#27 +BroadcastExchange (107) ++- * ColumnarToRow (106) + +- CometFilter (105) + +- CometScan parquet spark_catalog.default.date_dim (104) -(120) Scan parquet spark_catalog.default.date_dim +(104) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#28, d_year#29, d_qoy#30] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(121) CometFilter +(105) CometFilter Input [3]: [d_date_sk#28, d_year#29, d_qoy#30] Condition : ((((isnotnull(d_qoy#30) AND isnotnull(d_year#29)) AND (d_qoy#30 = 3)) AND (d_year#29 = 2000)) AND isnotnull(d_date_sk#28)) -(122) ColumnarToRow [codegen id : 1] +(106) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#28, d_year#29, d_qoy#30] -(123) BroadcastExchange +(107) BroadcastExchange Input [3]: [d_date_sk#28, d_year#29, d_qoy#30] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=15] -Subquery:4 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#37 IN dynamicpruning#4 +Subquery:4 Hosting operator id = 50 Hosting Expression = ws_sold_date_sk#37 IN dynamicpruning#4 -Subquery:5 Hosting operator id = 72 Hosting Expression = ws_sold_date_sk#49 IN dynamicpruning#16 +Subquery:5 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#49 IN dynamicpruning#16 -Subquery:6 Hosting operator id = 91 Hosting Expression = ws_sold_date_sk#60 IN dynamicpruning#27 +Subquery:6 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#60 IN dynamicpruning#27 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt index 03a039d50..a94a8a94d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt @@ -63,15 +63,13 @@ WholeStageCodegen (13) CometBroadcastExchange #9 CometFilter [d_qoy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #10 - CometFilter [ca_address_sk,ca_county] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter - BroadcastExchange #11 + BroadcastExchange #10 WholeStageCodegen (5) HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] InputAdapter - Exchange [ca_county,d_qoy,d_year] #12 + Exchange [ca_county,d_qoy,d_year] #11 WholeStageCodegen (4) ColumnarToRow InputAdapter @@ -83,24 +81,22 @@ WholeStageCodegen (13) CometFilter [ss_addr_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #13 + BroadcastExchange #12 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_qoy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #14 + CometBroadcastExchange #13 CometFilter [d_qoy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #15 - CometFilter [ca_address_sk,ca_county] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter - BroadcastExchange #16 + BroadcastExchange #14 WholeStageCodegen (7) HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] InputAdapter - Exchange [ca_county,d_qoy,d_year] #17 + Exchange [ca_county,d_qoy,d_year] #15 WholeStageCodegen (6) ColumnarToRow InputAdapter @@ -112,18 +108,14 @@ WholeStageCodegen (13) CometFilter [ws_bill_addr_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #18 - CometFilter [d_qoy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #19 - CometFilter [ca_address_sk,ca_county] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + ReusedExchange [d_date_sk,d_year,d_qoy] #4 + ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter - BroadcastExchange #20 + BroadcastExchange #16 WholeStageCodegen (9) HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] InputAdapter - Exchange [ca_county,d_qoy,d_year] #21 + Exchange [ca_county,d_qoy,d_year] #17 WholeStageCodegen (8) ColumnarToRow InputAdapter @@ -135,18 +127,14 @@ WholeStageCodegen (13) CometFilter [ws_bill_addr_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #22 - CometFilter [d_qoy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #23 - CometFilter [ca_address_sk,ca_county] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + ReusedExchange [d_date_sk,d_year,d_qoy] #9 + ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter - BroadcastExchange #24 + BroadcastExchange #18 WholeStageCodegen (11) HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] InputAdapter - Exchange [ca_county,d_qoy,d_year] #25 + Exchange [ca_county,d_qoy,d_year] #19 WholeStageCodegen (10) ColumnarToRow InputAdapter @@ -158,9 +146,5 @@ WholeStageCodegen (13) CometFilter [ws_bill_addr_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange #26 - CometFilter [d_qoy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #27 - CometFilter [ca_address_sk,ca_county] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + ReusedExchange [d_date_sk,d_year,d_qoy] #13 + ReusedExchange [ca_address_sk,ca_county] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt index 0606225d8..7b5048ce7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (89) -+- * HashAggregate (88) - +- Exchange (87) - +- * HashAggregate (86) - +- Union (85) +TakeOrderedAndProject (63) ++- * HashAggregate (62) + +- Exchange (61) + +- * HashAggregate (60) + +- Union (59) :- * HashAggregate (28) : +- Exchange (27) : +- * ColumnarToRow (26) @@ -32,62 +32,36 @@ TakeOrderedAndProject (89) : +- CometProject (19) : +- CometFilter (18) : +- CometScan parquet spark_catalog.default.item (17) - :- * HashAggregate (56) - : +- Exchange (55) - : +- * ColumnarToRow (54) - : +- CometHashAggregate (53) - : +- CometProject (52) - : +- CometBroadcastHashJoin (51) - : :- CometProject (42) - : : +- CometBroadcastHashJoin (41) - : : :- CometProject (36) - : : : +- CometBroadcastHashJoin (35) + :- * HashAggregate (43) + : +- Exchange (42) + : +- * ColumnarToRow (41) + : +- CometHashAggregate (40) + : +- CometProject (39) + : +- CometBroadcastHashJoin (38) + : :- CometProject (36) + : : +- CometBroadcastHashJoin (35) + : : :- CometProject (33) + : : : +- CometBroadcastHashJoin (32) : : : :- CometFilter (30) : : : : +- CometScan parquet spark_catalog.default.catalog_sales (29) - : : : +- CometBroadcastExchange (34) - : : : +- CometProject (33) - : : : +- CometFilter (32) - : : : +- CometScan parquet spark_catalog.default.date_dim (31) - : : +- CometBroadcastExchange (40) - : : +- CometProject (39) - : : +- CometFilter (38) - : : +- CometScan parquet spark_catalog.default.customer_address (37) - : +- CometBroadcastExchange (50) - : +- CometBroadcastHashJoin (49) - : :- CometFilter (44) - : : +- CometScan parquet spark_catalog.default.item (43) - : +- CometBroadcastExchange (48) - : +- CometProject (47) - : +- CometFilter (46) - : +- CometScan parquet spark_catalog.default.item (45) - +- * HashAggregate (84) - +- Exchange (83) - +- * ColumnarToRow (82) - +- CometHashAggregate (81) - +- CometProject (80) - +- CometBroadcastHashJoin (79) - :- CometProject (70) - : +- CometBroadcastHashJoin (69) - : :- CometProject (64) - : : +- CometBroadcastHashJoin (63) - : : :- CometFilter (58) - : : : +- CometScan parquet spark_catalog.default.web_sales (57) - : : +- CometBroadcastExchange (62) - : : +- CometProject (61) - : : +- CometFilter (60) - : : +- CometScan parquet spark_catalog.default.date_dim (59) - : +- CometBroadcastExchange (68) - : +- CometProject (67) - : +- CometFilter (66) - : +- CometScan parquet spark_catalog.default.customer_address (65) - +- CometBroadcastExchange (78) - +- CometBroadcastHashJoin (77) - :- CometFilter (72) - : +- CometScan parquet spark_catalog.default.item (71) - +- CometBroadcastExchange (76) - +- CometProject (75) - +- CometFilter (74) - +- CometScan parquet spark_catalog.default.item (73) + : : : +- ReusedExchange (31) + : : +- ReusedExchange (34) + : +- ReusedExchange (37) + +- * HashAggregate (58) + +- Exchange (57) + +- * ColumnarToRow (56) + +- CometHashAggregate (55) + +- CometProject (54) + +- CometBroadcastHashJoin (53) + :- CometProject (51) + : +- CometBroadcastHashJoin (50) + : :- CometProject (48) + : : +- CometBroadcastHashJoin (47) + : : :- CometFilter (45) + : : : +- CometScan parquet spark_catalog.default.web_sales (44) + : : +- ReusedExchange (46) + : +- ReusedExchange (49) + +- ReusedExchange (52) (1) Scan parquet spark_catalog.default.store_sales @@ -237,322 +211,186 @@ ReadSchema: struct - -(32) CometFilter -Input [3]: [d_date_sk#23, d_year#24, d_moy#25] -Condition : ((((isnotnull(d_year#24) AND isnotnull(d_moy#25)) AND (d_year#24 = 1998)) AND (d_moy#25 = 5)) AND isnotnull(d_date_sk#23)) +(31) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#23] -(33) CometProject -Input [3]: [d_date_sk#23, d_year#24, d_moy#25] -Arguments: [d_date_sk#23], [d_date_sk#23] - -(34) CometBroadcastExchange -Input [1]: [d_date_sk#23] -Arguments: [d_date_sk#23] - -(35) CometBroadcastHashJoin +(32) CometBroadcastHashJoin Left output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] Right output [1]: [d_date_sk#23] Arguments: [cs_sold_date_sk#21], [d_date_sk#23], Inner -(36) CometProject +(33) CometProject Input [5]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#23] Arguments: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20], [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] -(37) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#26, ca_gmt_offset#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(38) CometFilter -Input [2]: [ca_address_sk#26, ca_gmt_offset#27] -Condition : ((isnotnull(ca_gmt_offset#27) AND (ca_gmt_offset#27 = -5.00)) AND isnotnull(ca_address_sk#26)) +(34) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#24] -(39) CometProject -Input [2]: [ca_address_sk#26, ca_gmt_offset#27] -Arguments: [ca_address_sk#26], [ca_address_sk#26] - -(40) CometBroadcastExchange -Input [1]: [ca_address_sk#26] -Arguments: [ca_address_sk#26] - -(41) CometBroadcastHashJoin +(35) CometBroadcastHashJoin Left output [3]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] -Right output [1]: [ca_address_sk#26] -Arguments: [cs_bill_addr_sk#18], [ca_address_sk#26], Inner +Right output [1]: [ca_address_sk#24] +Arguments: [cs_bill_addr_sk#18], [ca_address_sk#24], Inner -(42) CometProject -Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, ca_address_sk#26] +(36) CometProject +Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, ca_address_sk#24] Arguments: [cs_item_sk#19, cs_ext_sales_price#20], [cs_item_sk#19, cs_ext_sales_price#20] -(43) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#28, i_manufact_id#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(44) CometFilter -Input [2]: [i_item_sk#28, i_manufact_id#29] -Condition : isnotnull(i_item_sk#28) - -(45) Scan parquet spark_catalog.default.item -Output [2]: [i_category#30, i_manufact_id#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] -ReadSchema: struct - -(46) CometFilter -Input [2]: [i_category#30, i_manufact_id#31] -Condition : (isnotnull(i_category#30) AND (i_category#30 = Electronics )) - -(47) CometProject -Input [2]: [i_category#30, i_manufact_id#31] -Arguments: [i_manufact_id#31], [i_manufact_id#31] +(37) ReusedExchange [Reuses operator id: 22] +Output [2]: [i_item_sk#25, i_manufact_id#26] -(48) CometBroadcastExchange -Input [1]: [i_manufact_id#31] -Arguments: [i_manufact_id#31] - -(49) CometBroadcastHashJoin -Left output [2]: [i_item_sk#28, i_manufact_id#29] -Right output [1]: [i_manufact_id#31] -Arguments: [i_manufact_id#29], [i_manufact_id#31], LeftSemi - -(50) CometBroadcastExchange -Input [2]: [i_item_sk#28, i_manufact_id#29] -Arguments: [i_item_sk#28, i_manufact_id#29] - -(51) CometBroadcastHashJoin +(38) CometBroadcastHashJoin Left output [2]: [cs_item_sk#19, cs_ext_sales_price#20] -Right output [2]: [i_item_sk#28, i_manufact_id#29] -Arguments: [cs_item_sk#19], [i_item_sk#28], Inner +Right output [2]: [i_item_sk#25, i_manufact_id#26] +Arguments: [cs_item_sk#19], [i_item_sk#25], Inner -(52) CometProject -Input [4]: [cs_item_sk#19, cs_ext_sales_price#20, i_item_sk#28, i_manufact_id#29] -Arguments: [cs_ext_sales_price#20, i_manufact_id#29], [cs_ext_sales_price#20, i_manufact_id#29] +(39) CometProject +Input [4]: [cs_item_sk#19, cs_ext_sales_price#20, i_item_sk#25, i_manufact_id#26] +Arguments: [cs_ext_sales_price#20, i_manufact_id#26], [cs_ext_sales_price#20, i_manufact_id#26] -(53) CometHashAggregate -Input [2]: [cs_ext_sales_price#20, i_manufact_id#29] -Keys [1]: [i_manufact_id#29] +(40) CometHashAggregate +Input [2]: [cs_ext_sales_price#20, i_manufact_id#26] +Keys [1]: [i_manufact_id#26] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#20))] -(54) ColumnarToRow [codegen id : 3] -Input [2]: [i_manufact_id#29, sum#32] +(41) ColumnarToRow [codegen id : 3] +Input [2]: [i_manufact_id#26, sum#27] -(55) Exchange -Input [2]: [i_manufact_id#29, sum#32] -Arguments: hashpartitioning(i_manufact_id#29, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(42) Exchange +Input [2]: [i_manufact_id#26, sum#27] +Arguments: hashpartitioning(i_manufact_id#26, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(56) HashAggregate [codegen id : 4] -Input [2]: [i_manufact_id#29, sum#32] -Keys [1]: [i_manufact_id#29] +(43) HashAggregate [codegen id : 4] +Input [2]: [i_manufact_id#26, sum#27] +Keys [1]: [i_manufact_id#26] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#20))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#20))#33] -Results [2]: [i_manufact_id#29, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#20))#33,17,2) AS total_sales#34] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#20))#28] +Results [2]: [i_manufact_id#26, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#20))#28,17,2) AS total_sales#29] -(57) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#35, ws_bill_addr_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38] +(44) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#39)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#34)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(58) CometFilter -Input [4]: [ws_item_sk#35, ws_bill_addr_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38] -Condition : (isnotnull(ws_bill_addr_sk#36) AND isnotnull(ws_item_sk#35)) - -(59) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#40, d_year#41, d_moy#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,5), IsNotNull(d_date_sk)] -ReadSchema: struct +(45) CometFilter +Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] +Condition : (isnotnull(ws_bill_addr_sk#31) AND isnotnull(ws_item_sk#30)) -(60) CometFilter -Input [3]: [d_date_sk#40, d_year#41, d_moy#42] -Condition : ((((isnotnull(d_year#41) AND isnotnull(d_moy#42)) AND (d_year#41 = 1998)) AND (d_moy#42 = 5)) AND isnotnull(d_date_sk#40)) +(46) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#35] -(61) CometProject -Input [3]: [d_date_sk#40, d_year#41, d_moy#42] -Arguments: [d_date_sk#40], [d_date_sk#40] - -(62) CometBroadcastExchange -Input [1]: [d_date_sk#40] -Arguments: [d_date_sk#40] - -(63) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#35, ws_bill_addr_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38] -Right output [1]: [d_date_sk#40] -Arguments: [ws_sold_date_sk#38], [d_date_sk#40], Inner - -(64) CometProject -Input [5]: [ws_item_sk#35, ws_bill_addr_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38, d_date_sk#40] -Arguments: [ws_item_sk#35, ws_bill_addr_sk#36, ws_ext_sales_price#37], [ws_item_sk#35, ws_bill_addr_sk#36, ws_ext_sales_price#37] - -(65) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#43, ca_gmt_offset#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(66) CometFilter -Input [2]: [ca_address_sk#43, ca_gmt_offset#44] -Condition : ((isnotnull(ca_gmt_offset#44) AND (ca_gmt_offset#44 = -5.00)) AND isnotnull(ca_address_sk#43)) - -(67) CometProject -Input [2]: [ca_address_sk#43, ca_gmt_offset#44] -Arguments: [ca_address_sk#43], [ca_address_sk#43] - -(68) CometBroadcastExchange -Input [1]: [ca_address_sk#43] -Arguments: [ca_address_sk#43] - -(69) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#35, ws_bill_addr_sk#36, ws_ext_sales_price#37] -Right output [1]: [ca_address_sk#43] -Arguments: [ws_bill_addr_sk#36], [ca_address_sk#43], Inner - -(70) CometProject -Input [4]: [ws_item_sk#35, ws_bill_addr_sk#36, ws_ext_sales_price#37, ca_address_sk#43] -Arguments: [ws_item_sk#35, ws_ext_sales_price#37], [ws_item_sk#35, ws_ext_sales_price#37] - -(71) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#45, i_manufact_id#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(72) CometFilter -Input [2]: [i_item_sk#45, i_manufact_id#46] -Condition : isnotnull(i_item_sk#45) - -(73) Scan parquet spark_catalog.default.item -Output [2]: [i_category#47, i_manufact_id#48] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] -ReadSchema: struct +(47) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] +Right output [1]: [d_date_sk#35] +Arguments: [ws_sold_date_sk#33], [d_date_sk#35], Inner -(74) CometFilter -Input [2]: [i_category#47, i_manufact_id#48] -Condition : (isnotnull(i_category#47) AND (i_category#47 = Electronics )) +(48) CometProject +Input [5]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33, d_date_sk#35] +Arguments: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32], [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] -(75) CometProject -Input [2]: [i_category#47, i_manufact_id#48] -Arguments: [i_manufact_id#48], [i_manufact_id#48] +(49) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#36] -(76) CometBroadcastExchange -Input [1]: [i_manufact_id#48] -Arguments: [i_manufact_id#48] +(50) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] +Right output [1]: [ca_address_sk#36] +Arguments: [ws_bill_addr_sk#31], [ca_address_sk#36], Inner -(77) CometBroadcastHashJoin -Left output [2]: [i_item_sk#45, i_manufact_id#46] -Right output [1]: [i_manufact_id#48] -Arguments: [i_manufact_id#46], [i_manufact_id#48], LeftSemi +(51) CometProject +Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ca_address_sk#36] +Arguments: [ws_item_sk#30, ws_ext_sales_price#32], [ws_item_sk#30, ws_ext_sales_price#32] -(78) CometBroadcastExchange -Input [2]: [i_item_sk#45, i_manufact_id#46] -Arguments: [i_item_sk#45, i_manufact_id#46] +(52) ReusedExchange [Reuses operator id: 22] +Output [2]: [i_item_sk#37, i_manufact_id#38] -(79) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#35, ws_ext_sales_price#37] -Right output [2]: [i_item_sk#45, i_manufact_id#46] -Arguments: [ws_item_sk#35], [i_item_sk#45], Inner +(53) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#30, ws_ext_sales_price#32] +Right output [2]: [i_item_sk#37, i_manufact_id#38] +Arguments: [ws_item_sk#30], [i_item_sk#37], Inner -(80) CometProject -Input [4]: [ws_item_sk#35, ws_ext_sales_price#37, i_item_sk#45, i_manufact_id#46] -Arguments: [ws_ext_sales_price#37, i_manufact_id#46], [ws_ext_sales_price#37, i_manufact_id#46] +(54) CometProject +Input [4]: [ws_item_sk#30, ws_ext_sales_price#32, i_item_sk#37, i_manufact_id#38] +Arguments: [ws_ext_sales_price#32, i_manufact_id#38], [ws_ext_sales_price#32, i_manufact_id#38] -(81) CometHashAggregate -Input [2]: [ws_ext_sales_price#37, i_manufact_id#46] -Keys [1]: [i_manufact_id#46] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#37))] +(55) CometHashAggregate +Input [2]: [ws_ext_sales_price#32, i_manufact_id#38] +Keys [1]: [i_manufact_id#38] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#32))] -(82) ColumnarToRow [codegen id : 5] -Input [2]: [i_manufact_id#46, sum#49] +(56) ColumnarToRow [codegen id : 5] +Input [2]: [i_manufact_id#38, sum#39] -(83) Exchange -Input [2]: [i_manufact_id#46, sum#49] -Arguments: hashpartitioning(i_manufact_id#46, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(57) Exchange +Input [2]: [i_manufact_id#38, sum#39] +Arguments: hashpartitioning(i_manufact_id#38, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(84) HashAggregate [codegen id : 6] -Input [2]: [i_manufact_id#46, sum#49] -Keys [1]: [i_manufact_id#46] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#37))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#37))#50] -Results [2]: [i_manufact_id#46, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#37))#50,17,2) AS total_sales#51] +(58) HashAggregate [codegen id : 6] +Input [2]: [i_manufact_id#38, sum#39] +Keys [1]: [i_manufact_id#38] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#32))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#32))#40] +Results [2]: [i_manufact_id#38, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#32))#40,17,2) AS total_sales#41] -(85) Union +(59) Union -(86) HashAggregate [codegen id : 7] +(60) HashAggregate [codegen id : 7] Input [2]: [i_manufact_id#12, total_sales#17] Keys [1]: [i_manufact_id#12] Functions [1]: [partial_sum(total_sales#17)] -Aggregate Attributes [2]: [sum#52, isEmpty#53] -Results [3]: [i_manufact_id#12, sum#54, isEmpty#55] +Aggregate Attributes [2]: [sum#42, isEmpty#43] +Results [3]: [i_manufact_id#12, sum#44, isEmpty#45] -(87) Exchange -Input [3]: [i_manufact_id#12, sum#54, isEmpty#55] +(61) Exchange +Input [3]: [i_manufact_id#12, sum#44, isEmpty#45] Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(88) HashAggregate [codegen id : 8] -Input [3]: [i_manufact_id#12, sum#54, isEmpty#55] +(62) HashAggregate [codegen id : 8] +Input [3]: [i_manufact_id#12, sum#44, isEmpty#45] Keys [1]: [i_manufact_id#12] Functions [1]: [sum(total_sales#17)] -Aggregate Attributes [1]: [sum(total_sales#17)#56] -Results [2]: [i_manufact_id#12, sum(total_sales#17)#56 AS total_sales#57] +Aggregate Attributes [1]: [sum(total_sales#17)#46] +Results [2]: [i_manufact_id#12, sum(total_sales#17)#46 AS total_sales#47] -(89) TakeOrderedAndProject -Input [2]: [i_manufact_id#12, total_sales#57] -Arguments: 100, [total_sales#57 ASC NULLS FIRST], [i_manufact_id#12, total_sales#57] +(63) TakeOrderedAndProject +Input [2]: [i_manufact_id#12, total_sales#47] +Arguments: 100, [total_sales#47 ASC NULLS FIRST], [i_manufact_id#12, total_sales#47] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (94) -+- * ColumnarToRow (93) - +- CometProject (92) - +- CometFilter (91) - +- CometScan parquet spark_catalog.default.date_dim (90) +BroadcastExchange (68) ++- * ColumnarToRow (67) + +- CometProject (66) + +- CometFilter (65) + +- CometScan parquet spark_catalog.default.date_dim (64) -(90) Scan parquet spark_catalog.default.date_dim +(64) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#6, d_year#7, d_moy#8] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,5), IsNotNull(d_date_sk)] ReadSchema: struct -(91) CometFilter +(65) CometFilter Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 5)) AND isnotnull(d_date_sk#6)) -(92) CometProject +(66) CometProject Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(93) ColumnarToRow [codegen id : 1] +(67) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(94) BroadcastExchange +(68) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt index d1e7288bc..af2b7cb5d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt @@ -64,26 +64,13 @@ TakeOrderedAndProject [total_sales,i_manufact_id] CometFilter [cs_bill_addr_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #9 - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #10 - CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #11 - CometBroadcastHashJoin [i_manufact_id,i_manufact_id] - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange #12 - CometProject [i_manufact_id] - CometFilter [i_category] - CometScan parquet spark_catalog.default.item [i_category,i_manufact_id] + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_manufact_id] #6 WholeStageCodegen (6) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_manufact_id] #13 + Exchange [i_manufact_id] #9 WholeStageCodegen (5) ColumnarToRow InputAdapter @@ -97,19 +84,6 @@ TakeOrderedAndProject [total_sales,i_manufact_id] CometFilter [ws_bill_addr_sk,ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #14 - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #15 - CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #16 - CometBroadcastHashJoin [i_manufact_id,i_manufact_id] - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange #17 - CometProject [i_manufact_id] - CometFilter [i_category] - CometScan parquet spark_catalog.default.item [i_category,i_manufact_id] + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_manufact_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt index 25c9f6b1c..884ef2eda 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt @@ -1,16 +1,16 @@ == Physical Plan == -TakeOrderedAndProject (50) -+- * HashAggregate (49) - +- Exchange (48) - +- * HashAggregate (47) - +- * Project (46) - +- * BroadcastHashJoin Inner BuildRight (45) - :- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (34) - : : +- * Filter (33) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (32) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (22) +TakeOrderedAndProject (44) ++- * HashAggregate (43) + +- Exchange (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * Project (34) + : +- * BroadcastHashJoin Inner BuildRight (33) + : :- * Project (28) + : : +- * Filter (27) + : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) + : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) : : : :- * ColumnarToRow (12) : : : : +- CometBroadcastHashJoin (11) : : : : :- CometFilter (2) @@ -23,32 +23,26 @@ TakeOrderedAndProject (50) : : : : +- CometProject (6) : : : : +- CometFilter (5) : : : : +- CometScan parquet spark_catalog.default.date_dim (4) - : : : +- BroadcastExchange (21) - : : : +- * ColumnarToRow (20) - : : : +- CometProject (19) - : : : +- CometBroadcastHashJoin (18) + : : : +- BroadcastExchange (18) + : : : +- * ColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometBroadcastHashJoin (15) : : : :- CometScan parquet spark_catalog.default.web_sales (13) - : : : +- CometBroadcastExchange (17) - : : : +- CometProject (16) - : : : +- CometFilter (15) - : : : +- CometScan parquet spark_catalog.default.date_dim (14) - : : +- BroadcastExchange (31) - : : +- * ColumnarToRow (30) - : : +- CometProject (29) - : : +- CometBroadcastHashJoin (28) - : : :- CometScan parquet spark_catalog.default.catalog_sales (23) - : : +- CometBroadcastExchange (27) - : : +- CometProject (26) - : : +- CometFilter (25) - : : +- CometScan parquet spark_catalog.default.date_dim (24) - : +- BroadcastExchange (38) - : +- * ColumnarToRow (37) - : +- CometFilter (36) - : +- CometScan parquet spark_catalog.default.customer_address (35) - +- BroadcastExchange (44) - +- * ColumnarToRow (43) - +- CometFilter (42) - +- CometScan parquet spark_catalog.default.customer_demographics (41) + : : : +- ReusedExchange (14) + : : +- BroadcastExchange (25) + : : +- * ColumnarToRow (24) + : : +- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometScan parquet spark_catalog.default.catalog_sales (20) + : : +- ReusedExchange (21) + : +- BroadcastExchange (32) + : +- * ColumnarToRow (31) + : +- CometFilter (30) + : +- CometScan parquet spark_catalog.default.customer_address (29) + +- BroadcastExchange (38) + +- * ColumnarToRow (37) + +- CometFilter (36) + +- CometScan parquet spark_catalog.default.customer_demographics (35) (1) Scan parquet spark_catalog.default.customer @@ -116,215 +110,183 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#14)] ReadSchema: struct -(14) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Condition : ((((isnotnull(d_year#16) AND isnotnull(d_qoy#17)) AND (d_year#16 = 2002)) AND (d_qoy#17 < 4)) AND isnotnull(d_date_sk#15)) - -(16) CometProject -Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Arguments: [d_date_sk#15], [d_date_sk#15] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#15] -Arguments: [d_date_sk#15] +(14) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#15] -(18) CometBroadcastHashJoin +(15) CometBroadcastHashJoin Left output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] Right output [1]: [d_date_sk#15] Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner -(19) CometProject +(16) CometProject Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] -(20) ColumnarToRow [codegen id : 1] +(17) ColumnarToRow [codegen id : 1] Input [1]: [ws_bill_customer_sk#12] -(21) BroadcastExchange +(18) BroadcastExchange Input [1]: [ws_bill_customer_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(22) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#12] Join type: ExistenceJoin(exists#2) Join condition: None -(23) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] +(20) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#20)] +PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] ReadSchema: struct -(24) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#21, d_year#22, d_qoy#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [3]: [d_date_sk#21, d_year#22, d_qoy#23] -Condition : ((((isnotnull(d_year#22) AND isnotnull(d_qoy#23)) AND (d_year#22 = 2002)) AND (d_qoy#23 < 4)) AND isnotnull(d_date_sk#21)) - -(26) CometProject -Input [3]: [d_date_sk#21, d_year#22, d_qoy#23] -Arguments: [d_date_sk#21], [d_date_sk#21] +(21) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#19] -(27) CometBroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: [d_date_sk#21] +(22) CometBroadcastHashJoin +Left output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Right output [1]: [d_date_sk#19] +Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner -(28) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] -Right output [1]: [d_date_sk#21] -Arguments: [cs_sold_date_sk#19], [d_date_sk#21], Inner +(23) CometProject +Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] +Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] -(29) CometProject -Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#21] -Arguments: [cs_ship_customer_sk#18], [cs_ship_customer_sk#18] +(24) ColumnarToRow [codegen id : 2] +Input [1]: [cs_ship_customer_sk#16] -(30) ColumnarToRow [codegen id : 2] -Input [1]: [cs_ship_customer_sk#18] - -(31) BroadcastExchange -Input [1]: [cs_ship_customer_sk#18] +(25) BroadcastExchange +Input [1]: [cs_ship_customer_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(32) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#18] +Right keys [1]: [cs_ship_customer_sk#16] Join type: ExistenceJoin(exists#1) Join condition: None -(33) Filter [codegen id : 5] +(27) Filter [codegen id : 5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(34) Project [codegen id : 5] +(28) Project [codegen id : 5] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(35) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#24, ca_state#25] +(29) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#20, ca_state#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(36) CometFilter -Input [2]: [ca_address_sk#24, ca_state#25] -Condition : isnotnull(ca_address_sk#24) +(30) CometFilter +Input [2]: [ca_address_sk#20, ca_state#21] +Condition : isnotnull(ca_address_sk#20) -(37) ColumnarToRow [codegen id : 3] -Input [2]: [ca_address_sk#24, ca_state#25] +(31) ColumnarToRow [codegen id : 3] +Input [2]: [ca_address_sk#20, ca_state#21] -(38) BroadcastExchange -Input [2]: [ca_address_sk#24, ca_state#25] +(32) BroadcastExchange +Input [2]: [ca_address_sk#20, ca_state#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(39) BroadcastHashJoin [codegen id : 5] +(33) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#24] +Right keys [1]: [ca_address_sk#20] Join type: Inner Join condition: None -(40) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#4, ca_state#25] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#24, ca_state#25] +(34) Project [codegen id : 5] +Output [2]: [c_current_cdemo_sk#4, ca_state#21] +Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#21] -(41) Scan parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +(35) Scan parquet spark_catalog.default.customer_demographics +Output [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(42) CometFilter -Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Condition : isnotnull(cd_demo_sk#26) +(36) CometFilter +Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Condition : isnotnull(cd_demo_sk#22) -(43) ColumnarToRow [codegen id : 4] -Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +(37) ColumnarToRow [codegen id : 4] +Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -(44) BroadcastExchange -Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +(38) BroadcastExchange +Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(45) BroadcastHashJoin [codegen id : 5] +(39) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#26] +Right keys [1]: [cd_demo_sk#22] Join type: Inner Join condition: None -(46) Project [codegen id : 5] -Output [6]: [ca_state#25, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Input [8]: [c_current_cdemo_sk#4, ca_state#25, cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] - -(47) HashAggregate [codegen id : 5] -Input [6]: [ca_state#25, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Keys [6]: [ca_state#25, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Functions [10]: [partial_count(1), partial_min(cd_dep_count#29), partial_max(cd_dep_count#29), partial_avg(cd_dep_count#29), partial_min(cd_dep_employed_count#30), partial_max(cd_dep_employed_count#30), partial_avg(cd_dep_employed_count#30), partial_min(cd_dep_college_count#31), partial_max(cd_dep_college_count#31), partial_avg(cd_dep_college_count#31)] -Aggregate Attributes [13]: [count#32, min#33, max#34, sum#35, count#36, min#37, max#38, sum#39, count#40, min#41, max#42, sum#43, count#44] -Results [19]: [ca_state#25, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#45, min#46, max#47, sum#48, count#49, min#50, max#51, sum#52, count#53, min#54, max#55, sum#56, count#57] - -(48) Exchange -Input [19]: [ca_state#25, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#45, min#46, max#47, sum#48, count#49, min#50, max#51, sum#52, count#53, min#54, max#55, sum#56, count#57] -Arguments: hashpartitioning(ca_state#25, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(49) HashAggregate [codegen id : 6] -Input [19]: [ca_state#25, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#45, min#46, max#47, sum#48, count#49, min#50, max#51, sum#52, count#53, min#54, max#55, sum#56, count#57] -Keys [6]: [ca_state#25, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Functions [10]: [count(1), min(cd_dep_count#29), max(cd_dep_count#29), avg(cd_dep_count#29), min(cd_dep_employed_count#30), max(cd_dep_employed_count#30), avg(cd_dep_employed_count#30), min(cd_dep_college_count#31), max(cd_dep_college_count#31), avg(cd_dep_college_count#31)] -Aggregate Attributes [10]: [count(1)#58, min(cd_dep_count#29)#59, max(cd_dep_count#29)#60, avg(cd_dep_count#29)#61, min(cd_dep_employed_count#30)#62, max(cd_dep_employed_count#30)#63, avg(cd_dep_employed_count#30)#64, min(cd_dep_college_count#31)#65, max(cd_dep_college_count#31)#66, avg(cd_dep_college_count#31)#67] -Results [18]: [ca_state#25, cd_gender#27, cd_marital_status#28, count(1)#58 AS cnt1#68, min(cd_dep_count#29)#59 AS min(cd_dep_count)#69, max(cd_dep_count#29)#60 AS max(cd_dep_count)#70, avg(cd_dep_count#29)#61 AS avg(cd_dep_count)#71, cd_dep_employed_count#30, count(1)#58 AS cnt2#72, min(cd_dep_employed_count#30)#62 AS min(cd_dep_employed_count)#73, max(cd_dep_employed_count#30)#63 AS max(cd_dep_employed_count)#74, avg(cd_dep_employed_count#30)#64 AS avg(cd_dep_employed_count)#75, cd_dep_college_count#31, count(1)#58 AS cnt3#76, min(cd_dep_college_count#31)#65 AS min(cd_dep_college_count)#77, max(cd_dep_college_count#31)#66 AS max(cd_dep_college_count)#78, avg(cd_dep_college_count#31)#67 AS avg(cd_dep_college_count)#79, cd_dep_count#29] - -(50) TakeOrderedAndProject -Input [18]: [ca_state#25, cd_gender#27, cd_marital_status#28, cnt1#68, min(cd_dep_count)#69, max(cd_dep_count)#70, avg(cd_dep_count)#71, cd_dep_employed_count#30, cnt2#72, min(cd_dep_employed_count)#73, max(cd_dep_employed_count)#74, avg(cd_dep_employed_count)#75, cd_dep_college_count#31, cnt3#76, min(cd_dep_college_count)#77, max(cd_dep_college_count)#78, avg(cd_dep_college_count)#79, cd_dep_count#29] -Arguments: 100, [ca_state#25 ASC NULLS FIRST, cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, cd_dep_count#29 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [ca_state#25, cd_gender#27, cd_marital_status#28, cnt1#68, min(cd_dep_count)#69, max(cd_dep_count)#70, avg(cd_dep_count)#71, cd_dep_employed_count#30, cnt2#72, min(cd_dep_employed_count)#73, max(cd_dep_employed_count)#74, avg(cd_dep_employed_count)#75, cd_dep_college_count#31, cnt3#76, min(cd_dep_college_count)#77, max(cd_dep_college_count)#78, avg(cd_dep_college_count)#79] +(40) Project [codegen id : 5] +Output [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Input [8]: [c_current_cdemo_sk#4, ca_state#21, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] + +(41) HashAggregate [codegen id : 5] +Input [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Keys [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Functions [10]: [partial_count(1), partial_min(cd_dep_count#25), partial_max(cd_dep_count#25), partial_avg(cd_dep_count#25), partial_min(cd_dep_employed_count#26), partial_max(cd_dep_employed_count#26), partial_avg(cd_dep_employed_count#26), partial_min(cd_dep_college_count#27), partial_max(cd_dep_college_count#27), partial_avg(cd_dep_college_count#27)] +Aggregate Attributes [13]: [count#28, min#29, max#30, sum#31, count#32, min#33, max#34, sum#35, count#36, min#37, max#38, sum#39, count#40] +Results [19]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#41, min#42, max#43, sum#44, count#45, min#46, max#47, sum#48, count#49, min#50, max#51, sum#52, count#53] + +(42) Exchange +Input [19]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#41, min#42, max#43, sum#44, count#45, min#46, max#47, sum#48, count#49, min#50, max#51, sum#52, count#53] +Arguments: hashpartitioning(ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(43) HashAggregate [codegen id : 6] +Input [19]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#41, min#42, max#43, sum#44, count#45, min#46, max#47, sum#48, count#49, min#50, max#51, sum#52, count#53] +Keys [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Functions [10]: [count(1), min(cd_dep_count#25), max(cd_dep_count#25), avg(cd_dep_count#25), min(cd_dep_employed_count#26), max(cd_dep_employed_count#26), avg(cd_dep_employed_count#26), min(cd_dep_college_count#27), max(cd_dep_college_count#27), avg(cd_dep_college_count#27)] +Aggregate Attributes [10]: [count(1)#54, min(cd_dep_count#25)#55, max(cd_dep_count#25)#56, avg(cd_dep_count#25)#57, min(cd_dep_employed_count#26)#58, max(cd_dep_employed_count#26)#59, avg(cd_dep_employed_count#26)#60, min(cd_dep_college_count#27)#61, max(cd_dep_college_count#27)#62, avg(cd_dep_college_count#27)#63] +Results [18]: [ca_state#21, cd_gender#23, cd_marital_status#24, count(1)#54 AS cnt1#64, min(cd_dep_count#25)#55 AS min(cd_dep_count)#65, max(cd_dep_count#25)#56 AS max(cd_dep_count)#66, avg(cd_dep_count#25)#57 AS avg(cd_dep_count)#67, cd_dep_employed_count#26, count(1)#54 AS cnt2#68, min(cd_dep_employed_count#26)#58 AS min(cd_dep_employed_count)#69, max(cd_dep_employed_count#26)#59 AS max(cd_dep_employed_count)#70, avg(cd_dep_employed_count#26)#60 AS avg(cd_dep_employed_count)#71, cd_dep_college_count#27, count(1)#54 AS cnt3#72, min(cd_dep_college_count#27)#61 AS min(cd_dep_college_count)#73, max(cd_dep_college_count#27)#62 AS max(cd_dep_college_count)#74, avg(cd_dep_college_count#27)#63 AS avg(cd_dep_college_count)#75, cd_dep_count#25] + +(44) TakeOrderedAndProject +Input [18]: [ca_state#21, cd_gender#23, cd_marital_status#24, cnt1#64, min(cd_dep_count)#65, max(cd_dep_count)#66, avg(cd_dep_count)#67, cd_dep_employed_count#26, cnt2#68, min(cd_dep_employed_count)#69, max(cd_dep_employed_count)#70, avg(cd_dep_employed_count)#71, cd_dep_college_count#27, cnt3#72, min(cd_dep_college_count)#73, max(cd_dep_college_count)#74, avg(cd_dep_college_count)#75, cd_dep_count#25] +Arguments: 100, [ca_state#21 ASC NULLS FIRST, cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_dep_count#25 ASC NULLS FIRST, cd_dep_employed_count#26 ASC NULLS FIRST, cd_dep_college_count#27 ASC NULLS FIRST], [ca_state#21, cd_gender#23, cd_marital_status#24, cnt1#64, min(cd_dep_count)#65, max(cd_dep_count)#66, avg(cd_dep_count)#67, cd_dep_employed_count#26, cnt2#68, min(cd_dep_employed_count)#69, max(cd_dep_employed_count)#70, avg(cd_dep_employed_count)#71, cd_dep_college_count#27, cnt3#72, min(cd_dep_college_count)#73, max(cd_dep_college_count)#74, avg(cd_dep_college_count)#75] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (55) -+- * ColumnarToRow (54) - +- CometProject (53) - +- CometFilter (52) - +- CometScan parquet spark_catalog.default.date_dim (51) +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan parquet spark_catalog.default.date_dim (45) -(51) Scan parquet spark_catalog.default.date_dim +(45) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(52) CometFilter +(46) CometFilter Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) -(53) CometProject +(47) CometProject Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(54) ColumnarToRow [codegen id : 1] +(48) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(55) BroadcastExchange +(49) BroadcastExchange Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt index 3cec29351..efe0b0b4e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt @@ -43,12 +43,9 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #6 - CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #7 + BroadcastExchange #6 WholeStageCodegen (2) ColumnarToRow InputAdapter @@ -56,19 +53,16 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #8 - CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #9 + BroadcastExchange #7 WholeStageCodegen (3) ColumnarToRow InputAdapter CometFilter [ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #10 + BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt index cfe10c2d1..b1e58ca83 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -* HashAggregate (59) -+- Exchange (58) - +- * HashAggregate (57) - +- * Project (56) - +- * BroadcastHashJoin LeftSemi BuildRight (55) - :- * BroadcastHashJoin LeftSemi BuildRight (36) +* HashAggregate (49) ++- Exchange (48) + +- * HashAggregate (47) + +- * Project (46) + +- * BroadcastHashJoin LeftSemi BuildRight (45) + :- * BroadcastHashJoin LeftSemi BuildRight (31) : :- * HashAggregate (17) : : +- Exchange (16) : : +- * ColumnarToRow (15) @@ -22,42 +22,32 @@ : : +- CometBroadcastExchange (11) : : +- CometFilter (10) : : +- CometScan parquet spark_catalog.default.customer (9) - : +- BroadcastExchange (35) - : +- * HashAggregate (34) - : +- Exchange (33) - : +- * ColumnarToRow (32) - : +- CometHashAggregate (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometProject (25) - : : +- CometBroadcastHashJoin (24) + : +- BroadcastExchange (30) + : +- * HashAggregate (29) + : +- Exchange (28) + : +- * ColumnarToRow (27) + : +- CometHashAggregate (26) + : +- CometProject (25) + : +- CometBroadcastHashJoin (24) + : :- CometProject (22) + : : +- CometBroadcastHashJoin (21) : : :- CometFilter (19) : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) - : : +- CometBroadcastExchange (23) - : : +- CometProject (22) - : : +- CometFilter (21) - : : +- CometScan parquet spark_catalog.default.date_dim (20) - : +- CometBroadcastExchange (28) - : +- CometFilter (27) - : +- CometScan parquet spark_catalog.default.customer (26) - +- BroadcastExchange (54) - +- * HashAggregate (53) - +- Exchange (52) - +- * ColumnarToRow (51) - +- CometHashAggregate (50) - +- CometProject (49) - +- CometBroadcastHashJoin (48) - :- CometProject (44) - : +- CometBroadcastHashJoin (43) - : :- CometFilter (38) - : : +- CometScan parquet spark_catalog.default.web_sales (37) - : +- CometBroadcastExchange (42) - : +- CometProject (41) - : +- CometFilter (40) - : +- CometScan parquet spark_catalog.default.date_dim (39) - +- CometBroadcastExchange (47) - +- CometFilter (46) - +- CometScan parquet spark_catalog.default.customer (45) + : : +- ReusedExchange (20) + : +- ReusedExchange (23) + +- BroadcastExchange (44) + +- * HashAggregate (43) + +- Exchange (42) + +- * ColumnarToRow (41) + +- CometHashAggregate (40) + +- CometProject (39) + +- CometBroadcastHashJoin (38) + :- CometProject (36) + : +- CometBroadcastHashJoin (35) + : :- CometFilter (33) + : : +- CometScan parquet spark_catalog.default.web_sales (32) + : +- ReusedExchange (34) + +- ReusedExchange (37) (1) Scan parquet spark_catalog.default.store_sales @@ -155,236 +145,180 @@ ReadSchema: struct Input [2]: [cs_bill_customer_sk#10, cs_sold_date_sk#11] Condition : isnotnull(cs_bill_customer_sk#10) -(20) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_date#14, d_month_seq#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [d_date_sk#13, d_date#14, d_month_seq#15] -Condition : (((isnotnull(d_month_seq#15) AND (d_month_seq#15 >= 1200)) AND (d_month_seq#15 <= 1211)) AND isnotnull(d_date_sk#13)) +(20) ReusedExchange [Reuses operator id: 6] +Output [2]: [d_date_sk#13, d_date#14] -(22) CometProject -Input [3]: [d_date_sk#13, d_date#14, d_month_seq#15] -Arguments: [d_date_sk#13, d_date#14], [d_date_sk#13, d_date#14] - -(23) CometBroadcastExchange -Input [2]: [d_date_sk#13, d_date#14] -Arguments: [d_date_sk#13, d_date#14] - -(24) CometBroadcastHashJoin +(21) CometBroadcastHashJoin Left output [2]: [cs_bill_customer_sk#10, cs_sold_date_sk#11] Right output [2]: [d_date_sk#13, d_date#14] Arguments: [cs_sold_date_sk#11], [d_date_sk#13], Inner -(25) CometProject +(22) CometProject Input [4]: [cs_bill_customer_sk#10, cs_sold_date_sk#11, d_date_sk#13, d_date#14] Arguments: [cs_bill_customer_sk#10, d_date#14], [cs_bill_customer_sk#10, d_date#14] -(26) Scan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#16, c_first_name#17, c_last_name#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(27) CometFilter -Input [3]: [c_customer_sk#16, c_first_name#17, c_last_name#18] -Condition : isnotnull(c_customer_sk#16) - -(28) CometBroadcastExchange -Input [3]: [c_customer_sk#16, c_first_name#17, c_last_name#18] -Arguments: [c_customer_sk#16, c_first_name#17, c_last_name#18] +(23) ReusedExchange [Reuses operator id: 11] +Output [3]: [c_customer_sk#15, c_first_name#16, c_last_name#17] -(29) CometBroadcastHashJoin +(24) CometBroadcastHashJoin Left output [2]: [cs_bill_customer_sk#10, d_date#14] -Right output [3]: [c_customer_sk#16, c_first_name#17, c_last_name#18] -Arguments: [cs_bill_customer_sk#10], [c_customer_sk#16], Inner +Right output [3]: [c_customer_sk#15, c_first_name#16, c_last_name#17] +Arguments: [cs_bill_customer_sk#10], [c_customer_sk#15], Inner -(30) CometProject -Input [5]: [cs_bill_customer_sk#10, d_date#14, c_customer_sk#16, c_first_name#17, c_last_name#18] -Arguments: [c_last_name#18, c_first_name#17, d_date#14], [c_last_name#18, c_first_name#17, d_date#14] +(25) CometProject +Input [5]: [cs_bill_customer_sk#10, d_date#14, c_customer_sk#15, c_first_name#16, c_last_name#17] +Arguments: [c_last_name#17, c_first_name#16, d_date#14], [c_last_name#17, c_first_name#16, d_date#14] -(31) CometHashAggregate -Input [3]: [c_last_name#18, c_first_name#17, d_date#14] -Keys [3]: [c_last_name#18, c_first_name#17, d_date#14] +(26) CometHashAggregate +Input [3]: [c_last_name#17, c_first_name#16, d_date#14] +Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] Functions: [] -(32) ColumnarToRow [codegen id : 2] -Input [3]: [c_last_name#18, c_first_name#17, d_date#14] +(27) ColumnarToRow [codegen id : 2] +Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -(33) Exchange -Input [3]: [c_last_name#18, c_first_name#17, d_date#14] -Arguments: hashpartitioning(c_last_name#18, c_first_name#17, d_date#14, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(28) Exchange +Input [3]: [c_last_name#17, c_first_name#16, d_date#14] +Arguments: hashpartitioning(c_last_name#17, c_first_name#16, d_date#14, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(34) HashAggregate [codegen id : 3] -Input [3]: [c_last_name#18, c_first_name#17, d_date#14] -Keys [3]: [c_last_name#18, c_first_name#17, d_date#14] +(29) HashAggregate [codegen id : 3] +Input [3]: [c_last_name#17, c_first_name#16, d_date#14] +Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#18, c_first_name#17, d_date#14] +Results [3]: [c_last_name#17, c_first_name#16, d_date#14] -(35) BroadcastExchange -Input [3]: [c_last_name#18, c_first_name#17, d_date#14] +(30) BroadcastExchange +Input [3]: [c_last_name#17, c_first_name#16, d_date#14] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=3] -(36) BroadcastHashJoin [codegen id : 6] +(31) BroadcastHashJoin [codegen id : 6] Left keys [6]: [coalesce(c_last_name#9, ), isnull(c_last_name#9), coalesce(c_first_name#8, ), isnull(c_first_name#8), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#18, ), isnull(c_last_name#18), coalesce(c_first_name#17, ), isnull(c_first_name#17), coalesce(d_date#14, 1970-01-01), isnull(d_date#14)] +Right keys [6]: [coalesce(c_last_name#17, ), isnull(c_last_name#17), coalesce(c_first_name#16, ), isnull(c_first_name#16), coalesce(d_date#14, 1970-01-01), isnull(d_date#14)] Join type: LeftSemi Join condition: None -(37) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#19, ws_sold_date_sk#20] +(32) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#20), dynamicpruningexpression(ws_sold_date_sk#20 IN dynamicpruning#21)] +PartitionFilters: [isnotnull(ws_sold_date_sk#19), dynamicpruningexpression(ws_sold_date_sk#19 IN dynamicpruning#20)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(38) CometFilter -Input [2]: [ws_bill_customer_sk#19, ws_sold_date_sk#20] -Condition : isnotnull(ws_bill_customer_sk#19) +(33) CometFilter +Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] +Condition : isnotnull(ws_bill_customer_sk#18) -(39) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#22, d_date#23, d_month_seq#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct +(34) ReusedExchange [Reuses operator id: 6] +Output [2]: [d_date_sk#21, d_date#22] -(40) CometFilter -Input [3]: [d_date_sk#22, d_date#23, d_month_seq#24] -Condition : (((isnotnull(d_month_seq#24) AND (d_month_seq#24 >= 1200)) AND (d_month_seq#24 <= 1211)) AND isnotnull(d_date_sk#22)) +(35) CometBroadcastHashJoin +Left output [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] +Right output [2]: [d_date_sk#21, d_date#22] +Arguments: [ws_sold_date_sk#19], [d_date_sk#21], Inner -(41) CometProject -Input [3]: [d_date_sk#22, d_date#23, d_month_seq#24] -Arguments: [d_date_sk#22, d_date#23], [d_date_sk#22, d_date#23] +(36) CometProject +Input [4]: [ws_bill_customer_sk#18, ws_sold_date_sk#19, d_date_sk#21, d_date#22] +Arguments: [ws_bill_customer_sk#18, d_date#22], [ws_bill_customer_sk#18, d_date#22] -(42) CometBroadcastExchange -Input [2]: [d_date_sk#22, d_date#23] -Arguments: [d_date_sk#22, d_date#23] +(37) ReusedExchange [Reuses operator id: 11] +Output [3]: [c_customer_sk#23, c_first_name#24, c_last_name#25] -(43) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#19, ws_sold_date_sk#20] -Right output [2]: [d_date_sk#22, d_date#23] -Arguments: [ws_sold_date_sk#20], [d_date_sk#22], Inner +(38) CometBroadcastHashJoin +Left output [2]: [ws_bill_customer_sk#18, d_date#22] +Right output [3]: [c_customer_sk#23, c_first_name#24, c_last_name#25] +Arguments: [ws_bill_customer_sk#18], [c_customer_sk#23], Inner -(44) CometProject -Input [4]: [ws_bill_customer_sk#19, ws_sold_date_sk#20, d_date_sk#22, d_date#23] -Arguments: [ws_bill_customer_sk#19, d_date#23], [ws_bill_customer_sk#19, d_date#23] +(39) CometProject +Input [5]: [ws_bill_customer_sk#18, d_date#22, c_customer_sk#23, c_first_name#24, c_last_name#25] +Arguments: [c_last_name#25, c_first_name#24, d_date#22], [c_last_name#25, c_first_name#24, d_date#22] -(45) Scan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(46) CometFilter -Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Condition : isnotnull(c_customer_sk#25) - -(47) CometBroadcastExchange -Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: [c_customer_sk#25, c_first_name#26, c_last_name#27] - -(48) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#19, d_date#23] -Right output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: [ws_bill_customer_sk#19], [c_customer_sk#25], Inner - -(49) CometProject -Input [5]: [ws_bill_customer_sk#19, d_date#23, c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: [c_last_name#27, c_first_name#26, d_date#23], [c_last_name#27, c_first_name#26, d_date#23] - -(50) CometHashAggregate -Input [3]: [c_last_name#27, c_first_name#26, d_date#23] -Keys [3]: [c_last_name#27, c_first_name#26, d_date#23] +(40) CometHashAggregate +Input [3]: [c_last_name#25, c_first_name#24, d_date#22] +Keys [3]: [c_last_name#25, c_first_name#24, d_date#22] Functions: [] -(51) ColumnarToRow [codegen id : 4] -Input [3]: [c_last_name#27, c_first_name#26, d_date#23] +(41) ColumnarToRow [codegen id : 4] +Input [3]: [c_last_name#25, c_first_name#24, d_date#22] -(52) Exchange -Input [3]: [c_last_name#27, c_first_name#26, d_date#23] -Arguments: hashpartitioning(c_last_name#27, c_first_name#26, d_date#23, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(42) Exchange +Input [3]: [c_last_name#25, c_first_name#24, d_date#22] +Arguments: hashpartitioning(c_last_name#25, c_first_name#24, d_date#22, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(53) HashAggregate [codegen id : 5] -Input [3]: [c_last_name#27, c_first_name#26, d_date#23] -Keys [3]: [c_last_name#27, c_first_name#26, d_date#23] +(43) HashAggregate [codegen id : 5] +Input [3]: [c_last_name#25, c_first_name#24, d_date#22] +Keys [3]: [c_last_name#25, c_first_name#24, d_date#22] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#27, c_first_name#26, d_date#23] +Results [3]: [c_last_name#25, c_first_name#24, d_date#22] -(54) BroadcastExchange -Input [3]: [c_last_name#27, c_first_name#26, d_date#23] +(44) BroadcastExchange +Input [3]: [c_last_name#25, c_first_name#24, d_date#22] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=5] -(55) BroadcastHashJoin [codegen id : 6] +(45) BroadcastHashJoin [codegen id : 6] Left keys [6]: [coalesce(c_last_name#9, ), isnull(c_last_name#9), coalesce(c_first_name#8, ), isnull(c_first_name#8), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#27, ), isnull(c_last_name#27), coalesce(c_first_name#26, ), isnull(c_first_name#26), coalesce(d_date#23, 1970-01-01), isnull(d_date#23)] +Right keys [6]: [coalesce(c_last_name#25, ), isnull(c_last_name#25), coalesce(c_first_name#24, ), isnull(c_first_name#24), coalesce(d_date#22, 1970-01-01), isnull(d_date#22)] Join type: LeftSemi Join condition: None -(56) Project [codegen id : 6] +(46) Project [codegen id : 6] Output: [] Input [3]: [c_last_name#9, c_first_name#8, d_date#5] -(57) HashAggregate [codegen id : 6] +(47) HashAggregate [codegen id : 6] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#28] -Results [1]: [count#29] +Aggregate Attributes [1]: [count#26] +Results [1]: [count#27] -(58) Exchange -Input [1]: [count#29] +(48) Exchange +Input [1]: [count#27] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(59) HashAggregate [codegen id : 7] -Input [1]: [count#29] +(49) HashAggregate [codegen id : 7] +Input [1]: [count#27] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#30] -Results [1]: [count(1)#30 AS count(1)#31] +Aggregate Attributes [1]: [count(1)#28] +Results [1]: [count(1)#28 AS count(1)#29] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 -BroadcastExchange (64) -+- * ColumnarToRow (63) - +- CometProject (62) - +- CometFilter (61) - +- CometScan parquet spark_catalog.default.date_dim (60) +BroadcastExchange (54) ++- * ColumnarToRow (53) + +- CometProject (52) + +- CometFilter (51) + +- CometScan parquet spark_catalog.default.date_dim (50) -(60) Scan parquet spark_catalog.default.date_dim +(50) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#4, d_date#5, d_month_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(61) CometFilter +(51) CometFilter Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#4)) -(62) CometProject +(52) CometProject Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(63) ColumnarToRow [codegen id : 1] +(53) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#4, d_date#5] -(64) BroadcastExchange +(54) BroadcastExchange Input [2]: [d_date_sk#4, d_date#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] Subquery:2 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#11 IN dynamicpruning#3 -Subquery:3 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#20 IN dynamicpruning#3 +Subquery:3 Hosting operator id = 32 Hosting Expression = ws_sold_date_sk#19 IN dynamicpruning#3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt index 2dae3c911..cfac83844 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt @@ -52,19 +52,14 @@ WholeStageCodegen (7) CometFilter [cs_bill_customer_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #8 - CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #9 - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] + ReusedExchange [d_date_sk,d_date] #4 + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 InputAdapter - BroadcastExchange #10 + BroadcastExchange #8 WholeStageCodegen (5) HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_last_name,c_first_name,d_date] #11 + Exchange [c_last_name,c_first_name,d_date] #9 WholeStageCodegen (4) ColumnarToRow InputAdapter @@ -76,10 +71,5 @@ WholeStageCodegen (7) CometFilter [ws_bill_customer_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #12 - CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #13 - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] + ReusedExchange [d_date_sk,d_date] #4 + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt index d14525e5f..7bf41b1ef 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* Sort (52) -+- Exchange (51) - +- * BroadcastHashJoin Inner BuildRight (50) +* Sort (48) ++- Exchange (47) + +- * BroadcastHashJoin Inner BuildRight (46) :- * Project (24) : +- * Filter (23) : +- * HashAggregate (22) @@ -26,31 +26,27 @@ : +- CometProject (15) : +- CometFilter (14) : +- CometScan parquet spark_catalog.default.date_dim (13) - +- BroadcastExchange (49) - +- * Project (48) - +- * Filter (47) - +- * HashAggregate (46) - +- Exchange (45) - +- * ColumnarToRow (44) - +- CometHashAggregate (43) - +- CometProject (42) - +- CometBroadcastHashJoin (41) - :- CometProject (36) - : +- CometBroadcastHashJoin (35) - : :- CometProject (31) - : : +- CometBroadcastHashJoin (30) + +- BroadcastExchange (45) + +- * Project (44) + +- * Filter (43) + +- * HashAggregate (42) + +- Exchange (41) + +- * ColumnarToRow (40) + +- CometHashAggregate (39) + +- CometProject (38) + +- CometBroadcastHashJoin (37) + :- CometProject (32) + : +- CometBroadcastHashJoin (31) + : :- CometProject (29) + : : +- CometBroadcastHashJoin (28) : : :- CometFilter (26) : : : +- CometScan parquet spark_catalog.default.inventory (25) - : : +- CometBroadcastExchange (29) - : : +- CometFilter (28) - : : +- CometScan parquet spark_catalog.default.item (27) - : +- CometBroadcastExchange (34) - : +- CometFilter (33) - : +- CometScan parquet spark_catalog.default.warehouse (32) - +- CometBroadcastExchange (40) - +- CometProject (39) - +- CometFilter (38) - +- CometScan parquet spark_catalog.default.date_dim (37) + : : +- ReusedExchange (27) + : +- ReusedExchange (30) + +- CometBroadcastExchange (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) (1) Scan parquet spark_catalog.default.inventory @@ -180,186 +176,162 @@ ReadSchema: struct - -(28) CometFilter -Input [1]: [i_item_sk#27] -Condition : isnotnull(i_item_sk#27) - -(29) CometBroadcastExchange -Input [1]: [i_item_sk#27] -Arguments: [i_item_sk#27] -(30) CometBroadcastHashJoin +(28) CometBroadcastHashJoin Left output [4]: [inv_item_sk#22, inv_warehouse_sk#23, inv_quantity_on_hand#24, inv_date_sk#25] Right output [1]: [i_item_sk#27] Arguments: [inv_item_sk#22], [i_item_sk#27], Inner -(31) CometProject +(29) CometProject Input [5]: [inv_item_sk#22, inv_warehouse_sk#23, inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27] Arguments: [inv_warehouse_sk#23, inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27], [inv_warehouse_sk#23, inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27] -(32) Scan parquet spark_catalog.default.warehouse +(30) ReusedExchange [Reuses operator id: 10] Output [2]: [w_warehouse_sk#28, w_warehouse_name#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [w_warehouse_sk#28, w_warehouse_name#29] -Condition : isnotnull(w_warehouse_sk#28) - -(34) CometBroadcastExchange -Input [2]: [w_warehouse_sk#28, w_warehouse_name#29] -Arguments: [w_warehouse_sk#28, w_warehouse_name#29] -(35) CometBroadcastHashJoin +(31) CometBroadcastHashJoin Left output [4]: [inv_warehouse_sk#23, inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27] Right output [2]: [w_warehouse_sk#28, w_warehouse_name#29] Arguments: [inv_warehouse_sk#23], [w_warehouse_sk#28], Inner -(36) CometProject +(32) CometProject Input [6]: [inv_warehouse_sk#23, inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27, w_warehouse_sk#28, w_warehouse_name#29] Arguments: [inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27, w_warehouse_sk#28, w_warehouse_name#29], [inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27, w_warehouse_sk#28, w_warehouse_name#29] -(37) Scan parquet spark_catalog.default.date_dim +(33) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#30, d_year#31, d_moy#32] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(38) CometFilter +(34) CometFilter Input [3]: [d_date_sk#30, d_year#31, d_moy#32] Condition : ((((isnotnull(d_year#31) AND isnotnull(d_moy#32)) AND (d_year#31 = 2001)) AND (d_moy#32 = 2)) AND isnotnull(d_date_sk#30)) -(39) CometProject +(35) CometProject Input [3]: [d_date_sk#30, d_year#31, d_moy#32] Arguments: [d_date_sk#30, d_moy#32], [d_date_sk#30, d_moy#32] -(40) CometBroadcastExchange +(36) CometBroadcastExchange Input [2]: [d_date_sk#30, d_moy#32] Arguments: [d_date_sk#30, d_moy#32] -(41) CometBroadcastHashJoin +(37) CometBroadcastHashJoin Left output [5]: [inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27, w_warehouse_sk#28, w_warehouse_name#29] Right output [2]: [d_date_sk#30, d_moy#32] Arguments: [inv_date_sk#25], [d_date_sk#30], Inner -(42) CometProject +(38) CometProject Input [7]: [inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27, w_warehouse_sk#28, w_warehouse_name#29, d_date_sk#30, d_moy#32] Arguments: [inv_quantity_on_hand#24, i_item_sk#27, w_warehouse_sk#28, w_warehouse_name#29, d_moy#32], [inv_quantity_on_hand#24, i_item_sk#27, w_warehouse_sk#28, w_warehouse_name#29, d_moy#32] -(43) CometHashAggregate +(39) CometHashAggregate Input [5]: [inv_quantity_on_hand#24, i_item_sk#27, w_warehouse_sk#28, w_warehouse_name#29, d_moy#32] Keys [4]: [w_warehouse_name#29, w_warehouse_sk#28, i_item_sk#27, d_moy#32] Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#24 as double)), partial_avg(inv_quantity_on_hand#24)] -(44) ColumnarToRow [codegen id : 2] +(40) ColumnarToRow [codegen id : 2] Input [9]: [w_warehouse_name#29, w_warehouse_sk#28, i_item_sk#27, d_moy#32, n#33, avg#34, m2#35, sum#36, count#37] -(45) Exchange +(41) Exchange Input [9]: [w_warehouse_name#29, w_warehouse_sk#28, i_item_sk#27, d_moy#32, n#33, avg#34, m2#35, sum#36, count#37] Arguments: hashpartitioning(w_warehouse_name#29, w_warehouse_sk#28, i_item_sk#27, d_moy#32, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(46) HashAggregate [codegen id : 3] +(42) HashAggregate [codegen id : 3] Input [9]: [w_warehouse_name#29, w_warehouse_sk#28, i_item_sk#27, d_moy#32, n#33, avg#34, m2#35, sum#36, count#37] Keys [4]: [w_warehouse_name#29, w_warehouse_sk#28, i_item_sk#27, d_moy#32] Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#24 as double)), avg(inv_quantity_on_hand#24)] Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#24 as double))#17, avg(inv_quantity_on_hand#24)#18] Results [5]: [w_warehouse_sk#28, i_item_sk#27, d_moy#32, stddev_samp(cast(inv_quantity_on_hand#24 as double))#17 AS stdev#19, avg(inv_quantity_on_hand#24)#18 AS mean#20] -(47) Filter [codegen id : 3] +(43) Filter [codegen id : 3] Input [5]: [w_warehouse_sk#28, i_item_sk#27, d_moy#32, stdev#19, mean#20] Condition : CASE WHEN (mean#20 = 0.0) THEN false ELSE ((stdev#19 / mean#20) > 1.0) END -(48) Project [codegen id : 3] +(44) Project [codegen id : 3] Output [5]: [w_warehouse_sk#28, i_item_sk#27, d_moy#32, mean#20 AS mean#38, CASE WHEN (mean#20 = 0.0) THEN null ELSE (stdev#19 / mean#20) END AS cov#39] Input [5]: [w_warehouse_sk#28, i_item_sk#27, d_moy#32, stdev#19, mean#20] -(49) BroadcastExchange +(45) BroadcastExchange Input [5]: [w_warehouse_sk#28, i_item_sk#27, d_moy#32, mean#38, cov#39] Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=3] -(50) BroadcastHashJoin [codegen id : 4] +(46) BroadcastHashJoin [codegen id : 4] Left keys [2]: [i_item_sk#6, w_warehouse_sk#7] Right keys [2]: [i_item_sk#27, w_warehouse_sk#28] Join type: Inner Join condition: None -(51) Exchange +(47) Exchange Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#20, cov#21, w_warehouse_sk#28, i_item_sk#27, d_moy#32, mean#38, cov#39] Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#20 ASC NULLS FIRST, cov#21 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, mean#38 ASC NULLS FIRST, cov#39 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(52) Sort [codegen id : 5] +(48) Sort [codegen id : 5] Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#20, cov#21, w_warehouse_sk#28, i_item_sk#27, d_moy#32, mean#38, cov#39] Arguments: [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#20 ASC NULLS FIRST, cov#21 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, mean#38 ASC NULLS FIRST, cov#39 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (57) -+- * ColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan parquet spark_catalog.default.date_dim (53) +BroadcastExchange (53) ++- * ColumnarToRow (52) + +- CometProject (51) + +- CometFilter (50) + +- CometScan parquet spark_catalog.default.date_dim (49) -(53) Scan parquet spark_catalog.default.date_dim +(49) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#9, d_year#10, d_moy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] ReadSchema: struct -(54) CometFilter +(50) CometFilter Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2001)) AND (d_moy#11 = 1)) AND isnotnull(d_date_sk#9)) -(55) CometProject +(51) CometProject Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] -(56) ColumnarToRow [codegen id : 1] +(52) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_moy#11] -(57) BroadcastExchange +(53) BroadcastExchange Input [2]: [d_date_sk#9, d_moy#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:2 Hosting operator id = 25 Hosting Expression = inv_date_sk#25 IN dynamicpruning#26 -BroadcastExchange (62) -+- * ColumnarToRow (61) - +- CometProject (60) - +- CometFilter (59) - +- CometScan parquet spark_catalog.default.date_dim (58) +BroadcastExchange (58) ++- * ColumnarToRow (57) + +- CometProject (56) + +- CometFilter (55) + +- CometScan parquet spark_catalog.default.date_dim (54) -(58) Scan parquet spark_catalog.default.date_dim +(54) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#30, d_year#31, d_moy#32] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(59) CometFilter +(55) CometFilter Input [3]: [d_date_sk#30, d_year#31, d_moy#32] Condition : ((((isnotnull(d_year#31) AND isnotnull(d_moy#32)) AND (d_year#31 = 2001)) AND (d_moy#32 = 2)) AND isnotnull(d_date_sk#30)) -(60) CometProject +(56) CometProject Input [3]: [d_date_sk#30, d_year#31, d_moy#32] Arguments: [d_date_sk#30, d_moy#32], [d_date_sk#30, d_moy#32] -(61) ColumnarToRow [codegen id : 1] +(57) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#30, d_moy#32] -(62) BroadcastExchange +(58) BroadcastExchange Input [2]: [d_date_sk#30, d_moy#32] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt index 8fd3cfe22..7d28d5c10 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt @@ -67,13 +67,9 @@ WholeStageCodegen (5) CometProject [d_date_sk,d_moy] CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #10 - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange #11 - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #12 + ReusedExchange [i_item_sk] #4 + ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 + CometBroadcastExchange #10 CometProject [d_date_sk,d_moy] CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt index 77b9fc512..e0e2d82fe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -* Sort (52) -+- Exchange (51) - +- * BroadcastHashJoin Inner BuildRight (50) +* Sort (48) ++- Exchange (47) + +- * BroadcastHashJoin Inner BuildRight (46) :- * Project (24) : +- * Filter (23) : +- * HashAggregate (22) @@ -26,31 +26,27 @@ : +- CometProject (15) : +- CometFilter (14) : +- CometScan parquet spark_catalog.default.date_dim (13) - +- BroadcastExchange (49) - +- * Project (48) - +- * Filter (47) - +- * HashAggregate (46) - +- Exchange (45) - +- * ColumnarToRow (44) - +- CometHashAggregate (43) - +- CometProject (42) - +- CometBroadcastHashJoin (41) - :- CometProject (36) - : +- CometBroadcastHashJoin (35) - : :- CometProject (31) - : : +- CometBroadcastHashJoin (30) + +- BroadcastExchange (45) + +- * Project (44) + +- * Filter (43) + +- * HashAggregate (42) + +- Exchange (41) + +- * ColumnarToRow (40) + +- CometHashAggregate (39) + +- CometProject (38) + +- CometBroadcastHashJoin (37) + :- CometProject (32) + : +- CometBroadcastHashJoin (31) + : :- CometProject (29) + : : +- CometBroadcastHashJoin (28) : : :- CometFilter (26) : : : +- CometScan parquet spark_catalog.default.inventory (25) - : : +- CometBroadcastExchange (29) - : : +- CometFilter (28) - : : +- CometScan parquet spark_catalog.default.item (27) - : +- CometBroadcastExchange (34) - : +- CometFilter (33) - : +- CometScan parquet spark_catalog.default.warehouse (32) - +- CometBroadcastExchange (40) - +- CometProject (39) - +- CometFilter (38) - +- CometScan parquet spark_catalog.default.date_dim (37) + : : +- ReusedExchange (27) + : +- ReusedExchange (30) + +- CometBroadcastExchange (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) (1) Scan parquet spark_catalog.default.inventory @@ -180,186 +176,162 @@ ReadSchema: struct - -(28) CometFilter -Input [1]: [i_item_sk#27] -Condition : isnotnull(i_item_sk#27) - -(29) CometBroadcastExchange -Input [1]: [i_item_sk#27] -Arguments: [i_item_sk#27] -(30) CometBroadcastHashJoin +(28) CometBroadcastHashJoin Left output [4]: [inv_item_sk#22, inv_warehouse_sk#23, inv_quantity_on_hand#24, inv_date_sk#25] Right output [1]: [i_item_sk#27] Arguments: [inv_item_sk#22], [i_item_sk#27], Inner -(31) CometProject +(29) CometProject Input [5]: [inv_item_sk#22, inv_warehouse_sk#23, inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27] Arguments: [inv_warehouse_sk#23, inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27], [inv_warehouse_sk#23, inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27] -(32) Scan parquet spark_catalog.default.warehouse +(30) ReusedExchange [Reuses operator id: 10] Output [2]: [w_warehouse_sk#28, w_warehouse_name#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [w_warehouse_sk#28, w_warehouse_name#29] -Condition : isnotnull(w_warehouse_sk#28) - -(34) CometBroadcastExchange -Input [2]: [w_warehouse_sk#28, w_warehouse_name#29] -Arguments: [w_warehouse_sk#28, w_warehouse_name#29] -(35) CometBroadcastHashJoin +(31) CometBroadcastHashJoin Left output [4]: [inv_warehouse_sk#23, inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27] Right output [2]: [w_warehouse_sk#28, w_warehouse_name#29] Arguments: [inv_warehouse_sk#23], [w_warehouse_sk#28], Inner -(36) CometProject +(32) CometProject Input [6]: [inv_warehouse_sk#23, inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27, w_warehouse_sk#28, w_warehouse_name#29] Arguments: [inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27, w_warehouse_sk#28, w_warehouse_name#29], [inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27, w_warehouse_sk#28, w_warehouse_name#29] -(37) Scan parquet spark_catalog.default.date_dim +(33) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#30, d_year#31, d_moy#32] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(38) CometFilter +(34) CometFilter Input [3]: [d_date_sk#30, d_year#31, d_moy#32] Condition : ((((isnotnull(d_year#31) AND isnotnull(d_moy#32)) AND (d_year#31 = 2001)) AND (d_moy#32 = 2)) AND isnotnull(d_date_sk#30)) -(39) CometProject +(35) CometProject Input [3]: [d_date_sk#30, d_year#31, d_moy#32] Arguments: [d_date_sk#30, d_moy#32], [d_date_sk#30, d_moy#32] -(40) CometBroadcastExchange +(36) CometBroadcastExchange Input [2]: [d_date_sk#30, d_moy#32] Arguments: [d_date_sk#30, d_moy#32] -(41) CometBroadcastHashJoin +(37) CometBroadcastHashJoin Left output [5]: [inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27, w_warehouse_sk#28, w_warehouse_name#29] Right output [2]: [d_date_sk#30, d_moy#32] Arguments: [inv_date_sk#25], [d_date_sk#30], Inner -(42) CometProject +(38) CometProject Input [7]: [inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27, w_warehouse_sk#28, w_warehouse_name#29, d_date_sk#30, d_moy#32] Arguments: [inv_quantity_on_hand#24, i_item_sk#27, w_warehouse_sk#28, w_warehouse_name#29, d_moy#32], [inv_quantity_on_hand#24, i_item_sk#27, w_warehouse_sk#28, w_warehouse_name#29, d_moy#32] -(43) CometHashAggregate +(39) CometHashAggregate Input [5]: [inv_quantity_on_hand#24, i_item_sk#27, w_warehouse_sk#28, w_warehouse_name#29, d_moy#32] Keys [4]: [w_warehouse_name#29, w_warehouse_sk#28, i_item_sk#27, d_moy#32] Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#24 as double)), partial_avg(inv_quantity_on_hand#24)] -(44) ColumnarToRow [codegen id : 2] +(40) ColumnarToRow [codegen id : 2] Input [9]: [w_warehouse_name#29, w_warehouse_sk#28, i_item_sk#27, d_moy#32, n#33, avg#34, m2#35, sum#36, count#37] -(45) Exchange +(41) Exchange Input [9]: [w_warehouse_name#29, w_warehouse_sk#28, i_item_sk#27, d_moy#32, n#33, avg#34, m2#35, sum#36, count#37] Arguments: hashpartitioning(w_warehouse_name#29, w_warehouse_sk#28, i_item_sk#27, d_moy#32, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(46) HashAggregate [codegen id : 3] +(42) HashAggregate [codegen id : 3] Input [9]: [w_warehouse_name#29, w_warehouse_sk#28, i_item_sk#27, d_moy#32, n#33, avg#34, m2#35, sum#36, count#37] Keys [4]: [w_warehouse_name#29, w_warehouse_sk#28, i_item_sk#27, d_moy#32] Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#24 as double)), avg(inv_quantity_on_hand#24)] Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#24 as double))#17, avg(inv_quantity_on_hand#24)#18] Results [5]: [w_warehouse_sk#28, i_item_sk#27, d_moy#32, stddev_samp(cast(inv_quantity_on_hand#24 as double))#17 AS stdev#19, avg(inv_quantity_on_hand#24)#18 AS mean#20] -(47) Filter [codegen id : 3] +(43) Filter [codegen id : 3] Input [5]: [w_warehouse_sk#28, i_item_sk#27, d_moy#32, stdev#19, mean#20] Condition : CASE WHEN (mean#20 = 0.0) THEN false ELSE ((stdev#19 / mean#20) > 1.0) END -(48) Project [codegen id : 3] +(44) Project [codegen id : 3] Output [5]: [w_warehouse_sk#28, i_item_sk#27, d_moy#32, mean#20 AS mean#38, CASE WHEN (mean#20 = 0.0) THEN null ELSE (stdev#19 / mean#20) END AS cov#39] Input [5]: [w_warehouse_sk#28, i_item_sk#27, d_moy#32, stdev#19, mean#20] -(49) BroadcastExchange +(45) BroadcastExchange Input [5]: [w_warehouse_sk#28, i_item_sk#27, d_moy#32, mean#38, cov#39] Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=3] -(50) BroadcastHashJoin [codegen id : 4] +(46) BroadcastHashJoin [codegen id : 4] Left keys [2]: [i_item_sk#6, w_warehouse_sk#7] Right keys [2]: [i_item_sk#27, w_warehouse_sk#28] Join type: Inner Join condition: None -(51) Exchange +(47) Exchange Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#20, cov#21, w_warehouse_sk#28, i_item_sk#27, d_moy#32, mean#38, cov#39] Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#20 ASC NULLS FIRST, cov#21 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, mean#38 ASC NULLS FIRST, cov#39 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(52) Sort [codegen id : 5] +(48) Sort [codegen id : 5] Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#20, cov#21, w_warehouse_sk#28, i_item_sk#27, d_moy#32, mean#38, cov#39] Arguments: [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#20 ASC NULLS FIRST, cov#21 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, mean#38 ASC NULLS FIRST, cov#39 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (57) -+- * ColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan parquet spark_catalog.default.date_dim (53) +BroadcastExchange (53) ++- * ColumnarToRow (52) + +- CometProject (51) + +- CometFilter (50) + +- CometScan parquet spark_catalog.default.date_dim (49) -(53) Scan parquet spark_catalog.default.date_dim +(49) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#9, d_year#10, d_moy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] ReadSchema: struct -(54) CometFilter +(50) CometFilter Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2001)) AND (d_moy#11 = 1)) AND isnotnull(d_date_sk#9)) -(55) CometProject +(51) CometProject Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] -(56) ColumnarToRow [codegen id : 1] +(52) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_moy#11] -(57) BroadcastExchange +(53) BroadcastExchange Input [2]: [d_date_sk#9, d_moy#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:2 Hosting operator id = 25 Hosting Expression = inv_date_sk#25 IN dynamicpruning#26 -BroadcastExchange (62) -+- * ColumnarToRow (61) - +- CometProject (60) - +- CometFilter (59) - +- CometScan parquet spark_catalog.default.date_dim (58) +BroadcastExchange (58) ++- * ColumnarToRow (57) + +- CometProject (56) + +- CometFilter (55) + +- CometScan parquet spark_catalog.default.date_dim (54) -(58) Scan parquet spark_catalog.default.date_dim +(54) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#30, d_year#31, d_moy#32] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(59) CometFilter +(55) CometFilter Input [3]: [d_date_sk#30, d_year#31, d_moy#32] Condition : ((((isnotnull(d_year#31) AND isnotnull(d_moy#32)) AND (d_year#31 = 2001)) AND (d_moy#32 = 2)) AND isnotnull(d_date_sk#30)) -(60) CometProject +(56) CometProject Input [3]: [d_date_sk#30, d_year#31, d_moy#32] Arguments: [d_date_sk#30, d_moy#32], [d_date_sk#30, d_moy#32] -(61) ColumnarToRow [codegen id : 1] +(57) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#30, d_moy#32] -(62) BroadcastExchange +(58) BroadcastExchange Input [2]: [d_date_sk#30, d_moy#32] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt index 8fd3cfe22..7d28d5c10 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt @@ -67,13 +67,9 @@ WholeStageCodegen (5) CometProject [d_date_sk,d_moy] CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #10 - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange #11 - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #12 + ReusedExchange [i_item_sk] #4 + ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 + CometBroadcastExchange #10 CometProject [d_date_sk,d_moy] CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt index 48a8bb7fd..ce89b5e68 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt @@ -1,13 +1,13 @@ == Physical Plan == -TakeOrderedAndProject (114) -+- * Project (113) - +- * BroadcastHashJoin Inner BuildRight (112) - :- * Project (94) - : +- * BroadcastHashJoin Inner BuildRight (93) - : :- * Project (74) - : : +- * BroadcastHashJoin Inner BuildRight (73) - : : :- * Project (55) - : : : +- * BroadcastHashJoin Inner BuildRight (54) +TakeOrderedAndProject (106) ++- * Project (105) + +- * BroadcastHashJoin Inner BuildRight (104) + :- * Project (88) + : +- * BroadcastHashJoin Inner BuildRight (87) + : :- * Project (70) + : : +- * BroadcastHashJoin Inner BuildRight (69) + : : :- * Project (53) + : : : +- * BroadcastHashJoin Inner BuildRight (52) : : : :- * BroadcastHashJoin Inner BuildRight (35) : : : : :- * Filter (17) : : : : : +- * HashAggregate (16) @@ -43,14 +43,14 @@ TakeOrderedAndProject (114) : : : : +- CometBroadcastExchange (27) : : : : +- CometFilter (26) : : : : +- CometScan parquet spark_catalog.default.date_dim (25) - : : : +- BroadcastExchange (53) - : : : +- * Filter (52) - : : : +- * HashAggregate (51) - : : : +- Exchange (50) - : : : +- * ColumnarToRow (49) - : : : +- CometHashAggregate (48) - : : : +- CometProject (47) - : : : +- CometBroadcastHashJoin (46) + : : : +- BroadcastExchange (51) + : : : +- * Filter (50) + : : : +- * HashAggregate (49) + : : : +- Exchange (48) + : : : +- * ColumnarToRow (47) + : : : +- CometHashAggregate (46) + : : : +- CometProject (45) + : : : +- CometBroadcastHashJoin (44) : : : :- CometProject (42) : : : : +- CometBroadcastHashJoin (41) : : : : :- CometFilter (37) @@ -58,61 +58,53 @@ TakeOrderedAndProject (114) : : : : +- CometBroadcastExchange (40) : : : : +- CometFilter (39) : : : : +- CometScan parquet spark_catalog.default.catalog_sales (38) - : : : +- CometBroadcastExchange (45) - : : : +- CometFilter (44) - : : : +- CometScan parquet spark_catalog.default.date_dim (43) - : : +- BroadcastExchange (72) - : : +- * HashAggregate (71) - : : +- Exchange (70) - : : +- * ColumnarToRow (69) - : : +- CometHashAggregate (68) - : : +- CometProject (67) - : : +- CometBroadcastHashJoin (66) - : : :- CometProject (62) - : : : +- CometBroadcastHashJoin (61) - : : : :- CometFilter (57) - : : : : +- CometScan parquet spark_catalog.default.customer (56) - : : : +- CometBroadcastExchange (60) - : : : +- CometFilter (59) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (58) - : : +- CometBroadcastExchange (65) - : : +- CometFilter (64) - : : +- CometScan parquet spark_catalog.default.date_dim (63) - : +- BroadcastExchange (92) - : +- * Filter (91) - : +- * HashAggregate (90) - : +- Exchange (89) - : +- * ColumnarToRow (88) - : +- CometHashAggregate (87) - : +- CometProject (86) - : +- CometBroadcastHashJoin (85) - : :- CometProject (81) - : : +- CometBroadcastHashJoin (80) - : : :- CometFilter (76) - : : : +- CometScan parquet spark_catalog.default.customer (75) - : : +- CometBroadcastExchange (79) - : : +- CometFilter (78) - : : +- CometScan parquet spark_catalog.default.web_sales (77) - : +- CometBroadcastExchange (84) - : +- CometFilter (83) - : +- CometScan parquet spark_catalog.default.date_dim (82) - +- BroadcastExchange (111) - +- * HashAggregate (110) - +- Exchange (109) - +- * ColumnarToRow (108) - +- CometHashAggregate (107) - +- CometProject (106) - +- CometBroadcastHashJoin (105) - :- CometProject (101) - : +- CometBroadcastHashJoin (100) - : :- CometFilter (96) - : : +- CometScan parquet spark_catalog.default.customer (95) - : +- CometBroadcastExchange (99) - : +- CometFilter (98) - : +- CometScan parquet spark_catalog.default.web_sales (97) - +- CometBroadcastExchange (104) - +- CometFilter (103) - +- CometScan parquet spark_catalog.default.date_dim (102) + : : : +- ReusedExchange (43) + : : +- BroadcastExchange (68) + : : +- * HashAggregate (67) + : : +- Exchange (66) + : : +- * ColumnarToRow (65) + : : +- CometHashAggregate (64) + : : +- CometProject (63) + : : +- CometBroadcastHashJoin (62) + : : :- CometProject (60) + : : : +- CometBroadcastHashJoin (59) + : : : :- CometFilter (55) + : : : : +- CometScan parquet spark_catalog.default.customer (54) + : : : +- CometBroadcastExchange (58) + : : : +- CometFilter (57) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (56) + : : +- ReusedExchange (61) + : +- BroadcastExchange (86) + : +- * Filter (85) + : +- * HashAggregate (84) + : +- Exchange (83) + : +- * ColumnarToRow (82) + : +- CometHashAggregate (81) + : +- CometProject (80) + : +- CometBroadcastHashJoin (79) + : :- CometProject (77) + : : +- CometBroadcastHashJoin (76) + : : :- CometFilter (72) + : : : +- CometScan parquet spark_catalog.default.customer (71) + : : +- CometBroadcastExchange (75) + : : +- CometFilter (74) + : : +- CometScan parquet spark_catalog.default.web_sales (73) + : +- ReusedExchange (78) + +- BroadcastExchange (103) + +- * HashAggregate (102) + +- Exchange (101) + +- * ColumnarToRow (100) + +- CometHashAggregate (99) + +- CometProject (98) + +- CometBroadcastHashJoin (97) + :- CometProject (95) + : +- CometBroadcastHashJoin (94) + : :- CometFilter (90) + : : +- CometScan parquet spark_catalog.default.customer (89) + : +- CometBroadcastExchange (93) + : +- CometFilter (92) + : +- CometScan parquet spark_catalog.default.web_sales (91) + +- ReusedExchange (96) (1) Scan parquet spark_catalog.default.customer @@ -323,79 +315,67 @@ Arguments: [c_customer_sk#50], [cs_bill_customer_sk#58], Inner Input [14]: [c_customer_sk#50, c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57, cs_bill_customer_sk#58, cs_ext_discount_amt#59, cs_ext_sales_price#60, cs_ext_wholesale_cost#61, cs_ext_list_price#62, cs_sold_date_sk#63] Arguments: [c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57, cs_ext_discount_amt#59, cs_ext_sales_price#60, cs_ext_wholesale_cost#61, cs_ext_list_price#62, cs_sold_date_sk#63], [c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57, cs_ext_discount_amt#59, cs_ext_sales_price#60, cs_ext_wholesale_cost#61, cs_ext_list_price#62, cs_sold_date_sk#63] -(43) Scan parquet spark_catalog.default.date_dim +(43) ReusedExchange [Reuses operator id: 10] Output [2]: [d_date_sk#65, d_year#66] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct -(44) CometFilter -Input [2]: [d_date_sk#65, d_year#66] -Condition : ((isnotnull(d_year#66) AND (d_year#66 = 2001)) AND isnotnull(d_date_sk#65)) - -(45) CometBroadcastExchange -Input [2]: [d_date_sk#65, d_year#66] -Arguments: [d_date_sk#65, d_year#66] - -(46) CometBroadcastHashJoin +(44) CometBroadcastHashJoin Left output [12]: [c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57, cs_ext_discount_amt#59, cs_ext_sales_price#60, cs_ext_wholesale_cost#61, cs_ext_list_price#62, cs_sold_date_sk#63] Right output [2]: [d_date_sk#65, d_year#66] Arguments: [cs_sold_date_sk#63], [d_date_sk#65], Inner -(47) CometProject +(45) CometProject Input [14]: [c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57, cs_ext_discount_amt#59, cs_ext_sales_price#60, cs_ext_wholesale_cost#61, cs_ext_list_price#62, cs_sold_date_sk#63, d_date_sk#65, d_year#66] Arguments: [c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57, cs_ext_discount_amt#59, cs_ext_sales_price#60, cs_ext_wholesale_cost#61, cs_ext_list_price#62, d_year#66], [c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57, cs_ext_discount_amt#59, cs_ext_sales_price#60, cs_ext_wholesale_cost#61, cs_ext_list_price#62, d_year#66] -(48) CometHashAggregate +(46) CometHashAggregate Input [12]: [c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57, cs_ext_discount_amt#59, cs_ext_sales_price#60, cs_ext_wholesale_cost#61, cs_ext_list_price#62, d_year#66] Keys [8]: [c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57, d_year#66] Functions [1]: [partial_sum(((((cs_ext_list_price#62 - cs_ext_wholesale_cost#61) - cs_ext_discount_amt#59) + cs_ext_sales_price#60) / 2))] -(49) ColumnarToRow [codegen id : 4] +(47) ColumnarToRow [codegen id : 4] Input [10]: [c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57, d_year#66, sum#67, isEmpty#68] -(50) Exchange +(48) Exchange Input [10]: [c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57, d_year#66, sum#67, isEmpty#68] Arguments: hashpartitioning(c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57, d_year#66, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(51) HashAggregate [codegen id : 5] +(49) HashAggregate [codegen id : 5] Input [10]: [c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57, d_year#66, sum#67, isEmpty#68] Keys [8]: [c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57, d_year#66] Functions [1]: [sum(((((cs_ext_list_price#62 - cs_ext_wholesale_cost#61) - cs_ext_discount_amt#59) + cs_ext_sales_price#60) / 2))] Aggregate Attributes [1]: [sum(((((cs_ext_list_price#62 - cs_ext_wholesale_cost#61) - cs_ext_discount_amt#59) + cs_ext_sales_price#60) / 2))#69] Results [2]: [c_customer_id#51 AS customer_id#70, sum(((((cs_ext_list_price#62 - cs_ext_wholesale_cost#61) - cs_ext_discount_amt#59) + cs_ext_sales_price#60) / 2))#69 AS year_total#71] -(52) Filter [codegen id : 5] +(50) Filter [codegen id : 5] Input [2]: [customer_id#70, year_total#71] Condition : (isnotnull(year_total#71) AND (year_total#71 > 0.000000)) -(53) BroadcastExchange +(51) BroadcastExchange Input [2]: [customer_id#70, year_total#71] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(54) BroadcastHashJoin [codegen id : 12] +(52) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#21] Right keys [1]: [customer_id#70] Join type: Inner Join condition: None -(55) Project [codegen id : 12] +(53) Project [codegen id : 12] Output [11]: [customer_id#21, year_total#22, customer_id#42, customer_first_name#43, customer_last_name#44, customer_preferred_cust_flag#45, customer_birth_country#46, customer_login#47, customer_email_address#48, year_total#49, year_total#71] Input [12]: [customer_id#21, year_total#22, customer_id#42, customer_first_name#43, customer_last_name#44, customer_preferred_cust_flag#45, customer_birth_country#46, customer_login#47, customer_email_address#48, year_total#49, customer_id#70, year_total#71] -(56) Scan parquet spark_catalog.default.customer +(54) Scan parquet spark_catalog.default.customer Output [8]: [c_customer_sk#72, c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(57) CometFilter +(55) CometFilter Input [8]: [c_customer_sk#72, c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79] Condition : (isnotnull(c_customer_sk#72) AND isnotnull(c_customer_id#73)) -(58) Scan parquet spark_catalog.default.catalog_sales +(56) Scan parquet spark_catalog.default.catalog_sales Output [6]: [cs_bill_customer_sk#80, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, cs_sold_date_sk#85] Batched: true Location: InMemoryFileIndex [] @@ -403,92 +383,80 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#85), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(59) CometFilter +(57) CometFilter Input [6]: [cs_bill_customer_sk#80, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, cs_sold_date_sk#85] Condition : isnotnull(cs_bill_customer_sk#80) -(60) CometBroadcastExchange +(58) CometBroadcastExchange Input [6]: [cs_bill_customer_sk#80, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, cs_sold_date_sk#85] Arguments: [cs_bill_customer_sk#80, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, cs_sold_date_sk#85] -(61) CometBroadcastHashJoin +(59) CometBroadcastHashJoin Left output [8]: [c_customer_sk#72, c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79] Right output [6]: [cs_bill_customer_sk#80, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, cs_sold_date_sk#85] Arguments: [c_customer_sk#72], [cs_bill_customer_sk#80], Inner -(62) CometProject +(60) CometProject Input [14]: [c_customer_sk#72, c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, cs_bill_customer_sk#80, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, cs_sold_date_sk#85] Arguments: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, cs_sold_date_sk#85], [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, cs_sold_date_sk#85] -(63) Scan parquet spark_catalog.default.date_dim +(61) ReusedExchange [Reuses operator id: 27] Output [2]: [d_date_sk#87, d_year#88] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(64) CometFilter -Input [2]: [d_date_sk#87, d_year#88] -Condition : ((isnotnull(d_year#88) AND (d_year#88 = 2002)) AND isnotnull(d_date_sk#87)) -(65) CometBroadcastExchange -Input [2]: [d_date_sk#87, d_year#88] -Arguments: [d_date_sk#87, d_year#88] - -(66) CometBroadcastHashJoin +(62) CometBroadcastHashJoin Left output [12]: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, cs_sold_date_sk#85] Right output [2]: [d_date_sk#87, d_year#88] Arguments: [cs_sold_date_sk#85], [d_date_sk#87], Inner -(67) CometProject +(63) CometProject Input [14]: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, cs_sold_date_sk#85, d_date_sk#87, d_year#88] Arguments: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, d_year#88], [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, d_year#88] -(68) CometHashAggregate +(64) CometHashAggregate Input [12]: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, d_year#88] Keys [8]: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, d_year#88] Functions [1]: [partial_sum(((((cs_ext_list_price#84 - cs_ext_wholesale_cost#83) - cs_ext_discount_amt#81) + cs_ext_sales_price#82) / 2))] -(69) ColumnarToRow [codegen id : 6] +(65) ColumnarToRow [codegen id : 6] Input [10]: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, d_year#88, sum#89, isEmpty#90] -(70) Exchange +(66) Exchange Input [10]: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, d_year#88, sum#89, isEmpty#90] Arguments: hashpartitioning(c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, d_year#88, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(71) HashAggregate [codegen id : 7] +(67) HashAggregate [codegen id : 7] Input [10]: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, d_year#88, sum#89, isEmpty#90] Keys [8]: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, d_year#88] Functions [1]: [sum(((((cs_ext_list_price#84 - cs_ext_wholesale_cost#83) - cs_ext_discount_amt#81) + cs_ext_sales_price#82) / 2))] Aggregate Attributes [1]: [sum(((((cs_ext_list_price#84 - cs_ext_wholesale_cost#83) - cs_ext_discount_amt#81) + cs_ext_sales_price#82) / 2))#69] Results [2]: [c_customer_id#73 AS customer_id#91, sum(((((cs_ext_list_price#84 - cs_ext_wholesale_cost#83) - cs_ext_discount_amt#81) + cs_ext_sales_price#82) / 2))#69 AS year_total#92] -(72) BroadcastExchange +(68) BroadcastExchange Input [2]: [customer_id#91, year_total#92] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] -(73) BroadcastHashJoin [codegen id : 12] +(69) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#21] Right keys [1]: [customer_id#91] Join type: Inner Join condition: (CASE WHEN (year_total#71 > 0.000000) THEN (year_total#92 / year_total#71) END > CASE WHEN (year_total#22 > 0.000000) THEN (year_total#49 / year_total#22) END) -(74) Project [codegen id : 12] +(70) Project [codegen id : 12] Output [10]: [customer_id#21, customer_id#42, customer_first_name#43, customer_last_name#44, customer_preferred_cust_flag#45, customer_birth_country#46, customer_login#47, customer_email_address#48, year_total#71, year_total#92] Input [13]: [customer_id#21, year_total#22, customer_id#42, customer_first_name#43, customer_last_name#44, customer_preferred_cust_flag#45, customer_birth_country#46, customer_login#47, customer_email_address#48, year_total#49, year_total#71, customer_id#91, year_total#92] -(75) Scan parquet spark_catalog.default.customer +(71) Scan parquet spark_catalog.default.customer Output [8]: [c_customer_sk#93, c_customer_id#94, c_first_name#95, c_last_name#96, c_preferred_cust_flag#97, c_birth_country#98, c_login#99, c_email_address#100] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(76) CometFilter +(72) CometFilter Input [8]: [c_customer_sk#93, c_customer_id#94, c_first_name#95, c_last_name#96, c_preferred_cust_flag#97, c_birth_country#98, c_login#99, c_email_address#100] Condition : (isnotnull(c_customer_sk#93) AND isnotnull(c_customer_id#94)) -(77) Scan parquet spark_catalog.default.web_sales +(73) Scan parquet spark_catalog.default.web_sales Output [6]: [ws_bill_customer_sk#101, ws_ext_discount_amt#102, ws_ext_sales_price#103, ws_ext_wholesale_cost#104, ws_ext_list_price#105, ws_sold_date_sk#106] Batched: true Location: InMemoryFileIndex [] @@ -496,96 +464,84 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#106), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(78) CometFilter +(74) CometFilter Input [6]: [ws_bill_customer_sk#101, ws_ext_discount_amt#102, ws_ext_sales_price#103, ws_ext_wholesale_cost#104, ws_ext_list_price#105, ws_sold_date_sk#106] Condition : isnotnull(ws_bill_customer_sk#101) -(79) CometBroadcastExchange +(75) CometBroadcastExchange Input [6]: [ws_bill_customer_sk#101, ws_ext_discount_amt#102, ws_ext_sales_price#103, ws_ext_wholesale_cost#104, ws_ext_list_price#105, ws_sold_date_sk#106] Arguments: [ws_bill_customer_sk#101, ws_ext_discount_amt#102, ws_ext_sales_price#103, ws_ext_wholesale_cost#104, ws_ext_list_price#105, ws_sold_date_sk#106] -(80) CometBroadcastHashJoin +(76) CometBroadcastHashJoin Left output [8]: [c_customer_sk#93, c_customer_id#94, c_first_name#95, c_last_name#96, c_preferred_cust_flag#97, c_birth_country#98, c_login#99, c_email_address#100] Right output [6]: [ws_bill_customer_sk#101, ws_ext_discount_amt#102, ws_ext_sales_price#103, ws_ext_wholesale_cost#104, ws_ext_list_price#105, ws_sold_date_sk#106] Arguments: [c_customer_sk#93], [ws_bill_customer_sk#101], Inner -(81) CometProject +(77) CometProject Input [14]: [c_customer_sk#93, c_customer_id#94, c_first_name#95, c_last_name#96, c_preferred_cust_flag#97, c_birth_country#98, c_login#99, c_email_address#100, ws_bill_customer_sk#101, ws_ext_discount_amt#102, ws_ext_sales_price#103, ws_ext_wholesale_cost#104, ws_ext_list_price#105, ws_sold_date_sk#106] Arguments: [c_customer_id#94, c_first_name#95, c_last_name#96, c_preferred_cust_flag#97, c_birth_country#98, c_login#99, c_email_address#100, ws_ext_discount_amt#102, ws_ext_sales_price#103, ws_ext_wholesale_cost#104, ws_ext_list_price#105, ws_sold_date_sk#106], [c_customer_id#94, c_first_name#95, c_last_name#96, c_preferred_cust_flag#97, c_birth_country#98, c_login#99, c_email_address#100, ws_ext_discount_amt#102, ws_ext_sales_price#103, ws_ext_wholesale_cost#104, ws_ext_list_price#105, ws_sold_date_sk#106] -(82) Scan parquet spark_catalog.default.date_dim +(78) ReusedExchange [Reuses operator id: 10] Output [2]: [d_date_sk#108, d_year#109] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(83) CometFilter -Input [2]: [d_date_sk#108, d_year#109] -Condition : ((isnotnull(d_year#109) AND (d_year#109 = 2001)) AND isnotnull(d_date_sk#108)) - -(84) CometBroadcastExchange -Input [2]: [d_date_sk#108, d_year#109] -Arguments: [d_date_sk#108, d_year#109] -(85) CometBroadcastHashJoin +(79) CometBroadcastHashJoin Left output [12]: [c_customer_id#94, c_first_name#95, c_last_name#96, c_preferred_cust_flag#97, c_birth_country#98, c_login#99, c_email_address#100, ws_ext_discount_amt#102, ws_ext_sales_price#103, ws_ext_wholesale_cost#104, ws_ext_list_price#105, ws_sold_date_sk#106] Right output [2]: [d_date_sk#108, d_year#109] Arguments: [ws_sold_date_sk#106], [d_date_sk#108], Inner -(86) CometProject +(80) CometProject Input [14]: [c_customer_id#94, c_first_name#95, c_last_name#96, c_preferred_cust_flag#97, c_birth_country#98, c_login#99, c_email_address#100, ws_ext_discount_amt#102, ws_ext_sales_price#103, ws_ext_wholesale_cost#104, ws_ext_list_price#105, ws_sold_date_sk#106, d_date_sk#108, d_year#109] Arguments: [c_customer_id#94, c_first_name#95, c_last_name#96, c_preferred_cust_flag#97, c_birth_country#98, c_login#99, c_email_address#100, ws_ext_discount_amt#102, ws_ext_sales_price#103, ws_ext_wholesale_cost#104, ws_ext_list_price#105, d_year#109], [c_customer_id#94, c_first_name#95, c_last_name#96, c_preferred_cust_flag#97, c_birth_country#98, c_login#99, c_email_address#100, ws_ext_discount_amt#102, ws_ext_sales_price#103, ws_ext_wholesale_cost#104, ws_ext_list_price#105, d_year#109] -(87) CometHashAggregate +(81) CometHashAggregate Input [12]: [c_customer_id#94, c_first_name#95, c_last_name#96, c_preferred_cust_flag#97, c_birth_country#98, c_login#99, c_email_address#100, ws_ext_discount_amt#102, ws_ext_sales_price#103, ws_ext_wholesale_cost#104, ws_ext_list_price#105, d_year#109] Keys [8]: [c_customer_id#94, c_first_name#95, c_last_name#96, c_preferred_cust_flag#97, c_birth_country#98, c_login#99, c_email_address#100, d_year#109] Functions [1]: [partial_sum(((((ws_ext_list_price#105 - ws_ext_wholesale_cost#104) - ws_ext_discount_amt#102) + ws_ext_sales_price#103) / 2))] -(88) ColumnarToRow [codegen id : 8] +(82) ColumnarToRow [codegen id : 8] Input [10]: [c_customer_id#94, c_first_name#95, c_last_name#96, c_preferred_cust_flag#97, c_birth_country#98, c_login#99, c_email_address#100, d_year#109, sum#110, isEmpty#111] -(89) Exchange +(83) Exchange Input [10]: [c_customer_id#94, c_first_name#95, c_last_name#96, c_preferred_cust_flag#97, c_birth_country#98, c_login#99, c_email_address#100, d_year#109, sum#110, isEmpty#111] Arguments: hashpartitioning(c_customer_id#94, c_first_name#95, c_last_name#96, c_preferred_cust_flag#97, c_birth_country#98, c_login#99, c_email_address#100, d_year#109, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(90) HashAggregate [codegen id : 9] +(84) HashAggregate [codegen id : 9] Input [10]: [c_customer_id#94, c_first_name#95, c_last_name#96, c_preferred_cust_flag#97, c_birth_country#98, c_login#99, c_email_address#100, d_year#109, sum#110, isEmpty#111] Keys [8]: [c_customer_id#94, c_first_name#95, c_last_name#96, c_preferred_cust_flag#97, c_birth_country#98, c_login#99, c_email_address#100, d_year#109] Functions [1]: [sum(((((ws_ext_list_price#105 - ws_ext_wholesale_cost#104) - ws_ext_discount_amt#102) + ws_ext_sales_price#103) / 2))] Aggregate Attributes [1]: [sum(((((ws_ext_list_price#105 - ws_ext_wholesale_cost#104) - ws_ext_discount_amt#102) + ws_ext_sales_price#103) / 2))#112] Results [2]: [c_customer_id#94 AS customer_id#113, sum(((((ws_ext_list_price#105 - ws_ext_wholesale_cost#104) - ws_ext_discount_amt#102) + ws_ext_sales_price#103) / 2))#112 AS year_total#114] -(91) Filter [codegen id : 9] +(85) Filter [codegen id : 9] Input [2]: [customer_id#113, year_total#114] Condition : (isnotnull(year_total#114) AND (year_total#114 > 0.000000)) -(92) BroadcastExchange +(86) BroadcastExchange Input [2]: [customer_id#113, year_total#114] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=9] -(93) BroadcastHashJoin [codegen id : 12] +(87) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#21] Right keys [1]: [customer_id#113] Join type: Inner Join condition: None -(94) Project [codegen id : 12] +(88) Project [codegen id : 12] Output [11]: [customer_id#21, customer_id#42, customer_first_name#43, customer_last_name#44, customer_preferred_cust_flag#45, customer_birth_country#46, customer_login#47, customer_email_address#48, year_total#71, year_total#92, year_total#114] Input [12]: [customer_id#21, customer_id#42, customer_first_name#43, customer_last_name#44, customer_preferred_cust_flag#45, customer_birth_country#46, customer_login#47, customer_email_address#48, year_total#71, year_total#92, customer_id#113, year_total#114] -(95) Scan parquet spark_catalog.default.customer +(89) Scan parquet spark_catalog.default.customer Output [8]: [c_customer_sk#115, c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(96) CometFilter +(90) CometFilter Input [8]: [c_customer_sk#115, c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122] Condition : (isnotnull(c_customer_sk#115) AND isnotnull(c_customer_id#116)) -(97) Scan parquet spark_catalog.default.web_sales +(91) Scan parquet spark_catalog.default.web_sales Output [6]: [ws_bill_customer_sk#123, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, ws_sold_date_sk#128] Batched: true Location: InMemoryFileIndex [] @@ -593,142 +549,130 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#128), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(98) CometFilter +(92) CometFilter Input [6]: [ws_bill_customer_sk#123, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, ws_sold_date_sk#128] Condition : isnotnull(ws_bill_customer_sk#123) -(99) CometBroadcastExchange +(93) CometBroadcastExchange Input [6]: [ws_bill_customer_sk#123, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, ws_sold_date_sk#128] Arguments: [ws_bill_customer_sk#123, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, ws_sold_date_sk#128] -(100) CometBroadcastHashJoin +(94) CometBroadcastHashJoin Left output [8]: [c_customer_sk#115, c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122] Right output [6]: [ws_bill_customer_sk#123, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, ws_sold_date_sk#128] Arguments: [c_customer_sk#115], [ws_bill_customer_sk#123], Inner -(101) CometProject +(95) CometProject Input [14]: [c_customer_sk#115, c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, ws_bill_customer_sk#123, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, ws_sold_date_sk#128] Arguments: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, ws_sold_date_sk#128], [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, ws_sold_date_sk#128] -(102) Scan parquet spark_catalog.default.date_dim +(96) ReusedExchange [Reuses operator id: 27] 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,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(103) CometFilter -Input [2]: [d_date_sk#130, d_year#131] -Condition : ((isnotnull(d_year#131) AND (d_year#131 = 2002)) AND isnotnull(d_date_sk#130)) - -(104) CometBroadcastExchange -Input [2]: [d_date_sk#130, d_year#131] -Arguments: [d_date_sk#130, d_year#131] -(105) CometBroadcastHashJoin +(97) CometBroadcastHashJoin Left output [12]: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, ws_sold_date_sk#128] Right output [2]: [d_date_sk#130, d_year#131] Arguments: [ws_sold_date_sk#128], [d_date_sk#130], Inner -(106) CometProject +(98) CometProject Input [14]: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, ws_sold_date_sk#128, d_date_sk#130, d_year#131] Arguments: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, d_year#131], [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, d_year#131] -(107) CometHashAggregate +(99) CometHashAggregate Input [12]: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, d_year#131] Keys [8]: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, d_year#131] Functions [1]: [partial_sum(((((ws_ext_list_price#127 - ws_ext_wholesale_cost#126) - ws_ext_discount_amt#124) + ws_ext_sales_price#125) / 2))] -(108) ColumnarToRow [codegen id : 10] +(100) ColumnarToRow [codegen id : 10] Input [10]: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, d_year#131, sum#132, isEmpty#133] -(109) Exchange +(101) Exchange Input [10]: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, d_year#131, sum#132, isEmpty#133] Arguments: hashpartitioning(c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, d_year#131, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(110) HashAggregate [codegen id : 11] +(102) HashAggregate [codegen id : 11] Input [10]: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, d_year#131, sum#132, isEmpty#133] Keys [8]: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, d_year#131] Functions [1]: [sum(((((ws_ext_list_price#127 - ws_ext_wholesale_cost#126) - ws_ext_discount_amt#124) + ws_ext_sales_price#125) / 2))] Aggregate Attributes [1]: [sum(((((ws_ext_list_price#127 - ws_ext_wholesale_cost#126) - ws_ext_discount_amt#124) + ws_ext_sales_price#125) / 2))#112] Results [2]: [c_customer_id#116 AS customer_id#134, sum(((((ws_ext_list_price#127 - ws_ext_wholesale_cost#126) - ws_ext_discount_amt#124) + ws_ext_sales_price#125) / 2))#112 AS year_total#135] -(111) BroadcastExchange +(103) BroadcastExchange Input [2]: [customer_id#134, year_total#135] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(112) BroadcastHashJoin [codegen id : 12] +(104) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#21] Right keys [1]: [customer_id#134] Join type: Inner Join condition: (CASE WHEN (year_total#71 > 0.000000) THEN (year_total#92 / year_total#71) END > CASE WHEN (year_total#114 > 0.000000) THEN (year_total#135 / year_total#114) END) -(113) Project [codegen id : 12] +(105) Project [codegen id : 12] Output [7]: [customer_id#42, customer_first_name#43, customer_last_name#44, customer_preferred_cust_flag#45, customer_birth_country#46, customer_login#47, customer_email_address#48] Input [13]: [customer_id#21, customer_id#42, customer_first_name#43, customer_last_name#44, customer_preferred_cust_flag#45, customer_birth_country#46, customer_login#47, customer_email_address#48, year_total#71, year_total#92, year_total#114, customer_id#134, year_total#135] -(114) TakeOrderedAndProject +(106) TakeOrderedAndProject Input [7]: [customer_id#42, customer_first_name#43, customer_last_name#44, customer_preferred_cust_flag#45, customer_birth_country#46, customer_login#47, customer_email_address#48] Arguments: 100, [customer_id#42 ASC NULLS FIRST, customer_first_name#43 ASC NULLS FIRST, customer_last_name#44 ASC NULLS FIRST, customer_preferred_cust_flag#45 ASC NULLS FIRST, customer_birth_country#46 ASC NULLS FIRST, customer_login#47 ASC NULLS FIRST, customer_email_address#48 ASC NULLS FIRST], [customer_id#42, customer_first_name#43, customer_last_name#44, customer_preferred_cust_flag#45, customer_birth_country#46, customer_login#47, customer_email_address#48] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#15 -BroadcastExchange (118) -+- * ColumnarToRow (117) - +- CometFilter (116) - +- CometScan parquet spark_catalog.default.date_dim (115) +BroadcastExchange (110) ++- * ColumnarToRow (109) + +- CometFilter (108) + +- CometScan parquet spark_catalog.default.date_dim (107) -(115) Scan parquet spark_catalog.default.date_dim +(107) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#16, d_year#17] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(116) CometFilter +(108) CometFilter Input [2]: [d_date_sk#16, d_year#17] Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) -(117) ColumnarToRow [codegen id : 1] +(109) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#16, d_year#17] -(118) BroadcastExchange +(110) BroadcastExchange Input [2]: [d_date_sk#16, d_year#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#36 IN dynamicpruning#37 -BroadcastExchange (122) -+- * ColumnarToRow (121) - +- CometFilter (120) - +- CometScan parquet spark_catalog.default.date_dim (119) +BroadcastExchange (114) ++- * ColumnarToRow (113) + +- CometFilter (112) + +- CometScan parquet spark_catalog.default.date_dim (111) -(119) Scan parquet spark_catalog.default.date_dim +(111) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#38, d_year#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(120) CometFilter +(112) CometFilter Input [2]: [d_date_sk#38, d_year#39] Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) -(121) ColumnarToRow [codegen id : 1] +(113) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#38, d_year#39] -(122) BroadcastExchange +(114) BroadcastExchange Input [2]: [d_date_sk#38, d_year#39] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] Subquery:3 Hosting operator id = 38 Hosting Expression = cs_sold_date_sk#63 IN dynamicpruning#15 -Subquery:4 Hosting operator id = 58 Hosting Expression = cs_sold_date_sk#85 IN dynamicpruning#37 +Subquery:4 Hosting operator id = 56 Hosting Expression = cs_sold_date_sk#85 IN dynamicpruning#37 -Subquery:5 Hosting operator id = 77 Hosting Expression = ws_sold_date_sk#106 IN dynamicpruning#15 +Subquery:5 Hosting operator id = 73 Hosting Expression = ws_sold_date_sk#106 IN dynamicpruning#15 -Subquery:6 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#128 IN dynamicpruning#37 +Subquery:6 Hosting operator id = 91 Hosting Expression = ws_sold_date_sk#128 IN dynamicpruning#37 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt index 811a52663..048da153b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt @@ -86,15 +86,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom CometFilter [cs_bill_customer_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #13 - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk,d_year] #4 InputAdapter - BroadcastExchange #14 + BroadcastExchange #13 WholeStageCodegen (7) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #15 + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 WholeStageCodegen (6) ColumnarToRow InputAdapter @@ -105,20 +103,18 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom CometBroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #16 + CometBroadcastExchange #15 CometFilter [cs_bill_customer_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #17 - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk,d_year] #9 InputAdapter - BroadcastExchange #18 + BroadcastExchange #16 WholeStageCodegen (9) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #19 + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #17 WholeStageCodegen (8) ColumnarToRow InputAdapter @@ -129,19 +125,17 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #20 + CometBroadcastExchange #18 CometFilter [ws_bill_customer_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #21 - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk,d_year] #4 InputAdapter - BroadcastExchange #22 + BroadcastExchange #19 WholeStageCodegen (11) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #23 + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #20 WholeStageCodegen (10) ColumnarToRow InputAdapter @@ -152,10 +146,8 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #24 + CometBroadcastExchange #21 CometFilter [ws_bill_customer_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #25 - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt index cc3cc0287..1bcda9481 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (85) -+- * Project (84) - +- * BroadcastHashJoin Inner BuildRight (83) - :- * Project (56) - : +- * BroadcastHashJoin Inner BuildRight (55) +TakeOrderedAndProject (45) ++- * Project (44) + +- * BroadcastHashJoin Inner BuildRight (43) + :- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) : :- * Project (28) : : +- * Filter (27) : : +- Window (26) @@ -32,58 +32,18 @@ TakeOrderedAndProject (85) : : +- CometBroadcastExchange (15) : : +- CometFilter (14) : : +- CometScan parquet spark_catalog.default.store (13) - : +- BroadcastExchange (54) - : +- * Project (53) - : +- Window (52) - : +- * Sort (51) - : +- Exchange (50) - : +- * HashAggregate (49) - : +- Exchange (48) - : +- * ColumnarToRow (47) - : +- CometHashAggregate (46) - : +- CometProject (45) - : +- CometBroadcastHashJoin (44) - : :- CometProject (40) - : : +- CometBroadcastHashJoin (39) - : : :- CometProject (35) - : : : +- CometBroadcastHashJoin (34) - : : : :- CometFilter (30) - : : : : +- CometScan parquet spark_catalog.default.item (29) - : : : +- CometBroadcastExchange (33) - : : : +- CometFilter (32) - : : : +- CometScan parquet spark_catalog.default.store_sales (31) - : : +- CometBroadcastExchange (38) - : : +- CometFilter (37) - : : +- CometScan parquet spark_catalog.default.date_dim (36) - : +- CometBroadcastExchange (43) - : +- CometFilter (42) - : +- CometScan parquet spark_catalog.default.store (41) - +- BroadcastExchange (82) - +- * Project (81) - +- Window (80) - +- * Sort (79) - +- Exchange (78) - +- * HashAggregate (77) - +- Exchange (76) - +- * ColumnarToRow (75) - +- CometHashAggregate (74) - +- CometProject (73) - +- CometBroadcastHashJoin (72) - :- CometProject (68) - : +- CometBroadcastHashJoin (67) - : :- CometProject (63) - : : +- CometBroadcastHashJoin (62) - : : :- CometFilter (58) - : : : +- CometScan parquet spark_catalog.default.item (57) - : : +- CometBroadcastExchange (61) - : : +- CometFilter (60) - : : +- CometScan parquet spark_catalog.default.store_sales (59) - : +- CometBroadcastExchange (66) - : +- CometFilter (65) - : +- CometScan parquet spark_catalog.default.date_dim (64) - +- CometBroadcastExchange (71) - +- CometFilter (70) - +- CometScan parquet spark_catalog.default.store (69) + : +- BroadcastExchange (35) + : +- * Project (34) + : +- Window (33) + : +- * Sort (32) + : +- Exchange (31) + : +- * HashAggregate (30) + : +- ReusedExchange (29) + +- BroadcastExchange (42) + +- * Project (41) + +- Window (40) + +- * Sort (39) + +- ReusedExchange (38) (1) Scan parquet spark_catalog.default.item @@ -217,305 +177,104 @@ Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.0000 Output [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19] Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] -(29) Scan parquet spark_catalog.default.item -Output [3]: [i_item_sk#21, i_brand#22, i_category#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_category), IsNotNull(i_brand)] -ReadSchema: struct +(29) ReusedExchange [Reuses operator id: 20] +Output [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum#27] -(30) CometFilter -Input [3]: [i_item_sk#21, i_brand#22, i_category#23] -Condition : ((isnotnull(i_item_sk#21) AND isnotnull(i_category#23)) AND isnotnull(i_brand#22)) +(30) HashAggregate [codegen id : 6] +Input [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum#27] +Keys [6]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26] +Functions [1]: [sum(UnscaledValue(ss_sales_price#28))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#28))#16] +Results [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, MakeDecimal(sum(UnscaledValue(ss_sales_price#28))#16,17,2) AS sum_sales#17] -(31) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#24, ss_store_sk#25, ss_sales_price#26, ss_sold_date_sk#27] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#27), dynamicpruningexpression(ss_sold_date_sk#27 IN dynamicpruning#28)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct +(31) Exchange +Input [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17] +Arguments: hashpartitioning(i_category#21, i_brand#22, s_store_name#23, s_company_name#24, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(32) CometFilter -Input [4]: [ss_item_sk#24, ss_store_sk#25, ss_sales_price#26, ss_sold_date_sk#27] -Condition : (isnotnull(ss_item_sk#24) AND isnotnull(ss_store_sk#25)) +(32) Sort [codegen id : 7] +Input [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17] +Arguments: [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, s_store_name#23 ASC NULLS FIRST, s_company_name#24 ASC NULLS FIRST, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST], false, 0 -(33) CometBroadcastExchange -Input [4]: [ss_item_sk#24, ss_store_sk#25, ss_sales_price#26, ss_sold_date_sk#27] -Arguments: [ss_item_sk#24, ss_store_sk#25, ss_sales_price#26, ss_sold_date_sk#27] +(33) Window +Input [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17] +Arguments: [rank(d_year#25, d_moy#26) windowspecdefinition(i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#29], [i_category#21, i_brand#22, s_store_name#23, s_company_name#24], [d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] -(34) CometBroadcastHashJoin -Left output [3]: [i_item_sk#21, i_brand#22, i_category#23] -Right output [4]: [ss_item_sk#24, ss_store_sk#25, ss_sales_price#26, ss_sold_date_sk#27] -Arguments: [i_item_sk#21], [ss_item_sk#24], Inner +(34) Project [codegen id : 8] +Output [6]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, sum_sales#17 AS sum_sales#30, rn#29] +Input [8]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17, rn#29] -(35) CometProject -Input [7]: [i_item_sk#21, i_brand#22, i_category#23, ss_item_sk#24, ss_store_sk#25, ss_sales_price#26, ss_sold_date_sk#27] -Arguments: [i_brand#22, i_category#23, ss_store_sk#25, ss_sales_price#26, ss_sold_date_sk#27], [i_brand#22, i_category#23, ss_store_sk#25, ss_sales_price#26, ss_sold_date_sk#27] +(35) BroadcastExchange +Input [6]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, sum_sales#30, rn#29] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] -(36) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#29, d_year#30, d_moy#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(37) CometFilter -Input [3]: [d_date_sk#29, d_year#30, d_moy#31] -Condition : ((((d_year#30 = 1999) OR ((d_year#30 = 1998) AND (d_moy#31 = 12))) OR ((d_year#30 = 2000) AND (d_moy#31 = 1))) AND isnotnull(d_date_sk#29)) - -(38) CometBroadcastExchange -Input [3]: [d_date_sk#29, d_year#30, d_moy#31] -Arguments: [d_date_sk#29, d_year#30, d_moy#31] - -(39) CometBroadcastHashJoin -Left output [5]: [i_brand#22, i_category#23, ss_store_sk#25, ss_sales_price#26, ss_sold_date_sk#27] -Right output [3]: [d_date_sk#29, d_year#30, d_moy#31] -Arguments: [ss_sold_date_sk#27], [d_date_sk#29], Inner - -(40) CometProject -Input [8]: [i_brand#22, i_category#23, ss_store_sk#25, ss_sales_price#26, ss_sold_date_sk#27, d_date_sk#29, d_year#30, d_moy#31] -Arguments: [i_brand#22, i_category#23, ss_store_sk#25, ss_sales_price#26, d_year#30, d_moy#31], [i_brand#22, i_category#23, ss_store_sk#25, ss_sales_price#26, d_year#30, d_moy#31] - -(41) Scan parquet spark_catalog.default.store -Output [3]: [s_store_sk#32, s_store_name#33, s_company_name#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_company_name)] -ReadSchema: struct - -(42) CometFilter -Input [3]: [s_store_sk#32, s_store_name#33, s_company_name#34] -Condition : ((isnotnull(s_store_sk#32) AND isnotnull(s_store_name#33)) AND isnotnull(s_company_name#34)) - -(43) CometBroadcastExchange -Input [3]: [s_store_sk#32, s_store_name#33, s_company_name#34] -Arguments: [s_store_sk#32, s_store_name#33, s_company_name#34] - -(44) CometBroadcastHashJoin -Left output [6]: [i_brand#22, i_category#23, ss_store_sk#25, ss_sales_price#26, d_year#30, d_moy#31] -Right output [3]: [s_store_sk#32, s_store_name#33, s_company_name#34] -Arguments: [ss_store_sk#25], [s_store_sk#32], Inner - -(45) CometProject -Input [9]: [i_brand#22, i_category#23, ss_store_sk#25, ss_sales_price#26, d_year#30, d_moy#31, s_store_sk#32, s_store_name#33, s_company_name#34] -Arguments: [i_brand#22, i_category#23, ss_sales_price#26, d_year#30, d_moy#31, s_store_name#33, s_company_name#34], [i_brand#22, i_category#23, ss_sales_price#26, d_year#30, d_moy#31, s_store_name#33, s_company_name#34] - -(46) CometHashAggregate -Input [7]: [i_brand#22, i_category#23, ss_sales_price#26, d_year#30, d_moy#31, s_store_name#33, s_company_name#34] -Keys [6]: [i_category#23, i_brand#22, s_store_name#33, s_company_name#34, d_year#30, d_moy#31] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#26))] - -(47) ColumnarToRow [codegen id : 5] -Input [7]: [i_category#23, i_brand#22, s_store_name#33, s_company_name#34, d_year#30, d_moy#31, sum#35] - -(48) Exchange -Input [7]: [i_category#23, i_brand#22, s_store_name#33, s_company_name#34, d_year#30, d_moy#31, sum#35] -Arguments: hashpartitioning(i_category#23, i_brand#22, s_store_name#33, s_company_name#34, d_year#30, d_moy#31, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(49) HashAggregate [codegen id : 6] -Input [7]: [i_category#23, i_brand#22, s_store_name#33, s_company_name#34, d_year#30, d_moy#31, sum#35] -Keys [6]: [i_category#23, i_brand#22, s_store_name#33, s_company_name#34, d_year#30, d_moy#31] -Functions [1]: [sum(UnscaledValue(ss_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#26))#16] -Results [7]: [i_category#23, i_brand#22, s_store_name#33, s_company_name#34, d_year#30, d_moy#31, MakeDecimal(sum(UnscaledValue(ss_sales_price#26))#16,17,2) AS sum_sales#17] - -(50) Exchange -Input [7]: [i_category#23, i_brand#22, s_store_name#33, s_company_name#34, d_year#30, d_moy#31, sum_sales#17] -Arguments: hashpartitioning(i_category#23, i_brand#22, s_store_name#33, s_company_name#34, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(51) Sort [codegen id : 7] -Input [7]: [i_category#23, i_brand#22, s_store_name#33, s_company_name#34, d_year#30, d_moy#31, sum_sales#17] -Arguments: [i_category#23 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, s_store_name#33 ASC NULLS FIRST, s_company_name#34 ASC NULLS FIRST, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST], false, 0 - -(52) Window -Input [7]: [i_category#23, i_brand#22, s_store_name#33, s_company_name#34, d_year#30, d_moy#31, sum_sales#17] -Arguments: [rank(d_year#30, d_moy#31) windowspecdefinition(i_category#23, i_brand#22, s_store_name#33, s_company_name#34, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#23, i_brand#22, s_store_name#33, s_company_name#34], [d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] - -(53) Project [codegen id : 8] -Output [6]: [i_category#23, i_brand#22, s_store_name#33, s_company_name#34, sum_sales#17 AS sum_sales#37, rn#36] -Input [8]: [i_category#23, i_brand#22, s_store_name#33, s_company_name#34, d_year#30, d_moy#31, sum_sales#17, rn#36] - -(54) BroadcastExchange -Input [6]: [i_category#23, i_brand#22, s_store_name#33, s_company_name#34, sum_sales#37, rn#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=5] - -(55) BroadcastHashJoin [codegen id : 13] +(36) BroadcastHashJoin [codegen id : 13] Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#19] -Right keys [5]: [i_category#23, i_brand#22, s_store_name#33, s_company_name#34, (rn#36 + 1)] +Right keys [5]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, (rn#29 + 1)] Join type: Inner Join condition: None -(56) Project [codegen id : 13] -Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#37] -Input [15]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#23, i_brand#22, s_store_name#33, s_company_name#34, sum_sales#37, rn#36] - -(57) Scan parquet spark_catalog.default.item -Output [3]: [i_item_sk#38, i_brand#39, i_category#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_category), IsNotNull(i_brand)] -ReadSchema: struct - -(58) CometFilter -Input [3]: [i_item_sk#38, i_brand#39, i_category#40] -Condition : ((isnotnull(i_item_sk#38) AND isnotnull(i_category#40)) AND isnotnull(i_brand#39)) +(37) Project [codegen id : 13] +Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#30] +Input [15]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#21, i_brand#22, s_store_name#23, s_company_name#24, sum_sales#30, rn#29] -(59) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#41, ss_store_sk#42, ss_sales_price#43, ss_sold_date_sk#44] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#44), dynamicpruningexpression(ss_sold_date_sk#44 IN dynamicpruning#45)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct +(38) ReusedExchange [Reuses operator id: 31] +Output [7]: [i_category#31, i_brand#32, s_store_name#33, s_company_name#34, d_year#35, d_moy#36, sum_sales#17] -(60) CometFilter -Input [4]: [ss_item_sk#41, ss_store_sk#42, ss_sales_price#43, ss_sold_date_sk#44] -Condition : (isnotnull(ss_item_sk#41) AND isnotnull(ss_store_sk#42)) +(39) Sort [codegen id : 11] +Input [7]: [i_category#31, i_brand#32, s_store_name#33, s_company_name#34, d_year#35, d_moy#36, sum_sales#17] +Arguments: [i_category#31 ASC NULLS FIRST, i_brand#32 ASC NULLS FIRST, s_store_name#33 ASC NULLS FIRST, s_company_name#34 ASC NULLS FIRST, d_year#35 ASC NULLS FIRST, d_moy#36 ASC NULLS FIRST], false, 0 -(61) CometBroadcastExchange -Input [4]: [ss_item_sk#41, ss_store_sk#42, ss_sales_price#43, ss_sold_date_sk#44] -Arguments: [ss_item_sk#41, ss_store_sk#42, ss_sales_price#43, ss_sold_date_sk#44] +(40) Window +Input [7]: [i_category#31, i_brand#32, s_store_name#33, s_company_name#34, d_year#35, d_moy#36, sum_sales#17] +Arguments: [rank(d_year#35, d_moy#36) windowspecdefinition(i_category#31, i_brand#32, s_store_name#33, s_company_name#34, d_year#35 ASC NULLS FIRST, d_moy#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#31, i_brand#32, s_store_name#33, s_company_name#34], [d_year#35 ASC NULLS FIRST, d_moy#36 ASC NULLS FIRST] -(62) CometBroadcastHashJoin -Left output [3]: [i_item_sk#38, i_brand#39, i_category#40] -Right output [4]: [ss_item_sk#41, ss_store_sk#42, ss_sales_price#43, ss_sold_date_sk#44] -Arguments: [i_item_sk#38], [ss_item_sk#41], Inner +(41) Project [codegen id : 12] +Output [6]: [i_category#31, i_brand#32, s_store_name#33, s_company_name#34, sum_sales#17 AS sum_sales#38, rn#37] +Input [8]: [i_category#31, i_brand#32, s_store_name#33, s_company_name#34, d_year#35, d_moy#36, sum_sales#17, rn#37] -(63) CometProject -Input [7]: [i_item_sk#38, i_brand#39, i_category#40, ss_item_sk#41, ss_store_sk#42, ss_sales_price#43, ss_sold_date_sk#44] -Arguments: [i_brand#39, i_category#40, ss_store_sk#42, ss_sales_price#43, ss_sold_date_sk#44], [i_brand#39, i_category#40, ss_store_sk#42, ss_sales_price#43, ss_sold_date_sk#44] +(42) BroadcastExchange +Input [6]: [i_category#31, i_brand#32, s_store_name#33, s_company_name#34, sum_sales#38, rn#37] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] -(64) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#46, d_year#47, d_moy#48] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(65) CometFilter -Input [3]: [d_date_sk#46, d_year#47, d_moy#48] -Condition : ((((d_year#47 = 1999) OR ((d_year#47 = 1998) AND (d_moy#48 = 12))) OR ((d_year#47 = 2000) AND (d_moy#48 = 1))) AND isnotnull(d_date_sk#46)) - -(66) CometBroadcastExchange -Input [3]: [d_date_sk#46, d_year#47, d_moy#48] -Arguments: [d_date_sk#46, d_year#47, d_moy#48] - -(67) CometBroadcastHashJoin -Left output [5]: [i_brand#39, i_category#40, ss_store_sk#42, ss_sales_price#43, ss_sold_date_sk#44] -Right output [3]: [d_date_sk#46, d_year#47, d_moy#48] -Arguments: [ss_sold_date_sk#44], [d_date_sk#46], Inner - -(68) CometProject -Input [8]: [i_brand#39, i_category#40, ss_store_sk#42, ss_sales_price#43, ss_sold_date_sk#44, d_date_sk#46, d_year#47, d_moy#48] -Arguments: [i_brand#39, i_category#40, ss_store_sk#42, ss_sales_price#43, d_year#47, d_moy#48], [i_brand#39, i_category#40, ss_store_sk#42, ss_sales_price#43, d_year#47, d_moy#48] - -(69) Scan parquet spark_catalog.default.store -Output [3]: [s_store_sk#49, s_store_name#50, s_company_name#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_company_name)] -ReadSchema: struct - -(70) CometFilter -Input [3]: [s_store_sk#49, s_store_name#50, s_company_name#51] -Condition : ((isnotnull(s_store_sk#49) AND isnotnull(s_store_name#50)) AND isnotnull(s_company_name#51)) - -(71) CometBroadcastExchange -Input [3]: [s_store_sk#49, s_store_name#50, s_company_name#51] -Arguments: [s_store_sk#49, s_store_name#50, s_company_name#51] - -(72) CometBroadcastHashJoin -Left output [6]: [i_brand#39, i_category#40, ss_store_sk#42, ss_sales_price#43, d_year#47, d_moy#48] -Right output [3]: [s_store_sk#49, s_store_name#50, s_company_name#51] -Arguments: [ss_store_sk#42], [s_store_sk#49], Inner - -(73) CometProject -Input [9]: [i_brand#39, i_category#40, ss_store_sk#42, ss_sales_price#43, d_year#47, d_moy#48, s_store_sk#49, s_store_name#50, s_company_name#51] -Arguments: [i_brand#39, i_category#40, ss_sales_price#43, d_year#47, d_moy#48, s_store_name#50, s_company_name#51], [i_brand#39, i_category#40, ss_sales_price#43, d_year#47, d_moy#48, s_store_name#50, s_company_name#51] - -(74) CometHashAggregate -Input [7]: [i_brand#39, i_category#40, ss_sales_price#43, d_year#47, d_moy#48, s_store_name#50, s_company_name#51] -Keys [6]: [i_category#40, i_brand#39, s_store_name#50, s_company_name#51, d_year#47, d_moy#48] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#43))] - -(75) ColumnarToRow [codegen id : 9] -Input [7]: [i_category#40, i_brand#39, s_store_name#50, s_company_name#51, d_year#47, d_moy#48, sum#52] - -(76) Exchange -Input [7]: [i_category#40, i_brand#39, s_store_name#50, s_company_name#51, d_year#47, d_moy#48, sum#52] -Arguments: hashpartitioning(i_category#40, i_brand#39, s_store_name#50, s_company_name#51, d_year#47, d_moy#48, 5), ENSURE_REQUIREMENTS, [plan_id=6] - -(77) HashAggregate [codegen id : 10] -Input [7]: [i_category#40, i_brand#39, s_store_name#50, s_company_name#51, d_year#47, d_moy#48, sum#52] -Keys [6]: [i_category#40, i_brand#39, s_store_name#50, s_company_name#51, d_year#47, d_moy#48] -Functions [1]: [sum(UnscaledValue(ss_sales_price#43))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#43))#16] -Results [7]: [i_category#40, i_brand#39, s_store_name#50, s_company_name#51, d_year#47, d_moy#48, MakeDecimal(sum(UnscaledValue(ss_sales_price#43))#16,17,2) AS sum_sales#17] - -(78) Exchange -Input [7]: [i_category#40, i_brand#39, s_store_name#50, s_company_name#51, d_year#47, d_moy#48, sum_sales#17] -Arguments: hashpartitioning(i_category#40, i_brand#39, s_store_name#50, s_company_name#51, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(79) Sort [codegen id : 11] -Input [7]: [i_category#40, i_brand#39, s_store_name#50, s_company_name#51, d_year#47, d_moy#48, sum_sales#17] -Arguments: [i_category#40 ASC NULLS FIRST, i_brand#39 ASC NULLS FIRST, s_store_name#50 ASC NULLS FIRST, s_company_name#51 ASC NULLS FIRST, d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST], false, 0 - -(80) Window -Input [7]: [i_category#40, i_brand#39, s_store_name#50, s_company_name#51, d_year#47, d_moy#48, sum_sales#17] -Arguments: [rank(d_year#47, d_moy#48) windowspecdefinition(i_category#40, i_brand#39, s_store_name#50, s_company_name#51, d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#53], [i_category#40, i_brand#39, s_store_name#50, s_company_name#51], [d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST] - -(81) Project [codegen id : 12] -Output [6]: [i_category#40, i_brand#39, s_store_name#50, s_company_name#51, sum_sales#17 AS sum_sales#54, rn#53] -Input [8]: [i_category#40, i_brand#39, s_store_name#50, s_company_name#51, d_year#47, d_moy#48, sum_sales#17, rn#53] - -(82) BroadcastExchange -Input [6]: [i_category#40, i_brand#39, s_store_name#50, s_company_name#51, sum_sales#54, rn#53] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=8] - -(83) BroadcastHashJoin [codegen id : 13] +(43) BroadcastHashJoin [codegen id : 13] Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#19] -Right keys [5]: [i_category#40, i_brand#39, s_store_name#50, s_company_name#51, (rn#53 - 1)] +Right keys [5]: [i_category#31, i_brand#32, s_store_name#33, s_company_name#34, (rn#37 - 1)] Join type: Inner Join condition: None -(84) Project [codegen id : 13] -Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, sum_sales#37 AS psum#55, sum_sales#54 AS nsum#56] -Input [16]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#37, i_category#40, i_brand#39, s_store_name#50, s_company_name#51, sum_sales#54, rn#53] +(44) Project [codegen id : 13] +Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, sum_sales#30 AS psum#39, sum_sales#38 AS nsum#40] +Input [16]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#30, i_category#31, i_brand#32, s_store_name#33, s_company_name#34, sum_sales#38, rn#37] -(85) TakeOrderedAndProject -Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#55, nsum#56] -Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#55, nsum#56] +(45) TakeOrderedAndProject +Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#39, nsum#40] +Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#39, nsum#40] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (89) -+- * ColumnarToRow (88) - +- CometFilter (87) - +- CometScan parquet spark_catalog.default.date_dim (86) +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) -(86) Scan parquet spark_catalog.default.date_dim +(46) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#9, d_year#10, d_moy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(87) CometFilter +(47) CometFilter Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) -(88) ColumnarToRow [codegen id : 1] +(48) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -(89) BroadcastExchange +(49) BroadcastExchange Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#27 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 59 Hosting Expression = ss_sold_date_sk#44 IN dynamicpruning#8 +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/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt index 9ba84b970..d2e615f39 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt @@ -61,31 +61,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra WholeStageCodegen (6) HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #9 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometFilter [i_item_sk,i_category,i_brand] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #10 - CometFilter [ss_item_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #11 - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #12 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 InputAdapter - BroadcastExchange #13 + BroadcastExchange #9 WholeStageCodegen (12) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter @@ -93,30 +71,4 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra WholeStageCodegen (11) Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name] #14 - WholeStageCodegen (10) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] - InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #15 - WholeStageCodegen (9) - ColumnarToRow - InputAdapter - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometFilter [i_item_sk,i_category,i_brand] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #16 - CometFilter [ss_item_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #17 - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #18 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt index 5d7151b74..2445a3ee6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (83) -+- * HashAggregate (82) - +- Exchange (81) - +- * HashAggregate (80) - +- Union (79) +TakeOrderedAndProject (77) ++- * HashAggregate (76) + +- Exchange (75) + +- * HashAggregate (74) + +- Union (73) :- * Project (26) : +- * Filter (25) : +- Window (24) @@ -30,19 +30,19 @@ TakeOrderedAndProject (83) : +- CometProject (12) : +- CometFilter (11) : +- CometScan parquet spark_catalog.default.date_dim (10) - :- * Project (52) - : +- * Filter (51) - : +- Window (50) - : +- * Sort (49) - : +- Window (48) - : +- * Sort (47) - : +- Exchange (46) - : +- * HashAggregate (45) - : +- Exchange (44) - : +- * ColumnarToRow (43) - : +- CometHashAggregate (42) - : +- CometProject (41) - : +- CometBroadcastHashJoin (40) + :- * Project (49) + : +- * Filter (48) + : +- Window (47) + : +- * Sort (46) + : +- Window (45) + : +- * Sort (44) + : +- Exchange (43) + : +- * HashAggregate (42) + : +- Exchange (41) + : +- * ColumnarToRow (40) + : +- CometHashAggregate (39) + : +- CometProject (38) + : +- CometBroadcastHashJoin (37) : :- CometProject (35) : : +- CometBroadcastHashJoin (34) : : :- CometBroadcastExchange (30) @@ -52,36 +52,30 @@ TakeOrderedAndProject (83) : : +- CometProject (33) : : +- CometFilter (32) : : +- CometScan parquet spark_catalog.default.catalog_returns (31) - : +- CometBroadcastExchange (39) - : +- CometProject (38) - : +- CometFilter (37) - : +- CometScan parquet spark_catalog.default.date_dim (36) - +- * Project (78) - +- * Filter (77) - +- Window (76) - +- * Sort (75) - +- Window (74) - +- * Sort (73) - +- Exchange (72) - +- * HashAggregate (71) - +- Exchange (70) - +- * ColumnarToRow (69) - +- CometHashAggregate (68) - +- CometProject (67) - +- CometBroadcastHashJoin (66) - :- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometBroadcastExchange (56) - : : +- CometProject (55) - : : +- CometFilter (54) - : : +- CometScan parquet spark_catalog.default.store_sales (53) - : +- CometProject (59) - : +- CometFilter (58) - : +- CometScan parquet spark_catalog.default.store_returns (57) - +- CometBroadcastExchange (65) - +- CometProject (64) - +- CometFilter (63) - +- CometScan parquet spark_catalog.default.date_dim (62) + : +- ReusedExchange (36) + +- * Project (72) + +- * Filter (71) + +- Window (70) + +- * Sort (69) + +- Window (68) + +- * Sort (67) + +- Exchange (66) + +- * HashAggregate (65) + +- Exchange (64) + +- * ColumnarToRow (63) + +- CometHashAggregate (62) + +- CometProject (61) + +- CometBroadcastHashJoin (60) + :- CometProject (58) + : +- CometBroadcastHashJoin (57) + : :- CometBroadcastExchange (53) + : : +- CometProject (52) + : : +- CometFilter (51) + : : +- CometScan parquet spark_catalog.default.store_sales (50) + : +- CometProject (56) + : +- CometFilter (55) + : +- CometScan parquet spark_catalog.default.store_returns (54) + +- ReusedExchange (59) (1) Scan parquet spark_catalog.default.web_sales @@ -247,258 +241,226 @@ Arguments: [cs_order_number#33, cs_item_sk#32], [cr_order_number#40, cr_item_sk# Input [9]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42] Arguments: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_return_quantity#41, cr_return_amount#42], [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_return_quantity#41, cr_return_amount#42] -(36) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#44, d_year#45, d_moy#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(37) CometFilter -Input [3]: [d_date_sk#44, d_year#45, d_moy#46] -Condition : ((((isnotnull(d_year#45) AND isnotnull(d_moy#46)) AND (d_year#45 = 2001)) AND (d_moy#46 = 12)) AND isnotnull(d_date_sk#44)) - -(38) CometProject -Input [3]: [d_date_sk#44, d_year#45, d_moy#46] -Arguments: [d_date_sk#44], [d_date_sk#44] +(36) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_date_sk#44] -(39) CometBroadcastExchange -Input [1]: [d_date_sk#44] -Arguments: [d_date_sk#44] - -(40) CometBroadcastHashJoin +(37) CometBroadcastHashJoin Left output [6]: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_return_quantity#41, cr_return_amount#42] Right output [1]: [d_date_sk#44] Arguments: [cs_sold_date_sk#37], [d_date_sk#44], Inner -(41) CometProject +(38) CometProject Input [7]: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_return_quantity#41, cr_return_amount#42, d_date_sk#44] Arguments: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cr_return_quantity#41, cr_return_amount#42], [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cr_return_quantity#41, cr_return_amount#42] -(42) CometHashAggregate +(39) CometHashAggregate Input [5]: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cr_return_quantity#41, cr_return_amount#42] Keys [1]: [cs_item_sk#32] Functions [4]: [partial_sum(coalesce(cr_return_quantity#41, 0)), partial_sum(coalesce(cs_quantity#34, 0)), partial_sum(coalesce(cast(cr_return_amount#42 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#35 as decimal(12,2)), 0.00))] -(43) ColumnarToRow [codegen id : 6] -Input [7]: [cs_item_sk#32, sum#47, sum#48, sum#49, isEmpty#50, sum#51, isEmpty#52] +(40) ColumnarToRow [codegen id : 6] +Input [7]: [cs_item_sk#32, sum#45, sum#46, sum#47, isEmpty#48, sum#49, isEmpty#50] -(44) Exchange -Input [7]: [cs_item_sk#32, sum#47, sum#48, sum#49, isEmpty#50, sum#51, isEmpty#52] +(41) Exchange +Input [7]: [cs_item_sk#32, sum#45, sum#46, sum#47, isEmpty#48, sum#49, isEmpty#50] Arguments: hashpartitioning(cs_item_sk#32, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(45) HashAggregate [codegen id : 7] -Input [7]: [cs_item_sk#32, sum#47, sum#48, sum#49, isEmpty#50, sum#51, isEmpty#52] +(42) HashAggregate [codegen id : 7] +Input [7]: [cs_item_sk#32, sum#45, sum#46, sum#47, isEmpty#48, sum#49, isEmpty#50] Keys [1]: [cs_item_sk#32] Functions [4]: [sum(coalesce(cr_return_quantity#41, 0)), sum(coalesce(cs_quantity#34, 0)), sum(coalesce(cast(cr_return_amount#42 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#35 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#41, 0))#53, sum(coalesce(cs_quantity#34, 0))#54, sum(coalesce(cast(cr_return_amount#42 as decimal(12,2)), 0.00))#55, sum(coalesce(cast(cs_net_paid#35 as decimal(12,2)), 0.00))#56] -Results [3]: [cs_item_sk#32 AS item#57, (cast(sum(coalesce(cr_return_quantity#41, 0))#53 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#34, 0))#54 as decimal(15,4))) AS return_ratio#58, (cast(sum(coalesce(cast(cr_return_amount#42 as decimal(12,2)), 0.00))#55 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#35 as decimal(12,2)), 0.00))#56 as decimal(15,4))) AS currency_ratio#59] +Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#41, 0))#51, sum(coalesce(cs_quantity#34, 0))#52, sum(coalesce(cast(cr_return_amount#42 as decimal(12,2)), 0.00))#53, sum(coalesce(cast(cs_net_paid#35 as decimal(12,2)), 0.00))#54] +Results [3]: [cs_item_sk#32 AS item#55, (cast(sum(coalesce(cr_return_quantity#41, 0))#51 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#34, 0))#52 as decimal(15,4))) AS return_ratio#56, (cast(sum(coalesce(cast(cr_return_amount#42 as decimal(12,2)), 0.00))#53 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#35 as decimal(12,2)), 0.00))#54 as decimal(15,4))) AS currency_ratio#57] -(46) Exchange -Input [3]: [item#57, return_ratio#58, currency_ratio#59] +(43) Exchange +Input [3]: [item#55, return_ratio#56, currency_ratio#57] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] -(47) Sort [codegen id : 8] -Input [3]: [item#57, return_ratio#58, currency_ratio#59] -Arguments: [return_ratio#58 ASC NULLS FIRST], false, 0 +(44) Sort [codegen id : 8] +Input [3]: [item#55, return_ratio#56, currency_ratio#57] +Arguments: [return_ratio#56 ASC NULLS FIRST], false, 0 -(48) Window -Input [3]: [item#57, return_ratio#58, currency_ratio#59] -Arguments: [rank(return_ratio#58) windowspecdefinition(return_ratio#58 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#60], [return_ratio#58 ASC NULLS FIRST] +(45) Window +Input [3]: [item#55, return_ratio#56, currency_ratio#57] +Arguments: [rank(return_ratio#56) windowspecdefinition(return_ratio#56 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#58], [return_ratio#56 ASC NULLS FIRST] -(49) Sort [codegen id : 9] -Input [4]: [item#57, return_ratio#58, currency_ratio#59, return_rank#60] -Arguments: [currency_ratio#59 ASC NULLS FIRST], false, 0 +(46) Sort [codegen id : 9] +Input [4]: [item#55, return_ratio#56, currency_ratio#57, return_rank#58] +Arguments: [currency_ratio#57 ASC NULLS FIRST], false, 0 -(50) Window -Input [4]: [item#57, return_ratio#58, currency_ratio#59, return_rank#60] -Arguments: [rank(currency_ratio#59) windowspecdefinition(currency_ratio#59 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#61], [currency_ratio#59 ASC NULLS FIRST] +(47) Window +Input [4]: [item#55, return_ratio#56, currency_ratio#57, return_rank#58] +Arguments: [rank(currency_ratio#57) windowspecdefinition(currency_ratio#57 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#59], [currency_ratio#57 ASC NULLS FIRST] -(51) Filter [codegen id : 10] -Input [5]: [item#57, return_ratio#58, currency_ratio#59, return_rank#60, currency_rank#61] -Condition : ((return_rank#60 <= 10) OR (currency_rank#61 <= 10)) +(48) Filter [codegen id : 10] +Input [5]: [item#55, return_ratio#56, currency_ratio#57, return_rank#58, currency_rank#59] +Condition : ((return_rank#58 <= 10) OR (currency_rank#59 <= 10)) -(52) Project [codegen id : 10] -Output [5]: [catalog AS channel#62, item#57, return_ratio#58, return_rank#60, currency_rank#61] -Input [5]: [item#57, return_ratio#58, currency_ratio#59, return_rank#60, currency_rank#61] +(49) Project [codegen id : 10] +Output [5]: [catalog AS channel#60, item#55, return_ratio#56, return_rank#58, currency_rank#59] +Input [5]: [item#55, return_ratio#56, currency_ratio#57, return_rank#58, currency_rank#59] -(53) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_net_profit#67, ss_sold_date_sk#68] +(50) Scan parquet spark_catalog.default.store_sales +Output [6]: [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_net_profit#65, ss_sold_date_sk#66] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#68), dynamicpruningexpression(ss_sold_date_sk#68 IN dynamicpruning#69)] +PartitionFilters: [isnotnull(ss_sold_date_sk#66), dynamicpruningexpression(ss_sold_date_sk#66 IN dynamicpruning#67)] PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(54) CometFilter -Input [6]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_net_profit#67, ss_sold_date_sk#68] -Condition : (((((((isnotnull(ss_net_profit#67) AND isnotnull(ss_net_paid#66)) AND isnotnull(ss_quantity#65)) AND (ss_net_profit#67 > 1.00)) AND (ss_net_paid#66 > 0.00)) AND (ss_quantity#65 > 0)) AND isnotnull(ss_ticket_number#64)) AND isnotnull(ss_item_sk#63)) +(51) CometFilter +Input [6]: [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_net_profit#65, ss_sold_date_sk#66] +Condition : (((((((isnotnull(ss_net_profit#65) AND isnotnull(ss_net_paid#64)) AND isnotnull(ss_quantity#63)) AND (ss_net_profit#65 > 1.00)) AND (ss_net_paid#64 > 0.00)) AND (ss_quantity#63 > 0)) AND isnotnull(ss_ticket_number#62)) AND isnotnull(ss_item_sk#61)) -(55) CometProject -Input [6]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_net_profit#67, ss_sold_date_sk#68] -Arguments: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68], [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68] +(52) CometProject +Input [6]: [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_net_profit#65, ss_sold_date_sk#66] +Arguments: [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66], [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66] -(56) CometBroadcastExchange -Input [5]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68] -Arguments: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68] +(53) CometBroadcastExchange +Input [5]: [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66] +Arguments: [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66] -(57) Scan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73, sr_returned_date_sk#74] +(54) Scan parquet spark_catalog.default.store_returns +Output [5]: [sr_item_sk#68, sr_ticket_number#69, sr_return_quantity#70, sr_return_amt#71, sr_returned_date_sk#72] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(58) CometFilter -Input [5]: [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73, sr_returned_date_sk#74] -Condition : (((isnotnull(sr_return_amt#73) AND (sr_return_amt#73 > 10000.00)) AND isnotnull(sr_ticket_number#71)) AND isnotnull(sr_item_sk#70)) +(55) CometFilter +Input [5]: [sr_item_sk#68, sr_ticket_number#69, sr_return_quantity#70, sr_return_amt#71, sr_returned_date_sk#72] +Condition : (((isnotnull(sr_return_amt#71) AND (sr_return_amt#71 > 10000.00)) AND isnotnull(sr_ticket_number#69)) AND isnotnull(sr_item_sk#68)) -(59) CometProject -Input [5]: [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73, sr_returned_date_sk#74] -Arguments: [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73], [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73] +(56) CometProject +Input [5]: [sr_item_sk#68, sr_ticket_number#69, sr_return_quantity#70, sr_return_amt#71, sr_returned_date_sk#72] +Arguments: [sr_item_sk#68, sr_ticket_number#69, sr_return_quantity#70, sr_return_amt#71], [sr_item_sk#68, sr_ticket_number#69, sr_return_quantity#70, sr_return_amt#71] -(60) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68] -Right output [4]: [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73] -Arguments: [ss_ticket_number#64, ss_item_sk#63], [sr_ticket_number#71, sr_item_sk#70], Inner +(57) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66] +Right output [4]: [sr_item_sk#68, sr_ticket_number#69, sr_return_quantity#70, sr_return_amt#71] +Arguments: [ss_ticket_number#62, ss_item_sk#61], [sr_ticket_number#69, sr_item_sk#68], Inner -(61) CometProject -Input [9]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68, sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73] -Arguments: [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68, sr_return_quantity#72, sr_return_amt#73], [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68, sr_return_quantity#72, sr_return_amt#73] +(58) CometProject +Input [9]: [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66, sr_item_sk#68, sr_ticket_number#69, sr_return_quantity#70, sr_return_amt#71] +Arguments: [ss_item_sk#61, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66, sr_return_quantity#70, sr_return_amt#71], [ss_item_sk#61, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66, sr_return_quantity#70, sr_return_amt#71] -(62) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#75, d_year#76, d_moy#77] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(63) CometFilter -Input [3]: [d_date_sk#75, d_year#76, d_moy#77] -Condition : ((((isnotnull(d_year#76) AND isnotnull(d_moy#77)) AND (d_year#76 = 2001)) AND (d_moy#77 = 12)) AND isnotnull(d_date_sk#75)) - -(64) CometProject -Input [3]: [d_date_sk#75, d_year#76, d_moy#77] -Arguments: [d_date_sk#75], [d_date_sk#75] - -(65) CometBroadcastExchange -Input [1]: [d_date_sk#75] -Arguments: [d_date_sk#75] - -(66) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68, sr_return_quantity#72, sr_return_amt#73] -Right output [1]: [d_date_sk#75] -Arguments: [ss_sold_date_sk#68], [d_date_sk#75], Inner +(59) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_date_sk#73] -(67) CometProject -Input [7]: [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68, sr_return_quantity#72, sr_return_amt#73, d_date_sk#75] -Arguments: [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, sr_return_quantity#72, sr_return_amt#73], [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, sr_return_quantity#72, sr_return_amt#73] - -(68) CometHashAggregate -Input [5]: [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, sr_return_quantity#72, sr_return_amt#73] -Keys [1]: [ss_item_sk#63] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#72, 0)), partial_sum(coalesce(ss_quantity#65, 0)), partial_sum(coalesce(cast(sr_return_amt#73 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#66 as decimal(12,2)), 0.00))] - -(69) ColumnarToRow [codegen id : 11] -Input [7]: [ss_item_sk#63, sum#78, sum#79, sum#80, isEmpty#81, sum#82, isEmpty#83] - -(70) Exchange -Input [7]: [ss_item_sk#63, sum#78, sum#79, sum#80, isEmpty#81, sum#82, isEmpty#83] -Arguments: hashpartitioning(ss_item_sk#63, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(71) HashAggregate [codegen id : 12] -Input [7]: [ss_item_sk#63, sum#78, sum#79, sum#80, isEmpty#81, sum#82, isEmpty#83] -Keys [1]: [ss_item_sk#63] -Functions [4]: [sum(coalesce(sr_return_quantity#72, 0)), sum(coalesce(ss_quantity#65, 0)), sum(coalesce(cast(sr_return_amt#73 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#66 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#72, 0))#84, sum(coalesce(ss_quantity#65, 0))#85, sum(coalesce(cast(sr_return_amt#73 as decimal(12,2)), 0.00))#86, sum(coalesce(cast(ss_net_paid#66 as decimal(12,2)), 0.00))#87] -Results [3]: [ss_item_sk#63 AS item#88, (cast(sum(coalesce(sr_return_quantity#72, 0))#84 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#65, 0))#85 as decimal(15,4))) AS return_ratio#89, (cast(sum(coalesce(cast(sr_return_amt#73 as decimal(12,2)), 0.00))#86 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#66 as decimal(12,2)), 0.00))#87 as decimal(15,4))) AS currency_ratio#90] +(60) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#61, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66, sr_return_quantity#70, sr_return_amt#71] +Right output [1]: [d_date_sk#73] +Arguments: [ss_sold_date_sk#66], [d_date_sk#73], Inner -(72) Exchange -Input [3]: [item#88, return_ratio#89, currency_ratio#90] +(61) CometProject +Input [7]: [ss_item_sk#61, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66, sr_return_quantity#70, sr_return_amt#71, d_date_sk#73] +Arguments: [ss_item_sk#61, ss_quantity#63, ss_net_paid#64, sr_return_quantity#70, sr_return_amt#71], [ss_item_sk#61, ss_quantity#63, ss_net_paid#64, sr_return_quantity#70, sr_return_amt#71] + +(62) CometHashAggregate +Input [5]: [ss_item_sk#61, ss_quantity#63, ss_net_paid#64, sr_return_quantity#70, sr_return_amt#71] +Keys [1]: [ss_item_sk#61] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#70, 0)), partial_sum(coalesce(ss_quantity#63, 0)), partial_sum(coalesce(cast(sr_return_amt#71 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#64 as decimal(12,2)), 0.00))] + +(63) ColumnarToRow [codegen id : 11] +Input [7]: [ss_item_sk#61, sum#74, sum#75, sum#76, isEmpty#77, sum#78, isEmpty#79] + +(64) Exchange +Input [7]: [ss_item_sk#61, sum#74, sum#75, sum#76, isEmpty#77, sum#78, isEmpty#79] +Arguments: hashpartitioning(ss_item_sk#61, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(65) HashAggregate [codegen id : 12] +Input [7]: [ss_item_sk#61, sum#74, sum#75, sum#76, isEmpty#77, sum#78, isEmpty#79] +Keys [1]: [ss_item_sk#61] +Functions [4]: [sum(coalesce(sr_return_quantity#70, 0)), sum(coalesce(ss_quantity#63, 0)), sum(coalesce(cast(sr_return_amt#71 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#64 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#70, 0))#80, sum(coalesce(ss_quantity#63, 0))#81, sum(coalesce(cast(sr_return_amt#71 as decimal(12,2)), 0.00))#82, sum(coalesce(cast(ss_net_paid#64 as decimal(12,2)), 0.00))#83] +Results [3]: [ss_item_sk#61 AS item#84, (cast(sum(coalesce(sr_return_quantity#70, 0))#80 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#63, 0))#81 as decimal(15,4))) AS return_ratio#85, (cast(sum(coalesce(cast(sr_return_amt#71 as decimal(12,2)), 0.00))#82 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#64 as decimal(12,2)), 0.00))#83 as decimal(15,4))) AS currency_ratio#86] + +(66) Exchange +Input [3]: [item#84, return_ratio#85, currency_ratio#86] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(73) Sort [codegen id : 13] -Input [3]: [item#88, return_ratio#89, currency_ratio#90] -Arguments: [return_ratio#89 ASC NULLS FIRST], false, 0 +(67) Sort [codegen id : 13] +Input [3]: [item#84, return_ratio#85, currency_ratio#86] +Arguments: [return_ratio#85 ASC NULLS FIRST], false, 0 -(74) Window -Input [3]: [item#88, return_ratio#89, currency_ratio#90] -Arguments: [rank(return_ratio#89) windowspecdefinition(return_ratio#89 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#91], [return_ratio#89 ASC NULLS FIRST] +(68) Window +Input [3]: [item#84, return_ratio#85, currency_ratio#86] +Arguments: [rank(return_ratio#85) windowspecdefinition(return_ratio#85 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#87], [return_ratio#85 ASC NULLS FIRST] -(75) Sort [codegen id : 14] -Input [4]: [item#88, return_ratio#89, currency_ratio#90, return_rank#91] -Arguments: [currency_ratio#90 ASC NULLS FIRST], false, 0 +(69) Sort [codegen id : 14] +Input [4]: [item#84, return_ratio#85, currency_ratio#86, return_rank#87] +Arguments: [currency_ratio#86 ASC NULLS FIRST], false, 0 -(76) Window -Input [4]: [item#88, return_ratio#89, currency_ratio#90, return_rank#91] -Arguments: [rank(currency_ratio#90) windowspecdefinition(currency_ratio#90 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#92], [currency_ratio#90 ASC NULLS FIRST] +(70) Window +Input [4]: [item#84, return_ratio#85, currency_ratio#86, return_rank#87] +Arguments: [rank(currency_ratio#86) windowspecdefinition(currency_ratio#86 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#88], [currency_ratio#86 ASC NULLS FIRST] -(77) Filter [codegen id : 15] -Input [5]: [item#88, return_ratio#89, currency_ratio#90, return_rank#91, currency_rank#92] -Condition : ((return_rank#91 <= 10) OR (currency_rank#92 <= 10)) +(71) Filter [codegen id : 15] +Input [5]: [item#84, return_ratio#85, currency_ratio#86, return_rank#87, currency_rank#88] +Condition : ((return_rank#87 <= 10) OR (currency_rank#88 <= 10)) -(78) Project [codegen id : 15] -Output [5]: [store AS channel#93, item#88, return_ratio#89, return_rank#91, currency_rank#92] -Input [5]: [item#88, return_ratio#89, currency_ratio#90, return_rank#91, currency_rank#92] +(72) Project [codegen id : 15] +Output [5]: [store AS channel#89, item#84, return_ratio#85, return_rank#87, currency_rank#88] +Input [5]: [item#84, return_ratio#85, currency_ratio#86, return_rank#87, currency_rank#88] -(79) Union +(73) Union -(80) HashAggregate [codegen id : 16] +(74) HashAggregate [codegen id : 16] Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] Keys [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] Functions: [] Aggregate Attributes: [] Results [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] -(81) Exchange +(75) Exchange Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] Arguments: hashpartitioning(channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(82) HashAggregate [codegen id : 17] +(76) HashAggregate [codegen id : 17] Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] Keys [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] Functions: [] Aggregate Attributes: [] Results [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] -(83) TakeOrderedAndProject +(77) TakeOrderedAndProject Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] Arguments: 100, [channel#31 ASC NULLS FIRST, return_rank#29 ASC NULLS FIRST, currency_rank#30 ASC NULLS FIRST], [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (88) -+- * ColumnarToRow (87) - +- CometProject (86) - +- CometFilter (85) - +- CometScan parquet spark_catalog.default.date_dim (84) +BroadcastExchange (82) ++- * ColumnarToRow (81) + +- CometProject (80) + +- CometFilter (79) + +- CometScan parquet spark_catalog.default.date_dim (78) -(84) Scan parquet spark_catalog.default.date_dim +(78) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#13, d_year#14, d_moy#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] ReadSchema: struct -(85) CometFilter +(79) CometFilter Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) -(86) CometProject +(80) CometProject Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(87) ColumnarToRow [codegen id : 1] +(81) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(88) BroadcastExchange +(82) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#37 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 53 Hosting Expression = ss_sold_date_sk#68 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#66 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt index 1ded798bd..8d7b158d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt @@ -84,10 +84,7 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] CometFilter [cr_return_amount,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] - CometBroadcastExchange #10 - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + ReusedExchange [d_date_sk] #6 WholeStageCodegen (15) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] @@ -100,11 +97,11 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (13) Sort [return_ratio] InputAdapter - Exchange #11 + Exchange #10 WholeStageCodegen (12) HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange [ss_item_sk] #12 + Exchange [ss_item_sk] #11 WholeStageCodegen (11) ColumnarToRow InputAdapter @@ -113,7 +110,7 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometBroadcastExchange #13 + CometBroadcastExchange #12 CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] @@ -121,7 +118,4 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] CometFilter [sr_return_amt,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] - CometBroadcastExchange #14 - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt index b590d22d4..4e887f93a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -TakeOrderedAndProject (76) -+- * HashAggregate (75) - +- Exchange (74) - +- * HashAggregate (73) - +- * Expand (72) - +- Union (71) +TakeOrderedAndProject (70) ++- * HashAggregate (69) + +- Exchange (68) + +- * HashAggregate (67) + +- * Expand (66) + +- Union (65) :- * HashAggregate (22) : +- Exchange (21) : +- * ColumnarToRow (20) @@ -27,14 +27,14 @@ TakeOrderedAndProject (76) : +- CometBroadcastExchange (16) : +- CometFilter (15) : +- CometScan parquet spark_catalog.default.store (14) - :- * HashAggregate (44) - : +- Exchange (43) - : +- * ColumnarToRow (42) - : +- CometHashAggregate (41) - : +- CometProject (40) - : +- CometBroadcastHashJoin (39) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) + :- * HashAggregate (41) + : +- Exchange (40) + : +- * ColumnarToRow (39) + : +- CometHashAggregate (38) + : +- CometProject (37) + : +- CometBroadcastHashJoin (36) + : :- CometProject (32) + : : +- CometBroadcastHashJoin (31) : : :- CometUnion (29) : : : :- CometProject (25) : : : : +- CometFilter (24) @@ -42,39 +42,33 @@ TakeOrderedAndProject (76) : : : +- CometProject (28) : : : +- CometFilter (27) : : : +- CometScan parquet spark_catalog.default.catalog_returns (26) - : : +- CometBroadcastExchange (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan parquet spark_catalog.default.date_dim (30) - : +- CometBroadcastExchange (38) - : +- CometFilter (37) - : +- CometScan parquet spark_catalog.default.catalog_page (36) - +- * HashAggregate (70) - +- Exchange (69) - +- * ColumnarToRow (68) - +- CometHashAggregate (67) - +- CometProject (66) - +- CometBroadcastHashJoin (65) - :- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometUnion (55) - : : :- CometProject (47) - : : : +- CometFilter (46) - : : : +- CometScan parquet spark_catalog.default.web_sales (45) - : : +- CometProject (54) - : : +- CometBroadcastHashJoin (53) - : : :- CometBroadcastExchange (49) - : : : +- CometScan parquet spark_catalog.default.web_returns (48) - : : +- CometProject (52) - : : +- CometFilter (51) - : : +- CometScan parquet spark_catalog.default.web_sales (50) - : +- CometBroadcastExchange (59) - : +- CometProject (58) - : +- CometFilter (57) - : +- CometScan parquet spark_catalog.default.date_dim (56) - +- CometBroadcastExchange (64) - +- CometFilter (63) - +- CometScan parquet spark_catalog.default.web_site (62) + : : +- ReusedExchange (30) + : +- CometBroadcastExchange (35) + : +- CometFilter (34) + : +- CometScan parquet spark_catalog.default.catalog_page (33) + +- * HashAggregate (64) + +- Exchange (63) + +- * ColumnarToRow (62) + +- CometHashAggregate (61) + +- CometProject (60) + +- CometBroadcastHashJoin (59) + :- CometProject (55) + : +- CometBroadcastHashJoin (54) + : :- CometUnion (52) + : : :- CometProject (44) + : : : +- CometFilter (43) + : : : +- CometScan parquet spark_catalog.default.web_sales (42) + : : +- CometProject (51) + : : +- CometBroadcastHashJoin (50) + : : :- CometBroadcastExchange (46) + : : : +- CometScan parquet spark_catalog.default.web_returns (45) + : : +- CometProject (49) + : : +- CometFilter (48) + : : +- CometScan parquet spark_catalog.default.web_sales (47) + : +- ReusedExchange (53) + +- CometBroadcastExchange (58) + +- CometFilter (57) + +- CometScan parquet spark_catalog.default.web_site (56) (1) Scan parquet spark_catalog.default.store_sales @@ -220,260 +214,228 @@ Arguments: [page_sk#54, date_sk#55, sales_price#56, profit#57, return_amt#58, ne Child 0 Input [6]: [page_sk#44, date_sk#45, sales_price#46, profit#47, return_amt#48, net_loss#49] Child 1 Input [6]: [page_sk#54, date_sk#55, sales_price#56, profit#57, return_amt#58, net_loss#59] -(30) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#60, d_date#61] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [2]: [d_date_sk#60, d_date#61] -Condition : (((isnotnull(d_date#61) AND (d_date#61 >= 2000-08-23)) AND (d_date#61 <= 2000-09-06)) AND isnotnull(d_date_sk#60)) +(30) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#60] -(32) CometProject -Input [2]: [d_date_sk#60, d_date#61] -Arguments: [d_date_sk#60], [d_date_sk#60] - -(33) CometBroadcastExchange -Input [1]: [d_date_sk#60] -Arguments: [d_date_sk#60] - -(34) CometBroadcastHashJoin +(31) CometBroadcastHashJoin Left output [6]: [page_sk#44, date_sk#45, sales_price#46, profit#47, return_amt#48, net_loss#49] Right output [1]: [d_date_sk#60] Arguments: [date_sk#45], [d_date_sk#60], Inner -(35) CometProject +(32) CometProject Input [7]: [page_sk#44, date_sk#45, sales_price#46, profit#47, return_amt#48, net_loss#49, d_date_sk#60] Arguments: [page_sk#44, sales_price#46, profit#47, return_amt#48, net_loss#49], [page_sk#44, sales_price#46, profit#47, return_amt#48, net_loss#49] -(36) Scan parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] +(33) Scan parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(37) CometFilter -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] -Condition : isnotnull(cp_catalog_page_sk#62) +(34) CometFilter +Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Condition : isnotnull(cp_catalog_page_sk#61) -(38) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] -Arguments: [cp_catalog_page_sk#62, cp_catalog_page_id#63] +(35) CometBroadcastExchange +Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Arguments: [cp_catalog_page_sk#61, cp_catalog_page_id#62] -(39) CometBroadcastHashJoin +(36) CometBroadcastHashJoin Left output [5]: [page_sk#44, sales_price#46, profit#47, return_amt#48, net_loss#49] -Right output [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] -Arguments: [page_sk#44], [cp_catalog_page_sk#62], Inner +Right output [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Arguments: [page_sk#44], [cp_catalog_page_sk#61], Inner -(40) CometProject -Input [7]: [page_sk#44, sales_price#46, profit#47, return_amt#48, net_loss#49, cp_catalog_page_sk#62, cp_catalog_page_id#63] -Arguments: [sales_price#46, profit#47, return_amt#48, net_loss#49, cp_catalog_page_id#63], [sales_price#46, profit#47, return_amt#48, net_loss#49, cp_catalog_page_id#63] +(37) CometProject +Input [7]: [page_sk#44, sales_price#46, profit#47, return_amt#48, net_loss#49, cp_catalog_page_sk#61, cp_catalog_page_id#62] +Arguments: [sales_price#46, profit#47, return_amt#48, net_loss#49, cp_catalog_page_id#62], [sales_price#46, profit#47, return_amt#48, net_loss#49, cp_catalog_page_id#62] -(41) CometHashAggregate -Input [5]: [sales_price#46, profit#47, return_amt#48, net_loss#49, cp_catalog_page_id#63] -Keys [1]: [cp_catalog_page_id#63] +(38) CometHashAggregate +Input [5]: [sales_price#46, profit#47, return_amt#48, net_loss#49, cp_catalog_page_id#62] +Keys [1]: [cp_catalog_page_id#62] Functions [4]: [partial_sum(UnscaledValue(sales_price#46)), partial_sum(UnscaledValue(return_amt#48)), partial_sum(UnscaledValue(profit#47)), partial_sum(UnscaledValue(net_loss#49))] -(42) ColumnarToRow [codegen id : 3] -Input [5]: [cp_catalog_page_id#63, sum#64, sum#65, sum#66, sum#67] +(39) ColumnarToRow [codegen id : 3] +Input [5]: [cp_catalog_page_id#62, sum#63, sum#64, sum#65, sum#66] -(43) Exchange -Input [5]: [cp_catalog_page_id#63, sum#64, sum#65, sum#66, sum#67] -Arguments: hashpartitioning(cp_catalog_page_id#63, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(40) Exchange +Input [5]: [cp_catalog_page_id#62, sum#63, sum#64, sum#65, sum#66] +Arguments: hashpartitioning(cp_catalog_page_id#62, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(44) HashAggregate [codegen id : 4] -Input [5]: [cp_catalog_page_id#63, sum#64, sum#65, sum#66, sum#67] -Keys [1]: [cp_catalog_page_id#63] +(41) HashAggregate [codegen id : 4] +Input [5]: [cp_catalog_page_id#62, sum#63, sum#64, sum#65, sum#66] +Keys [1]: [cp_catalog_page_id#62] Functions [4]: [sum(UnscaledValue(sales_price#46)), sum(UnscaledValue(return_amt#48)), sum(UnscaledValue(profit#47)), sum(UnscaledValue(net_loss#49))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#46))#68, sum(UnscaledValue(return_amt#48))#69, sum(UnscaledValue(profit#47))#70, sum(UnscaledValue(net_loss#49))#71] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#46))#68,17,2) AS sales#72, MakeDecimal(sum(UnscaledValue(return_amt#48))#69,17,2) AS returns#73, (MakeDecimal(sum(UnscaledValue(profit#47))#70,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#49))#71,17,2)) AS profit#74, catalog channel AS channel#75, concat(catalog_page, cp_catalog_page_id#63) AS id#76] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#46))#67, sum(UnscaledValue(return_amt#48))#68, sum(UnscaledValue(profit#47))#69, sum(UnscaledValue(net_loss#49))#70] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#46))#67,17,2) AS sales#71, MakeDecimal(sum(UnscaledValue(return_amt#48))#68,17,2) AS returns#72, (MakeDecimal(sum(UnscaledValue(profit#47))#69,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#49))#70,17,2)) AS profit#73, catalog channel AS channel#74, concat(catalog_page, cp_catalog_page_id#62) AS id#75] -(45) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#77, ws_ext_sales_price#78, ws_net_profit#79, ws_sold_date_sk#80] +(42) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_web_site_sk#76, ws_ext_sales_price#77, ws_net_profit#78, ws_sold_date_sk#79] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#80), dynamicpruningexpression(ws_sold_date_sk#80 IN dynamicpruning#81)] +PartitionFilters: [isnotnull(ws_sold_date_sk#79), dynamicpruningexpression(ws_sold_date_sk#79 IN dynamicpruning#80)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(46) CometFilter -Input [4]: [ws_web_site_sk#77, ws_ext_sales_price#78, ws_net_profit#79, ws_sold_date_sk#80] -Condition : isnotnull(ws_web_site_sk#77) +(43) CometFilter +Input [4]: [ws_web_site_sk#76, ws_ext_sales_price#77, ws_net_profit#78, ws_sold_date_sk#79] +Condition : isnotnull(ws_web_site_sk#76) -(47) CometProject -Input [4]: [ws_web_site_sk#77, ws_ext_sales_price#78, ws_net_profit#79, ws_sold_date_sk#80] -Arguments: [wsr_web_site_sk#82, date_sk#83, sales_price#84, profit#85, return_amt#86, net_loss#87], [ws_web_site_sk#77 AS wsr_web_site_sk#82, ws_sold_date_sk#80 AS date_sk#83, ws_ext_sales_price#78 AS sales_price#84, ws_net_profit#79 AS profit#85, 0.00 AS return_amt#86, 0.00 AS net_loss#87] +(44) CometProject +Input [4]: [ws_web_site_sk#76, ws_ext_sales_price#77, ws_net_profit#78, ws_sold_date_sk#79] +Arguments: [wsr_web_site_sk#81, date_sk#82, sales_price#83, profit#84, return_amt#85, net_loss#86], [ws_web_site_sk#76 AS wsr_web_site_sk#81, ws_sold_date_sk#79 AS date_sk#82, ws_ext_sales_price#77 AS sales_price#83, ws_net_profit#78 AS profit#84, 0.00 AS return_amt#85, 0.00 AS net_loss#86] -(48) Scan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#88, wr_order_number#89, wr_return_amt#90, wr_net_loss#91, wr_returned_date_sk#92] +(45) Scan parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#87, wr_order_number#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#92), dynamicpruningexpression(wr_returned_date_sk#92 IN dynamicpruning#81)] +PartitionFilters: [isnotnull(wr_returned_date_sk#91), dynamicpruningexpression(wr_returned_date_sk#91 IN dynamicpruning#80)] ReadSchema: struct -(49) CometBroadcastExchange -Input [5]: [wr_item_sk#88, wr_order_number#89, wr_return_amt#90, wr_net_loss#91, wr_returned_date_sk#92] -Arguments: [wr_item_sk#88, wr_order_number#89, wr_return_amt#90, wr_net_loss#91, wr_returned_date_sk#92] +(46) CometBroadcastExchange +Input [5]: [wr_item_sk#87, wr_order_number#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Arguments: [wr_item_sk#87, wr_order_number#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] -(50) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#93, ws_web_site_sk#94, ws_order_number#95, ws_sold_date_sk#96] +(47) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#92, ws_web_site_sk#93, ws_order_number#94, ws_sold_date_sk#95] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(51) CometFilter -Input [4]: [ws_item_sk#93, ws_web_site_sk#94, ws_order_number#95, ws_sold_date_sk#96] -Condition : ((isnotnull(ws_item_sk#93) AND isnotnull(ws_order_number#95)) AND isnotnull(ws_web_site_sk#94)) +(48) CometFilter +Input [4]: [ws_item_sk#92, ws_web_site_sk#93, ws_order_number#94, ws_sold_date_sk#95] +Condition : ((isnotnull(ws_item_sk#92) AND isnotnull(ws_order_number#94)) AND isnotnull(ws_web_site_sk#93)) -(52) CometProject -Input [4]: [ws_item_sk#93, ws_web_site_sk#94, ws_order_number#95, ws_sold_date_sk#96] -Arguments: [ws_item_sk#93, ws_web_site_sk#94, ws_order_number#95], [ws_item_sk#93, ws_web_site_sk#94, ws_order_number#95] +(49) CometProject +Input [4]: [ws_item_sk#92, ws_web_site_sk#93, ws_order_number#94, ws_sold_date_sk#95] +Arguments: [ws_item_sk#92, ws_web_site_sk#93, ws_order_number#94], [ws_item_sk#92, ws_web_site_sk#93, ws_order_number#94] -(53) CometBroadcastHashJoin -Left output [5]: [wr_item_sk#88, wr_order_number#89, wr_return_amt#90, wr_net_loss#91, wr_returned_date_sk#92] -Right output [3]: [ws_item_sk#93, ws_web_site_sk#94, ws_order_number#95] -Arguments: [wr_item_sk#88, wr_order_number#89], [ws_item_sk#93, ws_order_number#95], Inner +(50) CometBroadcastHashJoin +Left output [5]: [wr_item_sk#87, wr_order_number#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Right output [3]: [ws_item_sk#92, ws_web_site_sk#93, ws_order_number#94] +Arguments: [wr_item_sk#87, wr_order_number#88], [ws_item_sk#92, ws_order_number#94], Inner -(54) CometProject -Input [8]: [wr_item_sk#88, wr_order_number#89, wr_return_amt#90, wr_net_loss#91, wr_returned_date_sk#92, ws_item_sk#93, ws_web_site_sk#94, ws_order_number#95] -Arguments: [wsr_web_site_sk#97, date_sk#98, sales_price#99, profit#100, return_amt#101, net_loss#102], [ws_web_site_sk#94 AS wsr_web_site_sk#97, wr_returned_date_sk#92 AS date_sk#98, 0.00 AS sales_price#99, 0.00 AS profit#100, wr_return_amt#90 AS return_amt#101, wr_net_loss#91 AS net_loss#102] +(51) CometProject +Input [8]: [wr_item_sk#87, wr_order_number#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, ws_item_sk#92, ws_web_site_sk#93, ws_order_number#94] +Arguments: [wsr_web_site_sk#96, date_sk#97, sales_price#98, profit#99, return_amt#100, net_loss#101], [ws_web_site_sk#93 AS wsr_web_site_sk#96, wr_returned_date_sk#91 AS date_sk#97, 0.00 AS sales_price#98, 0.00 AS profit#99, wr_return_amt#89 AS return_amt#100, wr_net_loss#90 AS net_loss#101] -(55) CometUnion -Child 0 Input [6]: [wsr_web_site_sk#82, date_sk#83, sales_price#84, profit#85, return_amt#86, net_loss#87] -Child 1 Input [6]: [wsr_web_site_sk#97, date_sk#98, sales_price#99, profit#100, return_amt#101, net_loss#102] +(52) CometUnion +Child 0 Input [6]: [wsr_web_site_sk#81, date_sk#82, sales_price#83, profit#84, return_amt#85, net_loss#86] +Child 1 Input [6]: [wsr_web_site_sk#96, date_sk#97, sales_price#98, profit#99, return_amt#100, net_loss#101] -(56) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#103, d_date#104] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] -ReadSchema: struct +(53) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#102] -(57) CometFilter -Input [2]: [d_date_sk#103, d_date#104] -Condition : (((isnotnull(d_date#104) AND (d_date#104 >= 2000-08-23)) AND (d_date#104 <= 2000-09-06)) AND isnotnull(d_date_sk#103)) +(54) CometBroadcastHashJoin +Left output [6]: [wsr_web_site_sk#81, date_sk#82, sales_price#83, profit#84, return_amt#85, net_loss#86] +Right output [1]: [d_date_sk#102] +Arguments: [date_sk#82], [d_date_sk#102], Inner -(58) CometProject -Input [2]: [d_date_sk#103, d_date#104] -Arguments: [d_date_sk#103], [d_date_sk#103] +(55) CometProject +Input [7]: [wsr_web_site_sk#81, date_sk#82, sales_price#83, profit#84, return_amt#85, net_loss#86, d_date_sk#102] +Arguments: [wsr_web_site_sk#81, sales_price#83, profit#84, return_amt#85, net_loss#86], [wsr_web_site_sk#81, sales_price#83, profit#84, return_amt#85, net_loss#86] -(59) CometBroadcastExchange -Input [1]: [d_date_sk#103] -Arguments: [d_date_sk#103] - -(60) CometBroadcastHashJoin -Left output [6]: [wsr_web_site_sk#82, date_sk#83, sales_price#84, profit#85, return_amt#86, net_loss#87] -Right output [1]: [d_date_sk#103] -Arguments: [date_sk#83], [d_date_sk#103], Inner - -(61) CometProject -Input [7]: [wsr_web_site_sk#82, date_sk#83, sales_price#84, profit#85, return_amt#86, net_loss#87, d_date_sk#103] -Arguments: [wsr_web_site_sk#82, sales_price#84, profit#85, return_amt#86, net_loss#87], [wsr_web_site_sk#82, sales_price#84, profit#85, return_amt#86, net_loss#87] - -(62) Scan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#105, web_site_id#106] +(56) Scan parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#103, web_site_id#104] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(63) CometFilter -Input [2]: [web_site_sk#105, web_site_id#106] -Condition : isnotnull(web_site_sk#105) +(57) CometFilter +Input [2]: [web_site_sk#103, web_site_id#104] +Condition : isnotnull(web_site_sk#103) -(64) CometBroadcastExchange -Input [2]: [web_site_sk#105, web_site_id#106] -Arguments: [web_site_sk#105, web_site_id#106] +(58) CometBroadcastExchange +Input [2]: [web_site_sk#103, web_site_id#104] +Arguments: [web_site_sk#103, web_site_id#104] -(65) CometBroadcastHashJoin -Left output [5]: [wsr_web_site_sk#82, sales_price#84, profit#85, return_amt#86, net_loss#87] -Right output [2]: [web_site_sk#105, web_site_id#106] -Arguments: [wsr_web_site_sk#82], [web_site_sk#105], Inner +(59) CometBroadcastHashJoin +Left output [5]: [wsr_web_site_sk#81, sales_price#83, profit#84, return_amt#85, net_loss#86] +Right output [2]: [web_site_sk#103, web_site_id#104] +Arguments: [wsr_web_site_sk#81], [web_site_sk#103], Inner -(66) CometProject -Input [7]: [wsr_web_site_sk#82, sales_price#84, profit#85, return_amt#86, net_loss#87, web_site_sk#105, web_site_id#106] -Arguments: [sales_price#84, profit#85, return_amt#86, net_loss#87, web_site_id#106], [sales_price#84, profit#85, return_amt#86, net_loss#87, web_site_id#106] +(60) CometProject +Input [7]: [wsr_web_site_sk#81, sales_price#83, profit#84, return_amt#85, net_loss#86, web_site_sk#103, web_site_id#104] +Arguments: [sales_price#83, profit#84, return_amt#85, net_loss#86, web_site_id#104], [sales_price#83, profit#84, return_amt#85, net_loss#86, web_site_id#104] -(67) CometHashAggregate -Input [5]: [sales_price#84, profit#85, return_amt#86, net_loss#87, web_site_id#106] -Keys [1]: [web_site_id#106] -Functions [4]: [partial_sum(UnscaledValue(sales_price#84)), partial_sum(UnscaledValue(return_amt#86)), partial_sum(UnscaledValue(profit#85)), partial_sum(UnscaledValue(net_loss#87))] +(61) CometHashAggregate +Input [5]: [sales_price#83, profit#84, return_amt#85, net_loss#86, web_site_id#104] +Keys [1]: [web_site_id#104] +Functions [4]: [partial_sum(UnscaledValue(sales_price#83)), partial_sum(UnscaledValue(return_amt#85)), partial_sum(UnscaledValue(profit#84)), partial_sum(UnscaledValue(net_loss#86))] -(68) ColumnarToRow [codegen id : 5] -Input [5]: [web_site_id#106, sum#107, sum#108, sum#109, sum#110] +(62) ColumnarToRow [codegen id : 5] +Input [5]: [web_site_id#104, sum#105, sum#106, sum#107, sum#108] -(69) Exchange -Input [5]: [web_site_id#106, sum#107, sum#108, sum#109, sum#110] -Arguments: hashpartitioning(web_site_id#106, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(63) Exchange +Input [5]: [web_site_id#104, sum#105, sum#106, sum#107, sum#108] +Arguments: hashpartitioning(web_site_id#104, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(70) HashAggregate [codegen id : 6] -Input [5]: [web_site_id#106, sum#107, sum#108, sum#109, sum#110] -Keys [1]: [web_site_id#106] -Functions [4]: [sum(UnscaledValue(sales_price#84)), sum(UnscaledValue(return_amt#86)), sum(UnscaledValue(profit#85)), sum(UnscaledValue(net_loss#87))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#84))#111, sum(UnscaledValue(return_amt#86))#112, sum(UnscaledValue(profit#85))#113, sum(UnscaledValue(net_loss#87))#114] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#84))#111,17,2) AS sales#115, MakeDecimal(sum(UnscaledValue(return_amt#86))#112,17,2) AS returns#116, (MakeDecimal(sum(UnscaledValue(profit#85))#113,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#87))#114,17,2)) AS profit#117, web channel AS channel#118, concat(web_site, web_site_id#106) AS id#119] +(64) HashAggregate [codegen id : 6] +Input [5]: [web_site_id#104, sum#105, sum#106, sum#107, sum#108] +Keys [1]: [web_site_id#104] +Functions [4]: [sum(UnscaledValue(sales_price#83)), sum(UnscaledValue(return_amt#85)), sum(UnscaledValue(profit#84)), sum(UnscaledValue(net_loss#86))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#83))#109, sum(UnscaledValue(return_amt#85))#110, sum(UnscaledValue(profit#84))#111, sum(UnscaledValue(net_loss#86))#112] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#83))#109,17,2) AS sales#113, MakeDecimal(sum(UnscaledValue(return_amt#85))#110,17,2) AS returns#114, (MakeDecimal(sum(UnscaledValue(profit#84))#111,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#86))#112,17,2)) AS profit#115, web channel AS channel#116, concat(web_site, web_site_id#104) AS id#117] -(71) Union +(65) Union -(72) Expand [codegen id : 7] +(66) Expand [codegen id : 7] 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#120, id#121, spark_grouping_id#122] +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#118, id#119, spark_grouping_id#120] -(73) HashAggregate [codegen id : 7] -Input [6]: [sales#34, returns#35, profit#36, channel#120, id#121, spark_grouping_id#122] -Keys [3]: [channel#120, id#121, spark_grouping_id#122] +(67) HashAggregate [codegen id : 7] +Input [6]: [sales#34, returns#35, profit#36, channel#118, id#119, spark_grouping_id#120] +Keys [3]: [channel#118, id#119, spark_grouping_id#120] Functions [3]: [partial_sum(sales#34), partial_sum(returns#35), partial_sum(profit#36)] -Aggregate Attributes [6]: [sum#123, isEmpty#124, sum#125, isEmpty#126, sum#127, isEmpty#128] -Results [9]: [channel#120, id#121, spark_grouping_id#122, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] +Aggregate Attributes [6]: [sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] +Results [9]: [channel#118, id#119, spark_grouping_id#120, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -(74) Exchange -Input [9]: [channel#120, id#121, spark_grouping_id#122, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] -Arguments: hashpartitioning(channel#120, id#121, spark_grouping_id#122, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(68) Exchange +Input [9]: [channel#118, id#119, spark_grouping_id#120, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] +Arguments: hashpartitioning(channel#118, id#119, spark_grouping_id#120, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(75) HashAggregate [codegen id : 8] -Input [9]: [channel#120, id#121, spark_grouping_id#122, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] -Keys [3]: [channel#120, id#121, spark_grouping_id#122] +(69) HashAggregate [codegen id : 8] +Input [9]: [channel#118, id#119, spark_grouping_id#120, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] +Keys [3]: [channel#118, id#119, spark_grouping_id#120] Functions [3]: [sum(sales#34), sum(returns#35), sum(profit#36)] -Aggregate Attributes [3]: [sum(sales#34)#135, sum(returns#35)#136, sum(profit#36)#137] -Results [5]: [channel#120, id#121, sum(sales#34)#135 AS sales#138, sum(returns#35)#136 AS returns#139, sum(profit#36)#137 AS profit#140] +Aggregate Attributes [3]: [sum(sales#34)#133, sum(returns#35)#134, sum(profit#36)#135] +Results [5]: [channel#118, id#119, sum(sales#34)#133 AS sales#136, sum(returns#35)#134 AS returns#137, sum(profit#36)#135 AS profit#138] -(76) TakeOrderedAndProject -Input [5]: [channel#120, id#121, sales#138, returns#139, profit#140] -Arguments: 100, [channel#120 ASC NULLS FIRST, id#121 ASC NULLS FIRST], [channel#120, id#121, sales#138, returns#139, profit#140] +(70) TakeOrderedAndProject +Input [5]: [channel#118, id#119, sales#136, returns#137, profit#138] +Arguments: 100, [channel#118 ASC NULLS FIRST, id#119 ASC NULLS FIRST], [channel#118, id#119, sales#136, returns#137, profit#138] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (81) -+- * ColumnarToRow (80) - +- CometProject (79) - +- CometFilter (78) - +- CometScan parquet spark_catalog.default.date_dim (77) +BroadcastExchange (75) ++- * ColumnarToRow (74) + +- CometProject (73) + +- CometFilter (72) + +- CometScan parquet spark_catalog.default.date_dim (71) -(77) Scan parquet spark_catalog.default.date_dim +(71) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#22, d_date#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] ReadSchema: struct -(78) CometFilter +(72) CometFilter Input [2]: [d_date_sk#22, d_date#23] Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#23 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) -(79) CometProject +(73) CometProject Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(80) ColumnarToRow [codegen id : 1] +(74) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(81) BroadcastExchange +(75) BroadcastExchange Input [1]: [d_date_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] @@ -483,8 +445,8 @@ Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#42 IN d Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#53 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 45 Hosting Expression = ws_sold_date_sk#80 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#79 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 48 Hosting Expression = wr_returned_date_sk#92 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 45 Hosting Expression = wr_returned_date_sk#91 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt index a524c8b2c..c4ea8fe24 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt @@ -64,17 +64,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometFilter [cr_catalog_page_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #7 - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #8 + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange #7 CometFilter [cp_catalog_page_sk] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (6) HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] InputAdapter - Exchange [web_site_id] #9 + Exchange [web_site_id] #8 WholeStageCodegen (5) ColumnarToRow InputAdapter @@ -90,16 +87,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedSubquery [d_date_sk] #1 CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - CometBroadcastExchange #10 + CometBroadcastExchange #9 CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - CometBroadcastExchange #11 - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #12 + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange #10 CometFilter [web_site_sk] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt index 59340724a..ed52d859e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt @@ -1,11 +1,11 @@ == Physical Plan == -TakeOrderedAndProject (43) -+- * Filter (42) - +- Window (41) - +- * Sort (40) - +- Exchange (39) - +- * Project (38) - +- * SortMergeJoin FullOuter (37) +TakeOrderedAndProject (40) ++- * Filter (39) + +- Window (38) + +- * Sort (37) + +- Exchange (36) + +- * Project (35) + +- * SortMergeJoin FullOuter (34) :- * Sort (18) : +- Exchange (17) : +- * Project (16) @@ -24,24 +24,21 @@ TakeOrderedAndProject (43) : +- CometProject (5) : +- CometFilter (4) : +- CometScan parquet spark_catalog.default.date_dim (3) - +- * Sort (36) - +- Exchange (35) - +- * Project (34) - +- Window (33) - +- * Sort (32) - +- Exchange (31) - +- * HashAggregate (30) - +- Exchange (29) - +- * ColumnarToRow (28) - +- CometHashAggregate (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) + +- * Sort (33) + +- Exchange (32) + +- * Project (31) + +- Window (30) + +- * Sort (29) + +- Exchange (28) + +- * HashAggregate (27) + +- Exchange (26) + +- * ColumnarToRow (25) + +- CometHashAggregate (24) + +- CometProject (23) + +- CometBroadcastHashJoin (22) :- CometFilter (20) : +- CometScan parquet spark_catalog.default.store_sales (19) - +- CometBroadcastExchange (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan parquet spark_catalog.default.date_dim (21) + +- ReusedExchange (21) (1) Scan parquet spark_catalog.default.web_sales @@ -139,136 +136,120 @@ ReadSchema: struct Input [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] Condition : isnotnull(ss_item_sk#13) -(21) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_date#18, d_month_seq#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [d_date_sk#17, d_date#18, d_month_seq#19] -Condition : (((isnotnull(d_month_seq#19) AND (d_month_seq#19 >= 1200)) AND (d_month_seq#19 <= 1211)) AND isnotnull(d_date_sk#17)) +(21) ReusedExchange [Reuses operator id: 6] +Output [2]: [d_date_sk#17, d_date#18] -(23) CometProject -Input [3]: [d_date_sk#17, d_date#18, d_month_seq#19] -Arguments: [d_date_sk#17, d_date#18], [d_date_sk#17, d_date#18] - -(24) CometBroadcastExchange -Input [2]: [d_date_sk#17, d_date#18] -Arguments: [d_date_sk#17, d_date#18] - -(25) CometBroadcastHashJoin +(22) CometBroadcastHashJoin Left output [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] Right output [2]: [d_date_sk#17, d_date#18] Arguments: [ss_sold_date_sk#15], [d_date_sk#17], Inner -(26) CometProject +(23) CometProject Input [5]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15, d_date_sk#17, d_date#18] Arguments: [ss_item_sk#13, ss_sales_price#14, d_date#18], [ss_item_sk#13, ss_sales_price#14, d_date#18] -(27) CometHashAggregate +(24) CometHashAggregate Input [3]: [ss_item_sk#13, ss_sales_price#14, d_date#18] Keys [2]: [ss_item_sk#13, d_date#18] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#14))] -(28) ColumnarToRow [codegen id : 6] -Input [3]: [ss_item_sk#13, d_date#18, sum#20] +(25) ColumnarToRow [codegen id : 6] +Input [3]: [ss_item_sk#13, d_date#18, sum#19] -(29) Exchange -Input [3]: [ss_item_sk#13, d_date#18, sum#20] +(26) Exchange +Input [3]: [ss_item_sk#13, d_date#18, sum#19] Arguments: hashpartitioning(ss_item_sk#13, d_date#18, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(30) HashAggregate [codegen id : 7] -Input [3]: [ss_item_sk#13, d_date#18, sum#20] +(27) HashAggregate [codegen id : 7] +Input [3]: [ss_item_sk#13, d_date#18, sum#19] Keys [2]: [ss_item_sk#13, d_date#18] Functions [1]: [sum(UnscaledValue(ss_sales_price#14))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#14))#21] -Results [4]: [ss_item_sk#13 AS item_sk#22, d_date#18, MakeDecimal(sum(UnscaledValue(ss_sales_price#14))#21,17,2) AS _w0#23, ss_item_sk#13] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#14))#20] +Results [4]: [ss_item_sk#13 AS item_sk#21, d_date#18, MakeDecimal(sum(UnscaledValue(ss_sales_price#14))#20,17,2) AS _w0#22, ss_item_sk#13] -(31) Exchange -Input [4]: [item_sk#22, d_date#18, _w0#23, ss_item_sk#13] +(28) Exchange +Input [4]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13] Arguments: hashpartitioning(ss_item_sk#13, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(32) Sort [codegen id : 8] -Input [4]: [item_sk#22, d_date#18, _w0#23, ss_item_sk#13] +(29) Sort [codegen id : 8] +Input [4]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13] Arguments: [ss_item_sk#13 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST], false, 0 -(33) Window -Input [4]: [item_sk#22, d_date#18, _w0#23, ss_item_sk#13] -Arguments: [sum(_w0#23) windowspecdefinition(ss_item_sk#13, d_date#18 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#24], [ss_item_sk#13], [d_date#18 ASC NULLS FIRST] +(30) Window +Input [4]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13] +Arguments: [sum(_w0#22) windowspecdefinition(ss_item_sk#13, d_date#18 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#23], [ss_item_sk#13], [d_date#18 ASC NULLS FIRST] -(34) Project [codegen id : 9] -Output [3]: [item_sk#22, d_date#18, cume_sales#24] -Input [5]: [item_sk#22, d_date#18, _w0#23, ss_item_sk#13, cume_sales#24] +(31) Project [codegen id : 9] +Output [3]: [item_sk#21, d_date#18, cume_sales#23] +Input [5]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13, cume_sales#23] -(35) Exchange -Input [3]: [item_sk#22, d_date#18, cume_sales#24] -Arguments: hashpartitioning(item_sk#22, d_date#18, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(32) Exchange +Input [3]: [item_sk#21, d_date#18, cume_sales#23] +Arguments: hashpartitioning(item_sk#21, d_date#18, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(36) Sort [codegen id : 10] -Input [3]: [item_sk#22, d_date#18, cume_sales#24] -Arguments: [item_sk#22 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST], false, 0 +(33) Sort [codegen id : 10] +Input [3]: [item_sk#21, d_date#18, cume_sales#23] +Arguments: [item_sk#21 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST], false, 0 -(37) SortMergeJoin [codegen id : 11] +(34) SortMergeJoin [codegen id : 11] Left keys [2]: [item_sk#10, d_date#6] -Right keys [2]: [item_sk#22, d_date#18] +Right keys [2]: [item_sk#21, d_date#18] Join type: FullOuter Join condition: None -(38) Project [codegen id : 11] -Output [4]: [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#22 END AS item_sk#25, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#18 END AS d_date#26, cume_sales#12 AS web_sales#27, cume_sales#24 AS store_sales#28] -Input [6]: [item_sk#10, d_date#6, cume_sales#12, item_sk#22, d_date#18, cume_sales#24] +(35) Project [codegen id : 11] +Output [4]: [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#21 END AS item_sk#24, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#18 END AS d_date#25, cume_sales#12 AS web_sales#26, cume_sales#23 AS store_sales#27] +Input [6]: [item_sk#10, d_date#6, cume_sales#12, item_sk#21, d_date#18, cume_sales#23] -(39) Exchange -Input [4]: [item_sk#25, d_date#26, web_sales#27, store_sales#28] -Arguments: hashpartitioning(item_sk#25, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(36) Exchange +Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] +Arguments: hashpartitioning(item_sk#24, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(40) Sort [codegen id : 12] -Input [4]: [item_sk#25, d_date#26, web_sales#27, store_sales#28] -Arguments: [item_sk#25 ASC NULLS FIRST, d_date#26 ASC NULLS FIRST], false, 0 +(37) Sort [codegen id : 12] +Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] +Arguments: [item_sk#24 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST], false, 0 -(41) Window -Input [4]: [item_sk#25, d_date#26, web_sales#27, store_sales#28] -Arguments: [max(web_sales#27) windowspecdefinition(item_sk#25, d_date#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#29, max(store_sales#28) windowspecdefinition(item_sk#25, d_date#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#30], [item_sk#25], [d_date#26 ASC NULLS FIRST] +(38) Window +Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] +Arguments: [max(web_sales#26) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#28, max(store_sales#27) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#29], [item_sk#24], [d_date#25 ASC NULLS FIRST] -(42) Filter [codegen id : 13] -Input [6]: [item_sk#25, d_date#26, web_sales#27, store_sales#28, web_cumulative#29, store_cumulative#30] -Condition : ((isnotnull(web_cumulative#29) AND isnotnull(store_cumulative#30)) AND (web_cumulative#29 > store_cumulative#30)) +(39) Filter [codegen id : 13] +Input [6]: [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] +Condition : ((isnotnull(web_cumulative#28) AND isnotnull(store_cumulative#29)) AND (web_cumulative#28 > store_cumulative#29)) -(43) TakeOrderedAndProject -Input [6]: [item_sk#25, d_date#26, web_sales#27, store_sales#28, web_cumulative#29, store_cumulative#30] -Arguments: 100, [item_sk#25 ASC NULLS FIRST, d_date#26 ASC NULLS FIRST], [item_sk#25, d_date#26, web_sales#27, store_sales#28, web_cumulative#29, store_cumulative#30] +(40) TakeOrderedAndProject +Input [6]: [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] +Arguments: 100, [item_sk#24 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST], [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (48) -+- * ColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan parquet spark_catalog.default.date_dim (44) +BroadcastExchange (45) ++- * ColumnarToRow (44) + +- CometProject (43) + +- CometFilter (42) + +- CometScan parquet spark_catalog.default.date_dim (41) -(44) Scan parquet spark_catalog.default.date_dim +(41) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(45) CometFilter +(42) CometFilter Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#5)) -(46) CometProject +(43) CometProject Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(47) ColumnarToRow [codegen id : 1] +(44) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(48) BroadcastExchange +(45) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt index ba9ad48d8..ce6005da6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt @@ -73,7 +73,4 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #10 - CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + ReusedExchange [d_date_sk,d_date] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt index 8fed6e48f..a0c4b7286 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (89) -+- * HashAggregate (88) - +- Exchange (87) - +- * HashAggregate (86) - +- Union (85) +TakeOrderedAndProject (63) ++- * HashAggregate (62) + +- Exchange (61) + +- * HashAggregate (60) + +- Union (59) :- * HashAggregate (28) : +- Exchange (27) : +- * ColumnarToRow (26) @@ -32,62 +32,36 @@ TakeOrderedAndProject (89) : +- CometProject (19) : +- CometFilter (18) : +- CometScan parquet spark_catalog.default.item (17) - :- * HashAggregate (56) - : +- Exchange (55) - : +- * ColumnarToRow (54) - : +- CometHashAggregate (53) - : +- CometProject (52) - : +- CometBroadcastHashJoin (51) - : :- CometProject (42) - : : +- CometBroadcastHashJoin (41) - : : :- CometProject (36) - : : : +- CometBroadcastHashJoin (35) + :- * HashAggregate (43) + : +- Exchange (42) + : +- * ColumnarToRow (41) + : +- CometHashAggregate (40) + : +- CometProject (39) + : +- CometBroadcastHashJoin (38) + : :- CometProject (36) + : : +- CometBroadcastHashJoin (35) + : : :- CometProject (33) + : : : +- CometBroadcastHashJoin (32) : : : :- CometFilter (30) : : : : +- CometScan parquet spark_catalog.default.catalog_sales (29) - : : : +- CometBroadcastExchange (34) - : : : +- CometProject (33) - : : : +- CometFilter (32) - : : : +- CometScan parquet spark_catalog.default.date_dim (31) - : : +- CometBroadcastExchange (40) - : : +- CometProject (39) - : : +- CometFilter (38) - : : +- CometScan parquet spark_catalog.default.customer_address (37) - : +- CometBroadcastExchange (50) - : +- CometBroadcastHashJoin (49) - : :- CometFilter (44) - : : +- CometScan parquet spark_catalog.default.item (43) - : +- CometBroadcastExchange (48) - : +- CometProject (47) - : +- CometFilter (46) - : +- CometScan parquet spark_catalog.default.item (45) - +- * HashAggregate (84) - +- Exchange (83) - +- * ColumnarToRow (82) - +- CometHashAggregate (81) - +- CometProject (80) - +- CometBroadcastHashJoin (79) - :- CometProject (70) - : +- CometBroadcastHashJoin (69) - : :- CometProject (64) - : : +- CometBroadcastHashJoin (63) - : : :- CometFilter (58) - : : : +- CometScan parquet spark_catalog.default.web_sales (57) - : : +- CometBroadcastExchange (62) - : : +- CometProject (61) - : : +- CometFilter (60) - : : +- CometScan parquet spark_catalog.default.date_dim (59) - : +- CometBroadcastExchange (68) - : +- CometProject (67) - : +- CometFilter (66) - : +- CometScan parquet spark_catalog.default.customer_address (65) - +- CometBroadcastExchange (78) - +- CometBroadcastHashJoin (77) - :- CometFilter (72) - : +- CometScan parquet spark_catalog.default.item (71) - +- CometBroadcastExchange (76) - +- CometProject (75) - +- CometFilter (74) - +- CometScan parquet spark_catalog.default.item (73) + : : : +- ReusedExchange (31) + : : +- ReusedExchange (34) + : +- ReusedExchange (37) + +- * HashAggregate (58) + +- Exchange (57) + +- * ColumnarToRow (56) + +- CometHashAggregate (55) + +- CometProject (54) + +- CometBroadcastHashJoin (53) + :- CometProject (51) + : +- CometBroadcastHashJoin (50) + : :- CometProject (48) + : : +- CometBroadcastHashJoin (47) + : : :- CometFilter (45) + : : : +- CometScan parquet spark_catalog.default.web_sales (44) + : : +- ReusedExchange (46) + : +- ReusedExchange (49) + +- ReusedExchange (52) (1) Scan parquet spark_catalog.default.store_sales @@ -237,322 +211,186 @@ ReadSchema: struct - -(32) CometFilter -Input [3]: [d_date_sk#23, d_year#24, d_moy#25] -Condition : ((((isnotnull(d_year#24) AND isnotnull(d_moy#25)) AND (d_year#24 = 2001)) AND (d_moy#25 = 2)) AND isnotnull(d_date_sk#23)) +(31) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#23] -(33) CometProject -Input [3]: [d_date_sk#23, d_year#24, d_moy#25] -Arguments: [d_date_sk#23], [d_date_sk#23] - -(34) CometBroadcastExchange -Input [1]: [d_date_sk#23] -Arguments: [d_date_sk#23] - -(35) CometBroadcastHashJoin +(32) CometBroadcastHashJoin Left output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] Right output [1]: [d_date_sk#23] Arguments: [cs_sold_date_sk#21], [d_date_sk#23], Inner -(36) CometProject +(33) CometProject Input [5]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#23] Arguments: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20], [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] -(37) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#26, ca_gmt_offset#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(38) CometFilter -Input [2]: [ca_address_sk#26, ca_gmt_offset#27] -Condition : ((isnotnull(ca_gmt_offset#27) AND (ca_gmt_offset#27 = -5.00)) AND isnotnull(ca_address_sk#26)) +(34) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#24] -(39) CometProject -Input [2]: [ca_address_sk#26, ca_gmt_offset#27] -Arguments: [ca_address_sk#26], [ca_address_sk#26] - -(40) CometBroadcastExchange -Input [1]: [ca_address_sk#26] -Arguments: [ca_address_sk#26] - -(41) CometBroadcastHashJoin +(35) CometBroadcastHashJoin Left output [3]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] -Right output [1]: [ca_address_sk#26] -Arguments: [cs_bill_addr_sk#18], [ca_address_sk#26], Inner +Right output [1]: [ca_address_sk#24] +Arguments: [cs_bill_addr_sk#18], [ca_address_sk#24], Inner -(42) CometProject -Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, ca_address_sk#26] +(36) CometProject +Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, ca_address_sk#24] Arguments: [cs_item_sk#19, cs_ext_sales_price#20], [cs_item_sk#19, cs_ext_sales_price#20] -(43) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#28, i_item_id#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(44) CometFilter -Input [2]: [i_item_sk#28, i_item_id#29] -Condition : isnotnull(i_item_sk#28) - -(45) Scan parquet spark_catalog.default.item -Output [2]: [i_item_id#30, i_color#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_color, [blanched ,burnished ,slate ])] -ReadSchema: struct - -(46) CometFilter -Input [2]: [i_item_id#30, i_color#31] -Condition : i_color#31 IN (slate ,blanched ,burnished ) - -(47) CometProject -Input [2]: [i_item_id#30, i_color#31] -Arguments: [i_item_id#30], [i_item_id#30] +(37) ReusedExchange [Reuses operator id: 22] +Output [2]: [i_item_sk#25, i_item_id#26] -(48) CometBroadcastExchange -Input [1]: [i_item_id#30] -Arguments: [i_item_id#30] - -(49) CometBroadcastHashJoin -Left output [2]: [i_item_sk#28, i_item_id#29] -Right output [1]: [i_item_id#30] -Arguments: [i_item_id#29], [i_item_id#30], LeftSemi - -(50) CometBroadcastExchange -Input [2]: [i_item_sk#28, i_item_id#29] -Arguments: [i_item_sk#28, i_item_id#29] - -(51) CometBroadcastHashJoin +(38) CometBroadcastHashJoin Left output [2]: [cs_item_sk#19, cs_ext_sales_price#20] -Right output [2]: [i_item_sk#28, i_item_id#29] -Arguments: [cs_item_sk#19], [i_item_sk#28], Inner +Right output [2]: [i_item_sk#25, i_item_id#26] +Arguments: [cs_item_sk#19], [i_item_sk#25], Inner -(52) CometProject -Input [4]: [cs_item_sk#19, cs_ext_sales_price#20, i_item_sk#28, i_item_id#29] -Arguments: [cs_ext_sales_price#20, i_item_id#29], [cs_ext_sales_price#20, i_item_id#29] +(39) CometProject +Input [4]: [cs_item_sk#19, cs_ext_sales_price#20, i_item_sk#25, i_item_id#26] +Arguments: [cs_ext_sales_price#20, i_item_id#26], [cs_ext_sales_price#20, i_item_id#26] -(53) CometHashAggregate -Input [2]: [cs_ext_sales_price#20, i_item_id#29] -Keys [1]: [i_item_id#29] +(40) CometHashAggregate +Input [2]: [cs_ext_sales_price#20, i_item_id#26] +Keys [1]: [i_item_id#26] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#20))] -(54) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_id#29, sum#32] +(41) ColumnarToRow [codegen id : 3] +Input [2]: [i_item_id#26, sum#27] -(55) Exchange -Input [2]: [i_item_id#29, sum#32] -Arguments: hashpartitioning(i_item_id#29, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(42) Exchange +Input [2]: [i_item_id#26, sum#27] +Arguments: hashpartitioning(i_item_id#26, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(56) HashAggregate [codegen id : 4] -Input [2]: [i_item_id#29, sum#32] -Keys [1]: [i_item_id#29] +(43) HashAggregate [codegen id : 4] +Input [2]: [i_item_id#26, sum#27] +Keys [1]: [i_item_id#26] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#20))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#20))#33] -Results [2]: [i_item_id#29, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#20))#33,17,2) AS total_sales#34] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#20))#28] +Results [2]: [i_item_id#26, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#20))#28,17,2) AS total_sales#29] -(57) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#35, ws_bill_addr_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38] +(44) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#39)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#34)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(58) CometFilter -Input [4]: [ws_item_sk#35, ws_bill_addr_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38] -Condition : (isnotnull(ws_bill_addr_sk#36) AND isnotnull(ws_item_sk#35)) - -(59) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#40, d_year#41, d_moy#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct +(45) CometFilter +Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] +Condition : (isnotnull(ws_bill_addr_sk#31) AND isnotnull(ws_item_sk#30)) -(60) CometFilter -Input [3]: [d_date_sk#40, d_year#41, d_moy#42] -Condition : ((((isnotnull(d_year#41) AND isnotnull(d_moy#42)) AND (d_year#41 = 2001)) AND (d_moy#42 = 2)) AND isnotnull(d_date_sk#40)) +(46) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#35] -(61) CometProject -Input [3]: [d_date_sk#40, d_year#41, d_moy#42] -Arguments: [d_date_sk#40], [d_date_sk#40] - -(62) CometBroadcastExchange -Input [1]: [d_date_sk#40] -Arguments: [d_date_sk#40] - -(63) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#35, ws_bill_addr_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38] -Right output [1]: [d_date_sk#40] -Arguments: [ws_sold_date_sk#38], [d_date_sk#40], Inner - -(64) CometProject -Input [5]: [ws_item_sk#35, ws_bill_addr_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38, d_date_sk#40] -Arguments: [ws_item_sk#35, ws_bill_addr_sk#36, ws_ext_sales_price#37], [ws_item_sk#35, ws_bill_addr_sk#36, ws_ext_sales_price#37] - -(65) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#43, ca_gmt_offset#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(66) CometFilter -Input [2]: [ca_address_sk#43, ca_gmt_offset#44] -Condition : ((isnotnull(ca_gmt_offset#44) AND (ca_gmt_offset#44 = -5.00)) AND isnotnull(ca_address_sk#43)) - -(67) CometProject -Input [2]: [ca_address_sk#43, ca_gmt_offset#44] -Arguments: [ca_address_sk#43], [ca_address_sk#43] - -(68) CometBroadcastExchange -Input [1]: [ca_address_sk#43] -Arguments: [ca_address_sk#43] - -(69) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#35, ws_bill_addr_sk#36, ws_ext_sales_price#37] -Right output [1]: [ca_address_sk#43] -Arguments: [ws_bill_addr_sk#36], [ca_address_sk#43], Inner - -(70) CometProject -Input [4]: [ws_item_sk#35, ws_bill_addr_sk#36, ws_ext_sales_price#37, ca_address_sk#43] -Arguments: [ws_item_sk#35, ws_ext_sales_price#37], [ws_item_sk#35, ws_ext_sales_price#37] - -(71) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#45, i_item_id#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(72) CometFilter -Input [2]: [i_item_sk#45, i_item_id#46] -Condition : isnotnull(i_item_sk#45) - -(73) Scan parquet spark_catalog.default.item -Output [2]: [i_item_id#47, i_color#48] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_color, [blanched ,burnished ,slate ])] -ReadSchema: struct +(47) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] +Right output [1]: [d_date_sk#35] +Arguments: [ws_sold_date_sk#33], [d_date_sk#35], Inner -(74) CometFilter -Input [2]: [i_item_id#47, i_color#48] -Condition : i_color#48 IN (slate ,blanched ,burnished ) +(48) CometProject +Input [5]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33, d_date_sk#35] +Arguments: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32], [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] -(75) CometProject -Input [2]: [i_item_id#47, i_color#48] -Arguments: [i_item_id#47], [i_item_id#47] +(49) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#36] -(76) CometBroadcastExchange -Input [1]: [i_item_id#47] -Arguments: [i_item_id#47] +(50) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] +Right output [1]: [ca_address_sk#36] +Arguments: [ws_bill_addr_sk#31], [ca_address_sk#36], Inner -(77) CometBroadcastHashJoin -Left output [2]: [i_item_sk#45, i_item_id#46] -Right output [1]: [i_item_id#47] -Arguments: [i_item_id#46], [i_item_id#47], LeftSemi +(51) CometProject +Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ca_address_sk#36] +Arguments: [ws_item_sk#30, ws_ext_sales_price#32], [ws_item_sk#30, ws_ext_sales_price#32] -(78) CometBroadcastExchange -Input [2]: [i_item_sk#45, i_item_id#46] -Arguments: [i_item_sk#45, i_item_id#46] +(52) ReusedExchange [Reuses operator id: 22] +Output [2]: [i_item_sk#37, i_item_id#38] -(79) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#35, ws_ext_sales_price#37] -Right output [2]: [i_item_sk#45, i_item_id#46] -Arguments: [ws_item_sk#35], [i_item_sk#45], Inner +(53) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#30, ws_ext_sales_price#32] +Right output [2]: [i_item_sk#37, i_item_id#38] +Arguments: [ws_item_sk#30], [i_item_sk#37], Inner -(80) CometProject -Input [4]: [ws_item_sk#35, ws_ext_sales_price#37, i_item_sk#45, i_item_id#46] -Arguments: [ws_ext_sales_price#37, i_item_id#46], [ws_ext_sales_price#37, i_item_id#46] +(54) CometProject +Input [4]: [ws_item_sk#30, ws_ext_sales_price#32, i_item_sk#37, i_item_id#38] +Arguments: [ws_ext_sales_price#32, i_item_id#38], [ws_ext_sales_price#32, i_item_id#38] -(81) CometHashAggregate -Input [2]: [ws_ext_sales_price#37, i_item_id#46] -Keys [1]: [i_item_id#46] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#37))] +(55) CometHashAggregate +Input [2]: [ws_ext_sales_price#32, i_item_id#38] +Keys [1]: [i_item_id#38] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#32))] -(82) ColumnarToRow [codegen id : 5] -Input [2]: [i_item_id#46, sum#49] +(56) ColumnarToRow [codegen id : 5] +Input [2]: [i_item_id#38, sum#39] -(83) Exchange -Input [2]: [i_item_id#46, sum#49] -Arguments: hashpartitioning(i_item_id#46, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(57) Exchange +Input [2]: [i_item_id#38, sum#39] +Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(84) HashAggregate [codegen id : 6] -Input [2]: [i_item_id#46, sum#49] -Keys [1]: [i_item_id#46] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#37))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#37))#50] -Results [2]: [i_item_id#46, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#37))#50,17,2) AS total_sales#51] +(58) HashAggregate [codegen id : 6] +Input [2]: [i_item_id#38, sum#39] +Keys [1]: [i_item_id#38] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#32))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#32))#40] +Results [2]: [i_item_id#38, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#32))#40,17,2) AS total_sales#41] -(85) Union +(59) Union -(86) HashAggregate [codegen id : 7] +(60) HashAggregate [codegen id : 7] Input [2]: [i_item_id#12, total_sales#17] Keys [1]: [i_item_id#12] Functions [1]: [partial_sum(total_sales#17)] -Aggregate Attributes [2]: [sum#52, isEmpty#53] -Results [3]: [i_item_id#12, sum#54, isEmpty#55] +Aggregate Attributes [2]: [sum#42, isEmpty#43] +Results [3]: [i_item_id#12, sum#44, isEmpty#45] -(87) Exchange -Input [3]: [i_item_id#12, sum#54, isEmpty#55] +(61) Exchange +Input [3]: [i_item_id#12, sum#44, isEmpty#45] Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(88) HashAggregate [codegen id : 8] -Input [3]: [i_item_id#12, sum#54, isEmpty#55] +(62) HashAggregate [codegen id : 8] +Input [3]: [i_item_id#12, sum#44, isEmpty#45] Keys [1]: [i_item_id#12] Functions [1]: [sum(total_sales#17)] -Aggregate Attributes [1]: [sum(total_sales#17)#56] -Results [2]: [i_item_id#12, sum(total_sales#17)#56 AS total_sales#57] +Aggregate Attributes [1]: [sum(total_sales#17)#46] +Results [2]: [i_item_id#12, sum(total_sales#17)#46 AS total_sales#47] -(89) TakeOrderedAndProject -Input [2]: [i_item_id#12, total_sales#57] -Arguments: 100, [total_sales#57 ASC NULLS FIRST], [i_item_id#12, total_sales#57] +(63) TakeOrderedAndProject +Input [2]: [i_item_id#12, total_sales#47] +Arguments: 100, [total_sales#47 ASC NULLS FIRST], [i_item_id#12, total_sales#47] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (94) -+- * ColumnarToRow (93) - +- CometProject (92) - +- CometFilter (91) - +- CometScan parquet spark_catalog.default.date_dim (90) +BroadcastExchange (68) ++- * ColumnarToRow (67) + +- CometProject (66) + +- CometFilter (65) + +- CometScan parquet spark_catalog.default.date_dim (64) -(90) Scan parquet spark_catalog.default.date_dim +(64) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#6, d_year#7, d_moy#8] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(91) CometFilter +(65) CometFilter Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 2001)) AND (d_moy#8 = 2)) AND isnotnull(d_date_sk#6)) -(92) CometProject +(66) CometProject Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(93) ColumnarToRow [codegen id : 1] +(67) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(94) BroadcastExchange +(68) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt index 8b081b9f5..7fdead831 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt @@ -64,26 +64,13 @@ TakeOrderedAndProject [total_sales,i_item_id] CometFilter [cs_bill_addr_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #9 - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #10 - CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #11 - CometBroadcastHashJoin [i_item_id,i_item_id] - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #12 - CometProject [i_item_id] - CometFilter [i_color] - CometScan parquet spark_catalog.default.item [i_item_id,i_color] + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #6 WholeStageCodegen (6) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_item_id] #13 + Exchange [i_item_id] #9 WholeStageCodegen (5) ColumnarToRow InputAdapter @@ -97,19 +84,6 @@ TakeOrderedAndProject [total_sales,i_item_id] CometFilter [ws_bill_addr_sk,ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #14 - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #15 - CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #16 - CometBroadcastHashJoin [i_item_id,i_item_id] - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #17 - CometProject [i_item_id] - CometFilter [i_color] - CometScan parquet spark_catalog.default.item [i_item_id,i_color] + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt index e84042800..fa05f2d95 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (85) -+- * Project (84) - +- * BroadcastHashJoin Inner BuildRight (83) - :- * Project (56) - : +- * BroadcastHashJoin Inner BuildRight (55) +TakeOrderedAndProject (45) ++- * Project (44) + +- * BroadcastHashJoin Inner BuildRight (43) + :- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) : :- * Project (28) : : +- * Filter (27) : : +- Window (26) @@ -32,58 +32,18 @@ TakeOrderedAndProject (85) : : +- CometBroadcastExchange (15) : : +- CometFilter (14) : : +- CometScan parquet spark_catalog.default.call_center (13) - : +- BroadcastExchange (54) - : +- * Project (53) - : +- Window (52) - : +- * Sort (51) - : +- Exchange (50) - : +- * HashAggregate (49) - : +- Exchange (48) - : +- * ColumnarToRow (47) - : +- CometHashAggregate (46) - : +- CometProject (45) - : +- CometBroadcastHashJoin (44) - : :- CometProject (40) - : : +- CometBroadcastHashJoin (39) - : : :- CometProject (35) - : : : +- CometBroadcastHashJoin (34) - : : : :- CometFilter (30) - : : : : +- CometScan parquet spark_catalog.default.item (29) - : : : +- CometBroadcastExchange (33) - : : : +- CometFilter (32) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (31) - : : +- CometBroadcastExchange (38) - : : +- CometFilter (37) - : : +- CometScan parquet spark_catalog.default.date_dim (36) - : +- CometBroadcastExchange (43) - : +- CometFilter (42) - : +- CometScan parquet spark_catalog.default.call_center (41) - +- BroadcastExchange (82) - +- * Project (81) - +- Window (80) - +- * Sort (79) - +- Exchange (78) - +- * HashAggregate (77) - +- Exchange (76) - +- * ColumnarToRow (75) - +- CometHashAggregate (74) - +- CometProject (73) - +- CometBroadcastHashJoin (72) - :- CometProject (68) - : +- CometBroadcastHashJoin (67) - : :- CometProject (63) - : : +- CometBroadcastHashJoin (62) - : : :- CometFilter (58) - : : : +- CometScan parquet spark_catalog.default.item (57) - : : +- CometBroadcastExchange (61) - : : +- CometFilter (60) - : : +- CometScan parquet spark_catalog.default.catalog_sales (59) - : +- CometBroadcastExchange (66) - : +- CometFilter (65) - : +- CometScan parquet spark_catalog.default.date_dim (64) - +- CometBroadcastExchange (71) - +- CometFilter (70) - +- CometScan parquet spark_catalog.default.call_center (69) + : +- BroadcastExchange (35) + : +- * Project (34) + : +- Window (33) + : +- * Sort (32) + : +- Exchange (31) + : +- * HashAggregate (30) + : +- ReusedExchange (29) + +- BroadcastExchange (42) + +- * Project (41) + +- Window (40) + +- * Sort (39) + +- ReusedExchange (38) (1) Scan parquet spark_catalog.default.item @@ -217,305 +177,104 @@ Condition : ((isnotnull(avg_monthly_sales#19) AND (avg_monthly_sales#19 > 0.0000 Output [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18] Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17, rn#18, avg_monthly_sales#19] -(29) Scan parquet spark_catalog.default.item -Output [3]: [i_item_sk#20, i_brand#21, i_category#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_category), IsNotNull(i_brand)] -ReadSchema: struct +(29) ReusedExchange [Reuses operator id: 20] +Output [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum#25] -(30) CometFilter -Input [3]: [i_item_sk#20, i_brand#21, i_category#22] -Condition : ((isnotnull(i_item_sk#20) AND isnotnull(i_category#22)) AND isnotnull(i_brand#21)) +(30) HashAggregate [codegen id : 6] +Input [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum#25] +Keys [5]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24] +Functions [1]: [sum(UnscaledValue(cs_sales_price#26))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#26))#15] +Results [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, MakeDecimal(sum(UnscaledValue(cs_sales_price#26))#15,17,2) AS sum_sales#16] -(31) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#23, cs_item_sk#24, cs_sales_price#25, cs_sold_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#26), dynamicpruningexpression(cs_sold_date_sk#26 IN dynamicpruning#27)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_call_center_sk)] -ReadSchema: struct +(31) Exchange +Input [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16] +Arguments: hashpartitioning(i_category#20, i_brand#21, cc_name#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(32) CometFilter -Input [4]: [cs_call_center_sk#23, cs_item_sk#24, cs_sales_price#25, cs_sold_date_sk#26] -Condition : (isnotnull(cs_item_sk#24) AND isnotnull(cs_call_center_sk#23)) +(32) Sort [codegen id : 7] +Input [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16] +Arguments: [i_category#20 ASC NULLS FIRST, i_brand#21 ASC NULLS FIRST, cc_name#22 ASC NULLS FIRST, d_year#23 ASC NULLS FIRST, d_moy#24 ASC NULLS FIRST], false, 0 -(33) CometBroadcastExchange -Input [4]: [cs_call_center_sk#23, cs_item_sk#24, cs_sales_price#25, cs_sold_date_sk#26] -Arguments: [cs_call_center_sk#23, cs_item_sk#24, cs_sales_price#25, cs_sold_date_sk#26] +(33) Window +Input [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16] +Arguments: [rank(d_year#23, d_moy#24) windowspecdefinition(i_category#20, i_brand#21, cc_name#22, d_year#23 ASC NULLS FIRST, d_moy#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#27], [i_category#20, i_brand#21, cc_name#22], [d_year#23 ASC NULLS FIRST, d_moy#24 ASC NULLS FIRST] -(34) CometBroadcastHashJoin -Left output [3]: [i_item_sk#20, i_brand#21, i_category#22] -Right output [4]: [cs_call_center_sk#23, cs_item_sk#24, cs_sales_price#25, cs_sold_date_sk#26] -Arguments: [i_item_sk#20], [cs_item_sk#24], Inner +(34) Project [codegen id : 8] +Output [5]: [i_category#20, i_brand#21, cc_name#22, sum_sales#16 AS sum_sales#28, rn#27] +Input [7]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16, rn#27] -(35) CometProject -Input [7]: [i_item_sk#20, i_brand#21, i_category#22, cs_call_center_sk#23, cs_item_sk#24, cs_sales_price#25, cs_sold_date_sk#26] -Arguments: [i_brand#21, i_category#22, cs_call_center_sk#23, cs_sales_price#25, cs_sold_date_sk#26], [i_brand#21, i_category#22, cs_call_center_sk#23, cs_sales_price#25, cs_sold_date_sk#26] +(35) BroadcastExchange +Input [5]: [i_category#20, i_brand#21, cc_name#22, sum_sales#28, rn#27] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] -(36) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#28, d_year#29, d_moy#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(37) CometFilter -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Condition : ((((d_year#29 = 1999) OR ((d_year#29 = 1998) AND (d_moy#30 = 12))) OR ((d_year#29 = 2000) AND (d_moy#30 = 1))) AND isnotnull(d_date_sk#28)) - -(38) CometBroadcastExchange -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Arguments: [d_date_sk#28, d_year#29, d_moy#30] - -(39) CometBroadcastHashJoin -Left output [5]: [i_brand#21, i_category#22, cs_call_center_sk#23, cs_sales_price#25, cs_sold_date_sk#26] -Right output [3]: [d_date_sk#28, d_year#29, d_moy#30] -Arguments: [cs_sold_date_sk#26], [d_date_sk#28], Inner - -(40) CometProject -Input [8]: [i_brand#21, i_category#22, cs_call_center_sk#23, cs_sales_price#25, cs_sold_date_sk#26, d_date_sk#28, d_year#29, d_moy#30] -Arguments: [i_brand#21, i_category#22, cs_call_center_sk#23, cs_sales_price#25, d_year#29, d_moy#30], [i_brand#21, i_category#22, cs_call_center_sk#23, cs_sales_price#25, d_year#29, d_moy#30] - -(41) Scan parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#31, cc_name#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk), IsNotNull(cc_name)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [cc_call_center_sk#31, cc_name#32] -Condition : (isnotnull(cc_call_center_sk#31) AND isnotnull(cc_name#32)) - -(43) CometBroadcastExchange -Input [2]: [cc_call_center_sk#31, cc_name#32] -Arguments: [cc_call_center_sk#31, cc_name#32] - -(44) CometBroadcastHashJoin -Left output [6]: [i_brand#21, i_category#22, cs_call_center_sk#23, cs_sales_price#25, d_year#29, d_moy#30] -Right output [2]: [cc_call_center_sk#31, cc_name#32] -Arguments: [cs_call_center_sk#23], [cc_call_center_sk#31], Inner - -(45) CometProject -Input [8]: [i_brand#21, i_category#22, cs_call_center_sk#23, cs_sales_price#25, d_year#29, d_moy#30, cc_call_center_sk#31, cc_name#32] -Arguments: [i_brand#21, i_category#22, cs_sales_price#25, d_year#29, d_moy#30, cc_name#32], [i_brand#21, i_category#22, cs_sales_price#25, d_year#29, d_moy#30, cc_name#32] - -(46) CometHashAggregate -Input [6]: [i_brand#21, i_category#22, cs_sales_price#25, d_year#29, d_moy#30, cc_name#32] -Keys [5]: [i_category#22, i_brand#21, cc_name#32, d_year#29, d_moy#30] -Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#25))] - -(47) ColumnarToRow [codegen id : 5] -Input [6]: [i_category#22, i_brand#21, cc_name#32, d_year#29, d_moy#30, sum#33] - -(48) Exchange -Input [6]: [i_category#22, i_brand#21, cc_name#32, d_year#29, d_moy#30, sum#33] -Arguments: hashpartitioning(i_category#22, i_brand#21, cc_name#32, d_year#29, d_moy#30, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(49) HashAggregate [codegen id : 6] -Input [6]: [i_category#22, i_brand#21, cc_name#32, d_year#29, d_moy#30, sum#33] -Keys [5]: [i_category#22, i_brand#21, cc_name#32, d_year#29, d_moy#30] -Functions [1]: [sum(UnscaledValue(cs_sales_price#25))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#25))#15] -Results [6]: [i_category#22, i_brand#21, cc_name#32, d_year#29, d_moy#30, MakeDecimal(sum(UnscaledValue(cs_sales_price#25))#15,17,2) AS sum_sales#16] - -(50) Exchange -Input [6]: [i_category#22, i_brand#21, cc_name#32, d_year#29, d_moy#30, sum_sales#16] -Arguments: hashpartitioning(i_category#22, i_brand#21, cc_name#32, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(51) Sort [codegen id : 7] -Input [6]: [i_category#22, i_brand#21, cc_name#32, d_year#29, d_moy#30, sum_sales#16] -Arguments: [i_category#22 ASC NULLS FIRST, i_brand#21 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST], false, 0 - -(52) Window -Input [6]: [i_category#22, i_brand#21, cc_name#32, d_year#29, d_moy#30, sum_sales#16] -Arguments: [rank(d_year#29, d_moy#30) windowspecdefinition(i_category#22, i_brand#21, cc_name#32, d_year#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#34], [i_category#22, i_brand#21, cc_name#32], [d_year#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST] - -(53) Project [codegen id : 8] -Output [5]: [i_category#22, i_brand#21, cc_name#32, sum_sales#16 AS sum_sales#35, rn#34] -Input [7]: [i_category#22, i_brand#21, cc_name#32, d_year#29, d_moy#30, sum_sales#16, rn#34] - -(54) BroadcastExchange -Input [5]: [i_category#22, i_brand#21, cc_name#32, sum_sales#35, rn#34] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=5] - -(55) BroadcastHashJoin [codegen id : 13] +(36) BroadcastHashJoin [codegen id : 13] Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#18] -Right keys [4]: [i_category#22, i_brand#21, cc_name#32, (rn#34 + 1)] +Right keys [4]: [i_category#20, i_brand#21, cc_name#22, (rn#27 + 1)] Join type: Inner Join condition: None -(56) Project [codegen id : 13] -Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, sum_sales#35] -Input [13]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, i_category#22, i_brand#21, cc_name#32, sum_sales#35, rn#34] - -(57) Scan parquet spark_catalog.default.item -Output [3]: [i_item_sk#36, i_brand#37, i_category#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_category), IsNotNull(i_brand)] -ReadSchema: struct - -(58) CometFilter -Input [3]: [i_item_sk#36, i_brand#37, i_category#38] -Condition : ((isnotnull(i_item_sk#36) AND isnotnull(i_category#38)) AND isnotnull(i_brand#37)) +(37) Project [codegen id : 13] +Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, sum_sales#28] +Input [13]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, i_category#20, i_brand#21, cc_name#22, sum_sales#28, rn#27] -(59) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#39, cs_item_sk#40, cs_sales_price#41, cs_sold_date_sk#42] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#42), dynamicpruningexpression(cs_sold_date_sk#42 IN dynamicpruning#43)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_call_center_sk)] -ReadSchema: struct +(38) ReusedExchange [Reuses operator id: 31] +Output [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#16] -(60) CometFilter -Input [4]: [cs_call_center_sk#39, cs_item_sk#40, cs_sales_price#41, cs_sold_date_sk#42] -Condition : (isnotnull(cs_item_sk#40) AND isnotnull(cs_call_center_sk#39)) +(39) Sort [codegen id : 11] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#16] +Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST], false, 0 -(61) CometBroadcastExchange -Input [4]: [cs_call_center_sk#39, cs_item_sk#40, cs_sales_price#41, cs_sold_date_sk#42] -Arguments: [cs_call_center_sk#39, cs_item_sk#40, cs_sales_price#41, cs_sold_date_sk#42] +(40) Window +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#16] +Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#29, i_brand#30, cc_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#34], [i_category#29, i_brand#30, cc_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] -(62) CometBroadcastHashJoin -Left output [3]: [i_item_sk#36, i_brand#37, i_category#38] -Right output [4]: [cs_call_center_sk#39, cs_item_sk#40, cs_sales_price#41, cs_sold_date_sk#42] -Arguments: [i_item_sk#36], [cs_item_sk#40], Inner +(41) Project [codegen id : 12] +Output [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#16 AS sum_sales#35, rn#34] +Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#16, rn#34] -(63) CometProject -Input [7]: [i_item_sk#36, i_brand#37, i_category#38, cs_call_center_sk#39, cs_item_sk#40, cs_sales_price#41, cs_sold_date_sk#42] -Arguments: [i_brand#37, i_category#38, cs_call_center_sk#39, cs_sales_price#41, cs_sold_date_sk#42], [i_brand#37, i_category#38, cs_call_center_sk#39, cs_sales_price#41, cs_sold_date_sk#42] +(42) BroadcastExchange +Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#35, rn#34] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] -(64) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#44, d_year#45, d_moy#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(65) CometFilter -Input [3]: [d_date_sk#44, d_year#45, d_moy#46] -Condition : ((((d_year#45 = 1999) OR ((d_year#45 = 1998) AND (d_moy#46 = 12))) OR ((d_year#45 = 2000) AND (d_moy#46 = 1))) AND isnotnull(d_date_sk#44)) - -(66) CometBroadcastExchange -Input [3]: [d_date_sk#44, d_year#45, d_moy#46] -Arguments: [d_date_sk#44, d_year#45, d_moy#46] - -(67) CometBroadcastHashJoin -Left output [5]: [i_brand#37, i_category#38, cs_call_center_sk#39, cs_sales_price#41, cs_sold_date_sk#42] -Right output [3]: [d_date_sk#44, d_year#45, d_moy#46] -Arguments: [cs_sold_date_sk#42], [d_date_sk#44], Inner - -(68) CometProject -Input [8]: [i_brand#37, i_category#38, cs_call_center_sk#39, cs_sales_price#41, cs_sold_date_sk#42, d_date_sk#44, d_year#45, d_moy#46] -Arguments: [i_brand#37, i_category#38, cs_call_center_sk#39, cs_sales_price#41, d_year#45, d_moy#46], [i_brand#37, i_category#38, cs_call_center_sk#39, cs_sales_price#41, d_year#45, d_moy#46] - -(69) Scan parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#47, cc_name#48] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk), IsNotNull(cc_name)] -ReadSchema: struct - -(70) CometFilter -Input [2]: [cc_call_center_sk#47, cc_name#48] -Condition : (isnotnull(cc_call_center_sk#47) AND isnotnull(cc_name#48)) - -(71) CometBroadcastExchange -Input [2]: [cc_call_center_sk#47, cc_name#48] -Arguments: [cc_call_center_sk#47, cc_name#48] - -(72) CometBroadcastHashJoin -Left output [6]: [i_brand#37, i_category#38, cs_call_center_sk#39, cs_sales_price#41, d_year#45, d_moy#46] -Right output [2]: [cc_call_center_sk#47, cc_name#48] -Arguments: [cs_call_center_sk#39], [cc_call_center_sk#47], Inner - -(73) CometProject -Input [8]: [i_brand#37, i_category#38, cs_call_center_sk#39, cs_sales_price#41, d_year#45, d_moy#46, cc_call_center_sk#47, cc_name#48] -Arguments: [i_brand#37, i_category#38, cs_sales_price#41, d_year#45, d_moy#46, cc_name#48], [i_brand#37, i_category#38, cs_sales_price#41, d_year#45, d_moy#46, cc_name#48] - -(74) CometHashAggregate -Input [6]: [i_brand#37, i_category#38, cs_sales_price#41, d_year#45, d_moy#46, cc_name#48] -Keys [5]: [i_category#38, i_brand#37, cc_name#48, d_year#45, d_moy#46] -Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#41))] - -(75) ColumnarToRow [codegen id : 9] -Input [6]: [i_category#38, i_brand#37, cc_name#48, d_year#45, d_moy#46, sum#49] - -(76) Exchange -Input [6]: [i_category#38, i_brand#37, cc_name#48, d_year#45, d_moy#46, sum#49] -Arguments: hashpartitioning(i_category#38, i_brand#37, cc_name#48, d_year#45, d_moy#46, 5), ENSURE_REQUIREMENTS, [plan_id=6] - -(77) HashAggregate [codegen id : 10] -Input [6]: [i_category#38, i_brand#37, cc_name#48, d_year#45, d_moy#46, sum#49] -Keys [5]: [i_category#38, i_brand#37, cc_name#48, d_year#45, d_moy#46] -Functions [1]: [sum(UnscaledValue(cs_sales_price#41))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#41))#15] -Results [6]: [i_category#38, i_brand#37, cc_name#48, d_year#45, d_moy#46, MakeDecimal(sum(UnscaledValue(cs_sales_price#41))#15,17,2) AS sum_sales#16] - -(78) Exchange -Input [6]: [i_category#38, i_brand#37, cc_name#48, d_year#45, d_moy#46, sum_sales#16] -Arguments: hashpartitioning(i_category#38, i_brand#37, cc_name#48, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(79) Sort [codegen id : 11] -Input [6]: [i_category#38, i_brand#37, cc_name#48, d_year#45, d_moy#46, sum_sales#16] -Arguments: [i_category#38 ASC NULLS FIRST, i_brand#37 ASC NULLS FIRST, cc_name#48 ASC NULLS FIRST, d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST], false, 0 - -(80) Window -Input [6]: [i_category#38, i_brand#37, cc_name#48, d_year#45, d_moy#46, sum_sales#16] -Arguments: [rank(d_year#45, d_moy#46) windowspecdefinition(i_category#38, i_brand#37, cc_name#48, d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#50], [i_category#38, i_brand#37, cc_name#48], [d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST] - -(81) Project [codegen id : 12] -Output [5]: [i_category#38, i_brand#37, cc_name#48, sum_sales#16 AS sum_sales#51, rn#50] -Input [7]: [i_category#38, i_brand#37, cc_name#48, d_year#45, d_moy#46, sum_sales#16, rn#50] - -(82) BroadcastExchange -Input [5]: [i_category#38, i_brand#37, cc_name#48, sum_sales#51, rn#50] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=8] - -(83) BroadcastHashJoin [codegen id : 13] +(43) BroadcastHashJoin [codegen id : 13] Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#18] -Right keys [4]: [i_category#38, i_brand#37, cc_name#48, (rn#50 - 1)] +Right keys [4]: [i_category#29, i_brand#30, cc_name#31, (rn#34 - 1)] Join type: Inner Join condition: None -(84) Project [codegen id : 13] -Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#19, sum_sales#16, sum_sales#35 AS psum#52, sum_sales#51 AS nsum#53] -Input [14]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, sum_sales#35, i_category#38, i_brand#37, cc_name#48, sum_sales#51, rn#50] +(44) Project [codegen id : 13] +Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#19, sum_sales#16, sum_sales#28 AS psum#36, sum_sales#35 AS nsum#37] +Input [14]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, sum_sales#28, i_category#29, i_brand#30, cc_name#31, sum_sales#35, rn#34] -(85) TakeOrderedAndProject -Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#19, sum_sales#16, psum#52, nsum#53] -Arguments: 100, [(sum_sales#16 - avg_monthly_sales#19) ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#19, sum_sales#16, psum#52, nsum#53] +(45) TakeOrderedAndProject +Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#19, sum_sales#16, psum#36, nsum#37] +Arguments: 100, [(sum_sales#16 - avg_monthly_sales#19) ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#19, sum_sales#16, psum#36, nsum#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = cs_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (89) -+- * ColumnarToRow (88) - +- CometFilter (87) - +- CometScan parquet spark_catalog.default.date_dim (86) +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) -(86) Scan parquet spark_catalog.default.date_dim +(46) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#9, d_year#10, d_moy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(87) CometFilter +(47) CometFilter Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) -(88) ColumnarToRow [codegen id : 1] +(48) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -(89) BroadcastExchange +(49) BroadcastExchange Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 31 Hosting Expression = cs_sold_date_sk#26 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 59 Hosting Expression = cs_sold_date_sk#42 IN dynamicpruning#8 +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/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt index 01a95ed00..c630cad48 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt @@ -61,31 +61,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ WholeStageCodegen (6) HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] InputAdapter - Exchange [i_category,i_brand,cc_name,d_year,d_moy] #9 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,cs_item_sk] - CometFilter [i_item_sk,i_category,i_brand] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #10 - CometFilter [cs_item_sk,cs_call_center_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #11 - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #12 - CometFilter [cc_call_center_sk,cc_name] - CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 InputAdapter - BroadcastExchange #13 + BroadcastExchange #9 WholeStageCodegen (12) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter @@ -93,30 +71,4 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ WholeStageCodegen (11) Sort [i_category,i_brand,cc_name,d_year,d_moy] InputAdapter - Exchange [i_category,i_brand,cc_name] #14 - WholeStageCodegen (10) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] - InputAdapter - Exchange [i_category,i_brand,cc_name,d_year,d_moy] #15 - WholeStageCodegen (9) - ColumnarToRow - InputAdapter - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,cs_item_sk] - CometFilter [i_item_sk,i_category,i_brand] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #16 - CometFilter [cs_item_sk,cs_call_center_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #17 - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #18 - CometFilter [cc_call_center_sk,cc_name] - CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt index f34c9996f..e23beca6f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (67) -+- * Project (66) - +- * BroadcastHashJoin Inner BuildRight (65) - :- * Project (43) - : +- * BroadcastHashJoin Inner BuildRight (42) +TakeOrderedAndProject (53) ++- * Project (52) + +- * BroadcastHashJoin Inner BuildRight (51) + :- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) : :- * Filter (20) : : +- * HashAggregate (19) : : +- Exchange (18) @@ -24,48 +24,34 @@ TakeOrderedAndProject (67) : : :- CometFilter (9) : : : +- CometScan parquet spark_catalog.default.date_dim (8) : : +- ReusedExchange (10) - : +- BroadcastExchange (41) - : +- * Filter (40) - : +- * HashAggregate (39) - : +- Exchange (38) - : +- * ColumnarToRow (37) - : +- CometHashAggregate (36) - : +- CometProject (35) - : +- CometBroadcastHashJoin (34) - : :- CometProject (27) - : : +- CometBroadcastHashJoin (26) + : +- BroadcastExchange (34) + : +- * Filter (33) + : +- * HashAggregate (32) + : +- Exchange (31) + : +- * ColumnarToRow (30) + : +- CometHashAggregate (29) + : +- CometProject (28) + : +- CometBroadcastHashJoin (27) + : :- CometProject (25) + : : +- CometBroadcastHashJoin (24) : : :- CometFilter (22) : : : +- CometScan parquet spark_catalog.default.catalog_sales (21) - : : +- CometBroadcastExchange (25) - : : +- CometFilter (24) - : : +- CometScan parquet spark_catalog.default.item (23) - : +- CometBroadcastExchange (33) - : +- CometProject (32) - : +- CometBroadcastHashJoin (31) - : :- CometFilter (29) - : : +- CometScan parquet spark_catalog.default.date_dim (28) - : +- ReusedExchange (30) - +- BroadcastExchange (64) - +- * Filter (63) - +- * HashAggregate (62) - +- Exchange (61) - +- * ColumnarToRow (60) - +- CometHashAggregate (59) - +- CometProject (58) - +- CometBroadcastHashJoin (57) - :- CometProject (50) - : +- CometBroadcastHashJoin (49) - : :- CometFilter (45) - : : +- CometScan parquet spark_catalog.default.web_sales (44) - : +- CometBroadcastExchange (48) - : +- CometFilter (47) - : +- CometScan parquet spark_catalog.default.item (46) - +- CometBroadcastExchange (56) - +- CometProject (55) - +- CometBroadcastHashJoin (54) - :- CometFilter (52) - : +- CometScan parquet spark_catalog.default.date_dim (51) - +- ReusedExchange (53) + : : +- ReusedExchange (23) + : +- ReusedExchange (26) + +- BroadcastExchange (50) + +- * Filter (49) + +- * HashAggregate (48) + +- Exchange (47) + +- * ColumnarToRow (46) + +- CometHashAggregate (45) + +- CometProject (44) + +- CometBroadcastHashJoin (43) + :- CometProject (41) + : +- CometBroadcastHashJoin (40) + : :- CometFilter (38) + : : +- CometScan parquet spark_catalog.default.web_sales (37) + : +- ReusedExchange (39) + +- ReusedExchange (42) (1) Scan parquet spark_catalog.default.store_sales @@ -115,7 +101,7 @@ ReadSchema: struct Input [2]: [d_date_sk#7, d_date#8] Condition : isnotnull(d_date_sk#7) -(10) ReusedExchange [Reuses operator id: 73] +(10) ReusedExchange [Reuses operator id: 59] Output [1]: [d_date#9] (11) CometBroadcastHashJoin @@ -175,422 +161,232 @@ ReadSchema: struct Input [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] Condition : isnotnull(cs_item_sk#14) -(23) Scan parquet spark_catalog.default.item +(23) ReusedExchange [Reuses operator id: 5] Output [2]: [i_item_sk#18, i_item_id#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] -ReadSchema: struct - -(24) CometFilter -Input [2]: [i_item_sk#18, i_item_id#19] -Condition : (isnotnull(i_item_sk#18) AND isnotnull(i_item_id#19)) -(25) CometBroadcastExchange -Input [2]: [i_item_sk#18, i_item_id#19] -Arguments: [i_item_sk#18, i_item_id#19] - -(26) CometBroadcastHashJoin +(24) CometBroadcastHashJoin Left output [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] Right output [2]: [i_item_sk#18, i_item_id#19] Arguments: [cs_item_sk#14], [i_item_sk#18], Inner -(27) CometProject +(25) CometProject Input [5]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_sk#18, i_item_id#19] Arguments: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19], [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19] -(28) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_date#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(29) CometFilter -Input [2]: [d_date_sk#20, d_date#21] -Condition : isnotnull(d_date_sk#20) - -(30) ReusedExchange [Reuses operator id: 87] -Output [1]: [d_date#22] - -(31) CometBroadcastHashJoin -Left output [2]: [d_date_sk#20, d_date#21] -Right output [1]: [d_date#22] -Arguments: [d_date#21], [d_date#22], LeftSemi +(26) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_date_sk#20] -(32) CometProject -Input [2]: [d_date_sk#20, d_date#21] -Arguments: [d_date_sk#20], [d_date_sk#20] - -(33) CometBroadcastExchange -Input [1]: [d_date_sk#20] -Arguments: [d_date_sk#20] - -(34) CometBroadcastHashJoin +(27) CometBroadcastHashJoin Left output [3]: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19] Right output [1]: [d_date_sk#20] Arguments: [cs_sold_date_sk#16], [d_date_sk#20], Inner -(35) CometProject +(28) CometProject Input [4]: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19, d_date_sk#20] Arguments: [cs_ext_sales_price#15, i_item_id#19], [cs_ext_sales_price#15, i_item_id#19] -(36) CometHashAggregate +(29) CometHashAggregate Input [2]: [cs_ext_sales_price#15, i_item_id#19] Keys [1]: [i_item_id#19] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#15))] -(37) ColumnarToRow [codegen id : 2] -Input [2]: [i_item_id#19, sum#23] +(30) ColumnarToRow [codegen id : 2] +Input [2]: [i_item_id#19, sum#21] -(38) Exchange -Input [2]: [i_item_id#19, sum#23] +(31) Exchange +Input [2]: [i_item_id#19, sum#21] Arguments: hashpartitioning(i_item_id#19, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(39) HashAggregate [codegen id : 3] -Input [2]: [i_item_id#19, sum#23] +(32) HashAggregate [codegen id : 3] +Input [2]: [i_item_id#19, sum#21] Keys [1]: [i_item_id#19] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#15))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#15))#24] -Results [2]: [i_item_id#19 AS item_id#25, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#15))#24,17,2) AS cs_item_rev#26] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#15))#22] +Results [2]: [i_item_id#19 AS item_id#23, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#15))#22,17,2) AS cs_item_rev#24] -(40) Filter [codegen id : 3] -Input [2]: [item_id#25, cs_item_rev#26] -Condition : isnotnull(cs_item_rev#26) +(33) Filter [codegen id : 3] +Input [2]: [item_id#23, cs_item_rev#24] +Condition : isnotnull(cs_item_rev#24) -(41) BroadcastExchange -Input [2]: [item_id#25, cs_item_rev#26] +(34) BroadcastExchange +Input [2]: [item_id#23, cs_item_rev#24] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] -(42) BroadcastHashJoin [codegen id : 6] +(35) BroadcastHashJoin [codegen id : 6] Left keys [1]: [item_id#12] -Right keys [1]: [item_id#25] +Right keys [1]: [item_id#23] Join type: Inner -Join condition: ((((cast(ss_item_rev#13 as decimal(19,3)) >= (0.9 * cs_item_rev#26)) AND (cast(ss_item_rev#13 as decimal(20,3)) <= (1.1 * cs_item_rev#26))) AND (cast(cs_item_rev#26 as decimal(19,3)) >= (0.9 * ss_item_rev#13))) AND (cast(cs_item_rev#26 as decimal(20,3)) <= (1.1 * ss_item_rev#13))) +Join condition: ((((cast(ss_item_rev#13 as decimal(19,3)) >= (0.9 * cs_item_rev#24)) AND (cast(ss_item_rev#13 as decimal(20,3)) <= (1.1 * cs_item_rev#24))) AND (cast(cs_item_rev#24 as decimal(19,3)) >= (0.9 * ss_item_rev#13))) AND (cast(cs_item_rev#24 as decimal(20,3)) <= (1.1 * ss_item_rev#13))) -(43) Project [codegen id : 6] -Output [3]: [item_id#12, ss_item_rev#13, cs_item_rev#26] -Input [4]: [item_id#12, ss_item_rev#13, item_id#25, cs_item_rev#26] +(36) Project [codegen id : 6] +Output [3]: [item_id#12, ss_item_rev#13, cs_item_rev#24] +Input [4]: [item_id#12, ss_item_rev#13, item_id#23, cs_item_rev#24] -(44) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] +(37) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#30)] +PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#28)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(45) CometFilter -Input [3]: [ws_item_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] -Condition : isnotnull(ws_item_sk#27) +(38) CometFilter +Input [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] +Condition : isnotnull(ws_item_sk#25) -(46) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#31, i_item_id#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] -ReadSchema: struct +(39) ReusedExchange [Reuses operator id: 5] +Output [2]: [i_item_sk#29, i_item_id#30] -(47) CometFilter -Input [2]: [i_item_sk#31, i_item_id#32] -Condition : (isnotnull(i_item_sk#31) AND isnotnull(i_item_id#32)) +(40) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] +Right output [2]: [i_item_sk#29, i_item_id#30] +Arguments: [ws_item_sk#25], [i_item_sk#29], Inner -(48) CometBroadcastExchange -Input [2]: [i_item_sk#31, i_item_id#32] -Arguments: [i_item_sk#31, i_item_id#32] +(41) CometProject +Input [5]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_sk#29, i_item_id#30] +Arguments: [ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_id#30], [ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_id#30] -(49) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] -Right output [2]: [i_item_sk#31, i_item_id#32] -Arguments: [ws_item_sk#27], [i_item_sk#31], Inner +(42) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_date_sk#31] -(50) CometProject -Input [5]: [ws_item_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29, i_item_sk#31, i_item_id#32] -Arguments: [ws_ext_sales_price#28, ws_sold_date_sk#29, i_item_id#32], [ws_ext_sales_price#28, ws_sold_date_sk#29, i_item_id#32] - -(51) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_date#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct +(43) CometBroadcastHashJoin +Left output [3]: [ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_id#30] +Right output [1]: [d_date_sk#31] +Arguments: [ws_sold_date_sk#27], [d_date_sk#31], Inner -(52) CometFilter -Input [2]: [d_date_sk#33, d_date#34] -Condition : isnotnull(d_date_sk#33) +(44) CometProject +Input [4]: [ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_id#30, d_date_sk#31] +Arguments: [ws_ext_sales_price#26, i_item_id#30], [ws_ext_sales_price#26, i_item_id#30] -(53) ReusedExchange [Reuses operator id: 97] -Output [1]: [d_date#35] +(45) CometHashAggregate +Input [2]: [ws_ext_sales_price#26, i_item_id#30] +Keys [1]: [i_item_id#30] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#26))] -(54) CometBroadcastHashJoin -Left output [2]: [d_date_sk#33, d_date#34] -Right output [1]: [d_date#35] -Arguments: [d_date#34], [d_date#35], LeftSemi +(46) ColumnarToRow [codegen id : 4] +Input [2]: [i_item_id#30, sum#32] -(55) CometProject -Input [2]: [d_date_sk#33, d_date#34] -Arguments: [d_date_sk#33], [d_date_sk#33] +(47) Exchange +Input [2]: [i_item_id#30, sum#32] +Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(56) CometBroadcastExchange -Input [1]: [d_date_sk#33] -Arguments: [d_date_sk#33] +(48) HashAggregate [codegen id : 5] +Input [2]: [i_item_id#30, sum#32] +Keys [1]: [i_item_id#30] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#26))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#26))#33] +Results [2]: [i_item_id#30 AS item_id#34, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#26))#33,17,2) AS ws_item_rev#35] -(57) CometBroadcastHashJoin -Left output [3]: [ws_ext_sales_price#28, ws_sold_date_sk#29, i_item_id#32] -Right output [1]: [d_date_sk#33] -Arguments: [ws_sold_date_sk#29], [d_date_sk#33], Inner +(49) Filter [codegen id : 5] +Input [2]: [item_id#34, ws_item_rev#35] +Condition : isnotnull(ws_item_rev#35) -(58) CometProject -Input [4]: [ws_ext_sales_price#28, ws_sold_date_sk#29, i_item_id#32, d_date_sk#33] -Arguments: [ws_ext_sales_price#28, i_item_id#32], [ws_ext_sales_price#28, i_item_id#32] - -(59) CometHashAggregate -Input [2]: [ws_ext_sales_price#28, i_item_id#32] -Keys [1]: [i_item_id#32] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#28))] - -(60) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_id#32, sum#36] - -(61) Exchange -Input [2]: [i_item_id#32, sum#36] -Arguments: hashpartitioning(i_item_id#32, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(62) HashAggregate [codegen id : 5] -Input [2]: [i_item_id#32, sum#36] -Keys [1]: [i_item_id#32] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#28))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#28))#37] -Results [2]: [i_item_id#32 AS item_id#38, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#28))#37,17,2) AS ws_item_rev#39] - -(63) Filter [codegen id : 5] -Input [2]: [item_id#38, ws_item_rev#39] -Condition : isnotnull(ws_item_rev#39) - -(64) BroadcastExchange -Input [2]: [item_id#38, ws_item_rev#39] +(50) BroadcastExchange +Input [2]: [item_id#34, ws_item_rev#35] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(65) BroadcastHashJoin [codegen id : 6] +(51) BroadcastHashJoin [codegen id : 6] Left keys [1]: [item_id#12] -Right keys [1]: [item_id#38] +Right keys [1]: [item_id#34] Join type: Inner -Join condition: ((((((((cast(ss_item_rev#13 as decimal(19,3)) >= (0.9 * ws_item_rev#39)) AND (cast(ss_item_rev#13 as decimal(20,3)) <= (1.1 * ws_item_rev#39))) AND (cast(cs_item_rev#26 as decimal(19,3)) >= (0.9 * ws_item_rev#39))) AND (cast(cs_item_rev#26 as decimal(20,3)) <= (1.1 * ws_item_rev#39))) AND (cast(ws_item_rev#39 as decimal(19,3)) >= (0.9 * ss_item_rev#13))) AND (cast(ws_item_rev#39 as decimal(20,3)) <= (1.1 * ss_item_rev#13))) AND (cast(ws_item_rev#39 as decimal(19,3)) >= (0.9 * cs_item_rev#26))) AND (cast(ws_item_rev#39 as decimal(20,3)) <= (1.1 * cs_item_rev#26))) +Join condition: ((((((((cast(ss_item_rev#13 as decimal(19,3)) >= (0.9 * ws_item_rev#35)) AND (cast(ss_item_rev#13 as decimal(20,3)) <= (1.1 * ws_item_rev#35))) AND (cast(cs_item_rev#24 as decimal(19,3)) >= (0.9 * ws_item_rev#35))) AND (cast(cs_item_rev#24 as decimal(20,3)) <= (1.1 * ws_item_rev#35))) AND (cast(ws_item_rev#35 as decimal(19,3)) >= (0.9 * ss_item_rev#13))) AND (cast(ws_item_rev#35 as decimal(20,3)) <= (1.1 * ss_item_rev#13))) AND (cast(ws_item_rev#35 as decimal(19,3)) >= (0.9 * cs_item_rev#24))) AND (cast(ws_item_rev#35 as decimal(20,3)) <= (1.1 * cs_item_rev#24))) -(66) Project [codegen id : 6] -Output [8]: [item_id#12, ss_item_rev#13, (((ss_item_rev#13 / ((ss_item_rev#13 + cs_item_rev#26) + ws_item_rev#39)) / 3) * 100) AS ss_dev#40, cs_item_rev#26, (((cs_item_rev#26 / ((ss_item_rev#13 + cs_item_rev#26) + ws_item_rev#39)) / 3) * 100) AS cs_dev#41, ws_item_rev#39, (((ws_item_rev#39 / ((ss_item_rev#13 + cs_item_rev#26) + ws_item_rev#39)) / 3) * 100) AS ws_dev#42, (((ss_item_rev#13 + cs_item_rev#26) + ws_item_rev#39) / 3) AS average#43] -Input [5]: [item_id#12, ss_item_rev#13, cs_item_rev#26, item_id#38, ws_item_rev#39] +(52) Project [codegen id : 6] +Output [8]: [item_id#12, ss_item_rev#13, (((ss_item_rev#13 / ((ss_item_rev#13 + cs_item_rev#24) + ws_item_rev#35)) / 3) * 100) AS ss_dev#36, cs_item_rev#24, (((cs_item_rev#24 / ((ss_item_rev#13 + cs_item_rev#24) + ws_item_rev#35)) / 3) * 100) AS cs_dev#37, ws_item_rev#35, (((ws_item_rev#35 / ((ss_item_rev#13 + cs_item_rev#24) + ws_item_rev#35)) / 3) * 100) AS ws_dev#38, (((ss_item_rev#13 + cs_item_rev#24) + ws_item_rev#35) / 3) AS average#39] +Input [5]: [item_id#12, ss_item_rev#13, cs_item_rev#24, item_id#34, ws_item_rev#35] -(67) TakeOrderedAndProject -Input [8]: [item_id#12, ss_item_rev#13, ss_dev#40, cs_item_rev#26, cs_dev#41, ws_item_rev#39, ws_dev#42, average#43] -Arguments: 100, [item_id#12 ASC NULLS FIRST, ss_item_rev#13 ASC NULLS FIRST], [item_id#12, ss_item_rev#13, ss_dev#40, cs_item_rev#26, cs_dev#41, ws_item_rev#39, ws_dev#42, average#43] +(53) TakeOrderedAndProject +Input [8]: [item_id#12, ss_item_rev#13, ss_dev#36, cs_item_rev#24, cs_dev#37, ws_item_rev#35, ws_dev#38, average#39] +Arguments: 100, [item_id#12 ASC NULLS FIRST, ss_item_rev#13 ASC NULLS FIRST], [item_id#12, ss_item_rev#13, ss_dev#36, cs_item_rev#24, cs_dev#37, ws_item_rev#35, ws_dev#38, average#39] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (77) -+- * ColumnarToRow (76) - +- CometProject (75) - +- CometBroadcastHashJoin (74) - :- CometFilter (69) - : +- CometScan parquet spark_catalog.default.date_dim (68) - +- CometBroadcastExchange (73) - +- CometProject (72) - +- CometFilter (71) - +- CometScan parquet spark_catalog.default.date_dim (70) - - -(68) Scan parquet spark_catalog.default.date_dim +BroadcastExchange (63) ++- * ColumnarToRow (62) + +- CometProject (61) + +- CometBroadcastHashJoin (60) + :- CometFilter (55) + : +- CometScan parquet spark_catalog.default.date_dim (54) + +- CometBroadcastExchange (59) + +- CometProject (58) + +- CometFilter (57) + +- CometScan parquet spark_catalog.default.date_dim (56) + + +(54) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#7, d_date#8] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct -(69) CometFilter +(55) CometFilter Input [2]: [d_date_sk#7, d_date#8] Condition : isnotnull(d_date_sk#7) -(70) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date#9, d_week_seq#44] +(56) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date#9, d_week_seq#40] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct -(71) CometFilter -Input [2]: [d_date#9, d_week_seq#44] -Condition : (isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subquery#45, [id=#46])) +(57) CometFilter +Input [2]: [d_date#9, d_week_seq#40] +Condition : (isnotnull(d_week_seq#40) AND (d_week_seq#40 = Subquery scalar-subquery#41, [id=#42])) -(72) CometProject -Input [2]: [d_date#9, d_week_seq#44] +(58) CometProject +Input [2]: [d_date#9, d_week_seq#40] Arguments: [d_date#9], [d_date#9] -(73) CometBroadcastExchange +(59) CometBroadcastExchange Input [1]: [d_date#9] Arguments: [d_date#9] -(74) CometBroadcastHashJoin +(60) CometBroadcastHashJoin Left output [2]: [d_date_sk#7, d_date#8] Right output [1]: [d_date#9] Arguments: [d_date#8], [d_date#9], LeftSemi -(75) CometProject +(61) CometProject Input [2]: [d_date_sk#7, d_date#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(76) ColumnarToRow [codegen id : 1] +(62) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(77) BroadcastExchange +(63) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 71 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* ColumnarToRow (81) -+- CometProject (80) - +- CometFilter (79) - +- CometScan parquet spark_catalog.default.date_dim (78) +Subquery:2 Hosting operator id = 57 Hosting Expression = Subquery scalar-subquery#41, [id=#42] +* ColumnarToRow (67) ++- CometProject (66) + +- CometFilter (65) + +- CometScan parquet spark_catalog.default.date_dim (64) -(78) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date#47, d_week_seq#48] +(64) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date#43, d_week_seq#44] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] ReadSchema: struct -(79) CometFilter -Input [2]: [d_date#47, d_week_seq#48] -Condition : (isnotnull(d_date#47) AND (d_date#47 = 2000-01-03)) - -(80) CometProject -Input [2]: [d_date#47, d_week_seq#48] -Arguments: [d_week_seq#48], [d_week_seq#48] - -(81) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#48] - -Subquery:3 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#17 -BroadcastExchange (91) -+- * ColumnarToRow (90) - +- CometProject (89) - +- CometBroadcastHashJoin (88) - :- CometFilter (83) - : +- CometScan parquet spark_catalog.default.date_dim (82) - +- CometBroadcastExchange (87) - +- CometProject (86) - +- CometFilter (85) - +- CometScan parquet spark_catalog.default.date_dim (84) - - -(82) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_date#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(83) CometFilter -Input [2]: [d_date_sk#20, d_date#21] -Condition : isnotnull(d_date_sk#20) - -(84) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date#22, d_week_seq#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] -ReadSchema: struct - -(85) CometFilter -Input [2]: [d_date#22, d_week_seq#49] -Condition : (isnotnull(d_week_seq#49) AND (d_week_seq#49 = ReusedSubquery Subquery scalar-subquery#45, [id=#46])) - -(86) CometProject -Input [2]: [d_date#22, d_week_seq#49] -Arguments: [d_date#22], [d_date#22] - -(87) CometBroadcastExchange -Input [1]: [d_date#22] -Arguments: [d_date#22] - -(88) CometBroadcastHashJoin -Left output [2]: [d_date_sk#20, d_date#21] -Right output [1]: [d_date#22] -Arguments: [d_date#21], [d_date#22], LeftSemi - -(89) CometProject -Input [2]: [d_date_sk#20, d_date#21] -Arguments: [d_date_sk#20], [d_date_sk#20] - -(90) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#20] - -(91) BroadcastExchange -Input [1]: [d_date_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:4 Hosting operator id = 85 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] - -Subquery:5 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#30 -BroadcastExchange (101) -+- * ColumnarToRow (100) - +- CometProject (99) - +- CometBroadcastHashJoin (98) - :- CometFilter (93) - : +- CometScan parquet spark_catalog.default.date_dim (92) - +- CometBroadcastExchange (97) - +- CometProject (96) - +- CometFilter (95) - +- CometScan parquet spark_catalog.default.date_dim (94) - - -(92) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_date#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(93) CometFilter -Input [2]: [d_date_sk#33, d_date#34] -Condition : isnotnull(d_date_sk#33) - -(94) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date#35, d_week_seq#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] -ReadSchema: struct - -(95) CometFilter -Input [2]: [d_date#35, d_week_seq#50] -Condition : (isnotnull(d_week_seq#50) AND (d_week_seq#50 = ReusedSubquery Subquery scalar-subquery#45, [id=#46])) - -(96) CometProject -Input [2]: [d_date#35, d_week_seq#50] -Arguments: [d_date#35], [d_date#35] - -(97) CometBroadcastExchange -Input [1]: [d_date#35] -Arguments: [d_date#35] - -(98) CometBroadcastHashJoin -Left output [2]: [d_date_sk#33, d_date#34] -Right output [1]: [d_date#35] -Arguments: [d_date#34], [d_date#35], LeftSemi +(65) CometFilter +Input [2]: [d_date#43, d_week_seq#44] +Condition : (isnotnull(d_date#43) AND (d_date#43 = 2000-01-03)) -(99) CometProject -Input [2]: [d_date_sk#33, d_date#34] -Arguments: [d_date_sk#33], [d_date_sk#33] +(66) CometProject +Input [2]: [d_date#43, d_week_seq#44] +Arguments: [d_week_seq#44], [d_week_seq#44] -(100) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#33] +(67) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#44] -(101) BroadcastExchange -Input [1]: [d_date_sk#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Subquery:3 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#4 -Subquery:6 Hosting operator id = 95 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:4 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt index 23dc35eee..2ed2bde44 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt @@ -64,36 +64,16 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev CometBroadcastHashJoin [cs_item_sk,i_item_sk] CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #8 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #9 - CometProject [d_date] - CometFilter [d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #10 - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #11 - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date] #9 + ReusedSubquery [d_date_sk] #1 + ReusedExchange [i_item_sk,i_item_id] #4 + ReusedExchange [d_date_sk] #5 InputAdapter - BroadcastExchange #12 + BroadcastExchange #8 WholeStageCodegen (5) Filter [ws_item_rev] HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum] InputAdapter - Exchange [i_item_id] #13 + Exchange [i_item_id] #9 WholeStageCodegen (4) ColumnarToRow InputAdapter @@ -104,26 +84,6 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev CometBroadcastHashJoin [ws_item_sk,i_item_sk] CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #14 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #15 - CometProject [d_date] - CometFilter [d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #16 - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #17 - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date] #15 + ReusedSubquery [d_date_sk] #1 + ReusedExchange [i_item_sk,i_item_id] #4 + ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt index b45713436..e91d49030 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (89) -+- * HashAggregate (88) - +- Exchange (87) - +- * HashAggregate (86) - +- Union (85) +TakeOrderedAndProject (63) ++- * HashAggregate (62) + +- Exchange (61) + +- * HashAggregate (60) + +- Union (59) :- * HashAggregate (28) : +- Exchange (27) : +- * ColumnarToRow (26) @@ -32,62 +32,36 @@ TakeOrderedAndProject (89) : +- CometProject (19) : +- CometFilter (18) : +- CometScan parquet spark_catalog.default.item (17) - :- * HashAggregate (56) - : +- Exchange (55) - : +- * ColumnarToRow (54) - : +- CometHashAggregate (53) - : +- CometProject (52) - : +- CometBroadcastHashJoin (51) - : :- CometProject (42) - : : +- CometBroadcastHashJoin (41) - : : :- CometProject (36) - : : : +- CometBroadcastHashJoin (35) + :- * HashAggregate (43) + : +- Exchange (42) + : +- * ColumnarToRow (41) + : +- CometHashAggregate (40) + : +- CometProject (39) + : +- CometBroadcastHashJoin (38) + : :- CometProject (36) + : : +- CometBroadcastHashJoin (35) + : : :- CometProject (33) + : : : +- CometBroadcastHashJoin (32) : : : :- CometFilter (30) : : : : +- CometScan parquet spark_catalog.default.catalog_sales (29) - : : : +- CometBroadcastExchange (34) - : : : +- CometProject (33) - : : : +- CometFilter (32) - : : : +- CometScan parquet spark_catalog.default.date_dim (31) - : : +- CometBroadcastExchange (40) - : : +- CometProject (39) - : : +- CometFilter (38) - : : +- CometScan parquet spark_catalog.default.customer_address (37) - : +- CometBroadcastExchange (50) - : +- CometBroadcastHashJoin (49) - : :- CometFilter (44) - : : +- CometScan parquet spark_catalog.default.item (43) - : +- CometBroadcastExchange (48) - : +- CometProject (47) - : +- CometFilter (46) - : +- CometScan parquet spark_catalog.default.item (45) - +- * HashAggregate (84) - +- Exchange (83) - +- * ColumnarToRow (82) - +- CometHashAggregate (81) - +- CometProject (80) - +- CometBroadcastHashJoin (79) - :- CometProject (70) - : +- CometBroadcastHashJoin (69) - : :- CometProject (64) - : : +- CometBroadcastHashJoin (63) - : : :- CometFilter (58) - : : : +- CometScan parquet spark_catalog.default.web_sales (57) - : : +- CometBroadcastExchange (62) - : : +- CometProject (61) - : : +- CometFilter (60) - : : +- CometScan parquet spark_catalog.default.date_dim (59) - : +- CometBroadcastExchange (68) - : +- CometProject (67) - : +- CometFilter (66) - : +- CometScan parquet spark_catalog.default.customer_address (65) - +- CometBroadcastExchange (78) - +- CometBroadcastHashJoin (77) - :- CometFilter (72) - : +- CometScan parquet spark_catalog.default.item (71) - +- CometBroadcastExchange (76) - +- CometProject (75) - +- CometFilter (74) - +- CometScan parquet spark_catalog.default.item (73) + : : : +- ReusedExchange (31) + : : +- ReusedExchange (34) + : +- ReusedExchange (37) + +- * HashAggregate (58) + +- Exchange (57) + +- * ColumnarToRow (56) + +- CometHashAggregate (55) + +- CometProject (54) + +- CometBroadcastHashJoin (53) + :- CometProject (51) + : +- CometBroadcastHashJoin (50) + : :- CometProject (48) + : : +- CometBroadcastHashJoin (47) + : : :- CometFilter (45) + : : : +- CometScan parquet spark_catalog.default.web_sales (44) + : : +- ReusedExchange (46) + : +- ReusedExchange (49) + +- ReusedExchange (52) (1) Scan parquet spark_catalog.default.store_sales @@ -237,322 +211,186 @@ ReadSchema: struct - -(32) CometFilter -Input [3]: [d_date_sk#23, d_year#24, d_moy#25] -Condition : ((((isnotnull(d_year#24) AND isnotnull(d_moy#25)) AND (d_year#24 = 1998)) AND (d_moy#25 = 9)) AND isnotnull(d_date_sk#23)) +(31) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#23] -(33) CometProject -Input [3]: [d_date_sk#23, d_year#24, d_moy#25] -Arguments: [d_date_sk#23], [d_date_sk#23] - -(34) CometBroadcastExchange -Input [1]: [d_date_sk#23] -Arguments: [d_date_sk#23] - -(35) CometBroadcastHashJoin +(32) CometBroadcastHashJoin Left output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] Right output [1]: [d_date_sk#23] Arguments: [cs_sold_date_sk#21], [d_date_sk#23], Inner -(36) CometProject +(33) CometProject Input [5]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#23] Arguments: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20], [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] -(37) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#26, ca_gmt_offset#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(38) CometFilter -Input [2]: [ca_address_sk#26, ca_gmt_offset#27] -Condition : ((isnotnull(ca_gmt_offset#27) AND (ca_gmt_offset#27 = -5.00)) AND isnotnull(ca_address_sk#26)) +(34) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#24] -(39) CometProject -Input [2]: [ca_address_sk#26, ca_gmt_offset#27] -Arguments: [ca_address_sk#26], [ca_address_sk#26] - -(40) CometBroadcastExchange -Input [1]: [ca_address_sk#26] -Arguments: [ca_address_sk#26] - -(41) CometBroadcastHashJoin +(35) CometBroadcastHashJoin Left output [3]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] -Right output [1]: [ca_address_sk#26] -Arguments: [cs_bill_addr_sk#18], [ca_address_sk#26], Inner +Right output [1]: [ca_address_sk#24] +Arguments: [cs_bill_addr_sk#18], [ca_address_sk#24], Inner -(42) CometProject -Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, ca_address_sk#26] +(36) CometProject +Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, ca_address_sk#24] Arguments: [cs_item_sk#19, cs_ext_sales_price#20], [cs_item_sk#19, cs_ext_sales_price#20] -(43) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#28, i_item_id#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(44) CometFilter -Input [2]: [i_item_sk#28, i_item_id#29] -Condition : isnotnull(i_item_sk#28) - -(45) Scan parquet spark_catalog.default.item -Output [2]: [i_item_id#30, i_category#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music )] -ReadSchema: struct - -(46) CometFilter -Input [2]: [i_item_id#30, i_category#31] -Condition : (isnotnull(i_category#31) AND (i_category#31 = Music )) - -(47) CometProject -Input [2]: [i_item_id#30, i_category#31] -Arguments: [i_item_id#30], [i_item_id#30] +(37) ReusedExchange [Reuses operator id: 22] +Output [2]: [i_item_sk#25, i_item_id#26] -(48) CometBroadcastExchange -Input [1]: [i_item_id#30] -Arguments: [i_item_id#30] - -(49) CometBroadcastHashJoin -Left output [2]: [i_item_sk#28, i_item_id#29] -Right output [1]: [i_item_id#30] -Arguments: [i_item_id#29], [i_item_id#30], LeftSemi - -(50) CometBroadcastExchange -Input [2]: [i_item_sk#28, i_item_id#29] -Arguments: [i_item_sk#28, i_item_id#29] - -(51) CometBroadcastHashJoin +(38) CometBroadcastHashJoin Left output [2]: [cs_item_sk#19, cs_ext_sales_price#20] -Right output [2]: [i_item_sk#28, i_item_id#29] -Arguments: [cs_item_sk#19], [i_item_sk#28], Inner +Right output [2]: [i_item_sk#25, i_item_id#26] +Arguments: [cs_item_sk#19], [i_item_sk#25], Inner -(52) CometProject -Input [4]: [cs_item_sk#19, cs_ext_sales_price#20, i_item_sk#28, i_item_id#29] -Arguments: [cs_ext_sales_price#20, i_item_id#29], [cs_ext_sales_price#20, i_item_id#29] +(39) CometProject +Input [4]: [cs_item_sk#19, cs_ext_sales_price#20, i_item_sk#25, i_item_id#26] +Arguments: [cs_ext_sales_price#20, i_item_id#26], [cs_ext_sales_price#20, i_item_id#26] -(53) CometHashAggregate -Input [2]: [cs_ext_sales_price#20, i_item_id#29] -Keys [1]: [i_item_id#29] +(40) CometHashAggregate +Input [2]: [cs_ext_sales_price#20, i_item_id#26] +Keys [1]: [i_item_id#26] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#20))] -(54) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_id#29, sum#32] +(41) ColumnarToRow [codegen id : 3] +Input [2]: [i_item_id#26, sum#27] -(55) Exchange -Input [2]: [i_item_id#29, sum#32] -Arguments: hashpartitioning(i_item_id#29, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(42) Exchange +Input [2]: [i_item_id#26, sum#27] +Arguments: hashpartitioning(i_item_id#26, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(56) HashAggregate [codegen id : 4] -Input [2]: [i_item_id#29, sum#32] -Keys [1]: [i_item_id#29] +(43) HashAggregate [codegen id : 4] +Input [2]: [i_item_id#26, sum#27] +Keys [1]: [i_item_id#26] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#20))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#20))#33] -Results [2]: [i_item_id#29, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#20))#33,17,2) AS total_sales#34] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#20))#28] +Results [2]: [i_item_id#26, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#20))#28,17,2) AS total_sales#29] -(57) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#35, ws_bill_addr_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38] +(44) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#39)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#34)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(58) CometFilter -Input [4]: [ws_item_sk#35, ws_bill_addr_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38] -Condition : (isnotnull(ws_bill_addr_sk#36) AND isnotnull(ws_item_sk#35)) - -(59) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#40, d_year#41, d_moy#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,9), IsNotNull(d_date_sk)] -ReadSchema: struct +(45) CometFilter +Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] +Condition : (isnotnull(ws_bill_addr_sk#31) AND isnotnull(ws_item_sk#30)) -(60) CometFilter -Input [3]: [d_date_sk#40, d_year#41, d_moy#42] -Condition : ((((isnotnull(d_year#41) AND isnotnull(d_moy#42)) AND (d_year#41 = 1998)) AND (d_moy#42 = 9)) AND isnotnull(d_date_sk#40)) +(46) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#35] -(61) CometProject -Input [3]: [d_date_sk#40, d_year#41, d_moy#42] -Arguments: [d_date_sk#40], [d_date_sk#40] - -(62) CometBroadcastExchange -Input [1]: [d_date_sk#40] -Arguments: [d_date_sk#40] - -(63) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#35, ws_bill_addr_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38] -Right output [1]: [d_date_sk#40] -Arguments: [ws_sold_date_sk#38], [d_date_sk#40], Inner - -(64) CometProject -Input [5]: [ws_item_sk#35, ws_bill_addr_sk#36, ws_ext_sales_price#37, ws_sold_date_sk#38, d_date_sk#40] -Arguments: [ws_item_sk#35, ws_bill_addr_sk#36, ws_ext_sales_price#37], [ws_item_sk#35, ws_bill_addr_sk#36, ws_ext_sales_price#37] - -(65) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#43, ca_gmt_offset#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(66) CometFilter -Input [2]: [ca_address_sk#43, ca_gmt_offset#44] -Condition : ((isnotnull(ca_gmt_offset#44) AND (ca_gmt_offset#44 = -5.00)) AND isnotnull(ca_address_sk#43)) - -(67) CometProject -Input [2]: [ca_address_sk#43, ca_gmt_offset#44] -Arguments: [ca_address_sk#43], [ca_address_sk#43] - -(68) CometBroadcastExchange -Input [1]: [ca_address_sk#43] -Arguments: [ca_address_sk#43] - -(69) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#35, ws_bill_addr_sk#36, ws_ext_sales_price#37] -Right output [1]: [ca_address_sk#43] -Arguments: [ws_bill_addr_sk#36], [ca_address_sk#43], Inner - -(70) CometProject -Input [4]: [ws_item_sk#35, ws_bill_addr_sk#36, ws_ext_sales_price#37, ca_address_sk#43] -Arguments: [ws_item_sk#35, ws_ext_sales_price#37], [ws_item_sk#35, ws_ext_sales_price#37] - -(71) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#45, i_item_id#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(72) CometFilter -Input [2]: [i_item_sk#45, i_item_id#46] -Condition : isnotnull(i_item_sk#45) - -(73) Scan parquet spark_catalog.default.item -Output [2]: [i_item_id#47, i_category#48] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music )] -ReadSchema: struct +(47) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] +Right output [1]: [d_date_sk#35] +Arguments: [ws_sold_date_sk#33], [d_date_sk#35], Inner -(74) CometFilter -Input [2]: [i_item_id#47, i_category#48] -Condition : (isnotnull(i_category#48) AND (i_category#48 = Music )) +(48) CometProject +Input [5]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33, d_date_sk#35] +Arguments: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32], [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] -(75) CometProject -Input [2]: [i_item_id#47, i_category#48] -Arguments: [i_item_id#47], [i_item_id#47] +(49) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#36] -(76) CometBroadcastExchange -Input [1]: [i_item_id#47] -Arguments: [i_item_id#47] +(50) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] +Right output [1]: [ca_address_sk#36] +Arguments: [ws_bill_addr_sk#31], [ca_address_sk#36], Inner -(77) CometBroadcastHashJoin -Left output [2]: [i_item_sk#45, i_item_id#46] -Right output [1]: [i_item_id#47] -Arguments: [i_item_id#46], [i_item_id#47], LeftSemi +(51) CometProject +Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ca_address_sk#36] +Arguments: [ws_item_sk#30, ws_ext_sales_price#32], [ws_item_sk#30, ws_ext_sales_price#32] -(78) CometBroadcastExchange -Input [2]: [i_item_sk#45, i_item_id#46] -Arguments: [i_item_sk#45, i_item_id#46] +(52) ReusedExchange [Reuses operator id: 22] +Output [2]: [i_item_sk#37, i_item_id#38] -(79) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#35, ws_ext_sales_price#37] -Right output [2]: [i_item_sk#45, i_item_id#46] -Arguments: [ws_item_sk#35], [i_item_sk#45], Inner +(53) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#30, ws_ext_sales_price#32] +Right output [2]: [i_item_sk#37, i_item_id#38] +Arguments: [ws_item_sk#30], [i_item_sk#37], Inner -(80) CometProject -Input [4]: [ws_item_sk#35, ws_ext_sales_price#37, i_item_sk#45, i_item_id#46] -Arguments: [ws_ext_sales_price#37, i_item_id#46], [ws_ext_sales_price#37, i_item_id#46] +(54) CometProject +Input [4]: [ws_item_sk#30, ws_ext_sales_price#32, i_item_sk#37, i_item_id#38] +Arguments: [ws_ext_sales_price#32, i_item_id#38], [ws_ext_sales_price#32, i_item_id#38] -(81) CometHashAggregate -Input [2]: [ws_ext_sales_price#37, i_item_id#46] -Keys [1]: [i_item_id#46] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#37))] +(55) CometHashAggregate +Input [2]: [ws_ext_sales_price#32, i_item_id#38] +Keys [1]: [i_item_id#38] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#32))] -(82) ColumnarToRow [codegen id : 5] -Input [2]: [i_item_id#46, sum#49] +(56) ColumnarToRow [codegen id : 5] +Input [2]: [i_item_id#38, sum#39] -(83) Exchange -Input [2]: [i_item_id#46, sum#49] -Arguments: hashpartitioning(i_item_id#46, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(57) Exchange +Input [2]: [i_item_id#38, sum#39] +Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(84) HashAggregate [codegen id : 6] -Input [2]: [i_item_id#46, sum#49] -Keys [1]: [i_item_id#46] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#37))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#37))#50] -Results [2]: [i_item_id#46, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#37))#50,17,2) AS total_sales#51] +(58) HashAggregate [codegen id : 6] +Input [2]: [i_item_id#38, sum#39] +Keys [1]: [i_item_id#38] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#32))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#32))#40] +Results [2]: [i_item_id#38, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#32))#40,17,2) AS total_sales#41] -(85) Union +(59) Union -(86) HashAggregate [codegen id : 7] +(60) HashAggregate [codegen id : 7] Input [2]: [i_item_id#12, total_sales#17] Keys [1]: [i_item_id#12] Functions [1]: [partial_sum(total_sales#17)] -Aggregate Attributes [2]: [sum#52, isEmpty#53] -Results [3]: [i_item_id#12, sum#54, isEmpty#55] +Aggregate Attributes [2]: [sum#42, isEmpty#43] +Results [3]: [i_item_id#12, sum#44, isEmpty#45] -(87) Exchange -Input [3]: [i_item_id#12, sum#54, isEmpty#55] +(61) Exchange +Input [3]: [i_item_id#12, sum#44, isEmpty#45] Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(88) HashAggregate [codegen id : 8] -Input [3]: [i_item_id#12, sum#54, isEmpty#55] +(62) HashAggregate [codegen id : 8] +Input [3]: [i_item_id#12, sum#44, isEmpty#45] Keys [1]: [i_item_id#12] Functions [1]: [sum(total_sales#17)] -Aggregate Attributes [1]: [sum(total_sales#17)#56] -Results [2]: [i_item_id#12, sum(total_sales#17)#56 AS total_sales#57] +Aggregate Attributes [1]: [sum(total_sales#17)#46] +Results [2]: [i_item_id#12, sum(total_sales#17)#46 AS total_sales#47] -(89) TakeOrderedAndProject -Input [2]: [i_item_id#12, total_sales#57] -Arguments: 100, [i_item_id#12 ASC NULLS FIRST, total_sales#57 ASC NULLS FIRST], [i_item_id#12, total_sales#57] +(63) TakeOrderedAndProject +Input [2]: [i_item_id#12, total_sales#47] +Arguments: 100, [i_item_id#12 ASC NULLS FIRST, total_sales#47 ASC NULLS FIRST], [i_item_id#12, total_sales#47] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (94) -+- * ColumnarToRow (93) - +- CometProject (92) - +- CometFilter (91) - +- CometScan parquet spark_catalog.default.date_dim (90) +BroadcastExchange (68) ++- * ColumnarToRow (67) + +- CometProject (66) + +- CometFilter (65) + +- CometScan parquet spark_catalog.default.date_dim (64) -(90) Scan parquet spark_catalog.default.date_dim +(64) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#6, d_year#7, d_moy#8] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,9), IsNotNull(d_date_sk)] ReadSchema: struct -(91) CometFilter +(65) CometFilter Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 9)) AND isnotnull(d_date_sk#6)) -(92) CometProject +(66) CometProject Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(93) ColumnarToRow [codegen id : 1] +(67) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(94) BroadcastExchange +(68) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt index 0e5d07cc2..b76e7c9b3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt @@ -64,26 +64,13 @@ TakeOrderedAndProject [i_item_id,total_sales] CometFilter [cs_bill_addr_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #9 - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #10 - CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #11 - CometBroadcastHashJoin [i_item_id,i_item_id] - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #12 - CometProject [i_item_id] - CometFilter [i_category] - CometScan parquet spark_catalog.default.item [i_item_id,i_category] + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #6 WholeStageCodegen (6) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_item_id] #13 + Exchange [i_item_id] #9 WholeStageCodegen (5) ColumnarToRow InputAdapter @@ -97,19 +84,6 @@ TakeOrderedAndProject [i_item_id,total_sales] CometFilter [ws_bill_addr_sk,ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #14 - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #15 - CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #16 - CometBroadcastHashJoin [i_item_id,i_item_id] - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #17 - CometProject [i_item_id] - CometFilter [i_category] - CometScan parquet spark_catalog.default.item [i_item_id,i_category] + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt index 3efb41815..da32aa0eb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == -* Project (79) -+- * BroadcastNestedLoopJoin Inner BuildRight (78) +* Project (65) ++- * BroadcastNestedLoopJoin Inner BuildRight (64) :- * HashAggregate (41) : +- Exchange (40) : +- * ColumnarToRow (39) @@ -42,42 +42,28 @@ : +- CometProject (34) : +- CometFilter (33) : +- CometScan parquet spark_catalog.default.item (32) - +- BroadcastExchange (77) - +- * HashAggregate (76) - +- Exchange (75) - +- * ColumnarToRow (74) - +- CometHashAggregate (73) - +- CometProject (72) - +- CometBroadcastHashJoin (71) - :- CometProject (66) - : +- CometBroadcastHashJoin (65) - : :- CometProject (60) - : : +- CometBroadcastHashJoin (59) - : : :- CometProject (55) - : : : +- CometBroadcastHashJoin (54) - : : : :- CometProject (49) - : : : : +- CometBroadcastHashJoin (48) + +- BroadcastExchange (63) + +- * HashAggregate (62) + +- Exchange (61) + +- * ColumnarToRow (60) + +- CometHashAggregate (59) + +- CometProject (58) + +- CometBroadcastHashJoin (57) + :- CometProject (55) + : +- CometBroadcastHashJoin (54) + : :- CometProject (52) + : : +- CometBroadcastHashJoin (51) + : : :- CometProject (49) + : : : +- CometBroadcastHashJoin (48) + : : : :- CometProject (46) + : : : : +- CometBroadcastHashJoin (45) : : : : :- CometFilter (43) : : : : : +- CometScan parquet spark_catalog.default.store_sales (42) - : : : : +- CometBroadcastExchange (47) - : : : : +- CometProject (46) - : : : : +- CometFilter (45) - : : : : +- CometScan parquet spark_catalog.default.store (44) - : : : +- CometBroadcastExchange (53) - : : : +- CometProject (52) - : : : +- CometFilter (51) - : : : +- CometScan parquet spark_catalog.default.date_dim (50) - : : +- CometBroadcastExchange (58) - : : +- CometFilter (57) - : : +- CometScan parquet spark_catalog.default.customer (56) - : +- CometBroadcastExchange (64) - : +- CometProject (63) - : +- CometFilter (62) - : +- CometScan parquet spark_catalog.default.customer_address (61) - +- CometBroadcastExchange (70) - +- CometProject (69) - +- CometFilter (68) - +- CometScan parquet spark_catalog.default.item (67) + : : : : +- ReusedExchange (44) + : : : +- ReusedExchange (47) + : : +- ReusedExchange (50) + : +- ReusedExchange (53) + +- ReusedExchange (56) (1) Scan parquet spark_catalog.default.store_sales @@ -287,202 +273,126 @@ ReadSchema: struct - -(45) CometFilter -Input [2]: [s_store_sk#32, s_gmt_offset#33] -Condition : ((isnotnull(s_gmt_offset#33) AND (s_gmt_offset#33 = -5.00)) AND isnotnull(s_store_sk#32)) - -(46) CometProject -Input [2]: [s_store_sk#32, s_gmt_offset#33] -Arguments: [s_store_sk#32], [s_store_sk#32] - -(47) CometBroadcastExchange -Input [1]: [s_store_sk#32] -Arguments: [s_store_sk#32] +(44) ReusedExchange [Reuses operator id: 6] +Output [1]: [s_store_sk#32] -(48) CometBroadcastHashJoin +(45) CometBroadcastHashJoin Left output [5]: [ss_item_sk#26, ss_customer_sk#27, ss_store_sk#28, ss_ext_sales_price#29, ss_sold_date_sk#30] Right output [1]: [s_store_sk#32] Arguments: [ss_store_sk#28], [s_store_sk#32], Inner -(49) CometProject +(46) CometProject Input [6]: [ss_item_sk#26, ss_customer_sk#27, ss_store_sk#28, ss_ext_sales_price#29, ss_sold_date_sk#30, s_store_sk#32] Arguments: [ss_item_sk#26, ss_customer_sk#27, ss_ext_sales_price#29, ss_sold_date_sk#30], [ss_item_sk#26, ss_customer_sk#27, ss_ext_sales_price#29, ss_sold_date_sk#30] -(50) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#34, d_year#35, d_moy#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(51) CometFilter -Input [3]: [d_date_sk#34, d_year#35, d_moy#36] -Condition : ((((isnotnull(d_year#35) AND isnotnull(d_moy#36)) AND (d_year#35 = 1998)) AND (d_moy#36 = 11)) AND isnotnull(d_date_sk#34)) - -(52) CometProject -Input [3]: [d_date_sk#34, d_year#35, d_moy#36] -Arguments: [d_date_sk#34], [d_date_sk#34] +(47) ReusedExchange [Reuses operator id: 18] +Output [1]: [d_date_sk#33] -(53) CometBroadcastExchange -Input [1]: [d_date_sk#34] -Arguments: [d_date_sk#34] - -(54) CometBroadcastHashJoin +(48) CometBroadcastHashJoin Left output [4]: [ss_item_sk#26, ss_customer_sk#27, ss_ext_sales_price#29, ss_sold_date_sk#30] -Right output [1]: [d_date_sk#34] -Arguments: [ss_sold_date_sk#30], [d_date_sk#34], Inner +Right output [1]: [d_date_sk#33] +Arguments: [ss_sold_date_sk#30], [d_date_sk#33], Inner -(55) CometProject -Input [5]: [ss_item_sk#26, ss_customer_sk#27, ss_ext_sales_price#29, ss_sold_date_sk#30, d_date_sk#34] +(49) CometProject +Input [5]: [ss_item_sk#26, ss_customer_sk#27, ss_ext_sales_price#29, ss_sold_date_sk#30, d_date_sk#33] Arguments: [ss_item_sk#26, ss_customer_sk#27, ss_ext_sales_price#29], [ss_item_sk#26, ss_customer_sk#27, ss_ext_sales_price#29] -(56) Scan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#37, c_current_addr_sk#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(57) CometFilter -Input [2]: [c_customer_sk#37, c_current_addr_sk#38] -Condition : (isnotnull(c_customer_sk#37) AND isnotnull(c_current_addr_sk#38)) - -(58) CometBroadcastExchange -Input [2]: [c_customer_sk#37, c_current_addr_sk#38] -Arguments: [c_customer_sk#37, c_current_addr_sk#38] +(50) ReusedExchange [Reuses operator id: 23] +Output [2]: [c_customer_sk#34, c_current_addr_sk#35] -(59) CometBroadcastHashJoin +(51) CometBroadcastHashJoin Left output [3]: [ss_item_sk#26, ss_customer_sk#27, ss_ext_sales_price#29] -Right output [2]: [c_customer_sk#37, c_current_addr_sk#38] -Arguments: [ss_customer_sk#27], [c_customer_sk#37], Inner - -(60) CometProject -Input [5]: [ss_item_sk#26, ss_customer_sk#27, ss_ext_sales_price#29, c_customer_sk#37, c_current_addr_sk#38] -Arguments: [ss_item_sk#26, ss_ext_sales_price#29, c_current_addr_sk#38], [ss_item_sk#26, ss_ext_sales_price#29, c_current_addr_sk#38] - -(61) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#39, ca_gmt_offset#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct +Right output [2]: [c_customer_sk#34, c_current_addr_sk#35] +Arguments: [ss_customer_sk#27], [c_customer_sk#34], Inner -(62) CometFilter -Input [2]: [ca_address_sk#39, ca_gmt_offset#40] -Condition : ((isnotnull(ca_gmt_offset#40) AND (ca_gmt_offset#40 = -5.00)) AND isnotnull(ca_address_sk#39)) - -(63) CometProject -Input [2]: [ca_address_sk#39, ca_gmt_offset#40] -Arguments: [ca_address_sk#39], [ca_address_sk#39] +(52) CometProject +Input [5]: [ss_item_sk#26, ss_customer_sk#27, ss_ext_sales_price#29, c_customer_sk#34, c_current_addr_sk#35] +Arguments: [ss_item_sk#26, ss_ext_sales_price#29, c_current_addr_sk#35], [ss_item_sk#26, ss_ext_sales_price#29, c_current_addr_sk#35] -(64) CometBroadcastExchange -Input [1]: [ca_address_sk#39] -Arguments: [ca_address_sk#39] +(53) ReusedExchange [Reuses operator id: 29] +Output [1]: [ca_address_sk#36] -(65) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#26, ss_ext_sales_price#29, c_current_addr_sk#38] -Right output [1]: [ca_address_sk#39] -Arguments: [c_current_addr_sk#38], [ca_address_sk#39], Inner +(54) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#26, ss_ext_sales_price#29, c_current_addr_sk#35] +Right output [1]: [ca_address_sk#36] +Arguments: [c_current_addr_sk#35], [ca_address_sk#36], Inner -(66) CometProject -Input [4]: [ss_item_sk#26, ss_ext_sales_price#29, c_current_addr_sk#38, ca_address_sk#39] +(55) CometProject +Input [4]: [ss_item_sk#26, ss_ext_sales_price#29, c_current_addr_sk#35, ca_address_sk#36] Arguments: [ss_item_sk#26, ss_ext_sales_price#29], [ss_item_sk#26, ss_ext_sales_price#29] -(67) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#41, i_category#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Jewelry ), IsNotNull(i_item_sk)] -ReadSchema: struct - -(68) CometFilter -Input [2]: [i_item_sk#41, i_category#42] -Condition : ((isnotnull(i_category#42) AND (i_category#42 = Jewelry )) AND isnotnull(i_item_sk#41)) - -(69) CometProject -Input [2]: [i_item_sk#41, i_category#42] -Arguments: [i_item_sk#41], [i_item_sk#41] - -(70) CometBroadcastExchange -Input [1]: [i_item_sk#41] -Arguments: [i_item_sk#41] +(56) ReusedExchange [Reuses operator id: 35] +Output [1]: [i_item_sk#37] -(71) CometBroadcastHashJoin +(57) CometBroadcastHashJoin Left output [2]: [ss_item_sk#26, ss_ext_sales_price#29] -Right output [1]: [i_item_sk#41] -Arguments: [ss_item_sk#26], [i_item_sk#41], Inner +Right output [1]: [i_item_sk#37] +Arguments: [ss_item_sk#26], [i_item_sk#37], Inner -(72) CometProject -Input [3]: [ss_item_sk#26, ss_ext_sales_price#29, i_item_sk#41] +(58) CometProject +Input [3]: [ss_item_sk#26, ss_ext_sales_price#29, i_item_sk#37] Arguments: [ss_ext_sales_price#29], [ss_ext_sales_price#29] -(73) CometHashAggregate +(59) CometHashAggregate Input [1]: [ss_ext_sales_price#29] Keys: [] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#29))] -(74) ColumnarToRow [codegen id : 2] -Input [1]: [sum#43] +(60) ColumnarToRow [codegen id : 2] +Input [1]: [sum#38] -(75) Exchange -Input [1]: [sum#43] +(61) Exchange +Input [1]: [sum#38] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] -(76) HashAggregate [codegen id : 3] -Input [1]: [sum#43] +(62) HashAggregate [codegen id : 3] +Input [1]: [sum#38] Keys: [] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#29))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#29))#44] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#29))#44,17,2) AS total#45] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#29))#39] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#29))#39,17,2) AS total#40] -(77) BroadcastExchange -Input [1]: [total#45] +(63) BroadcastExchange +Input [1]: [total#40] Arguments: IdentityBroadcastMode, [plan_id=3] -(78) BroadcastNestedLoopJoin [codegen id : 4] +(64) BroadcastNestedLoopJoin [codegen id : 4] Join type: Inner Join condition: None -(79) Project [codegen id : 4] -Output [3]: [promotions#25, total#45, ((cast(promotions#25 as decimal(15,4)) / cast(total#45 as decimal(15,4))) * 100) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#46] -Input [2]: [promotions#25, total#45] +(65) Project [codegen id : 4] +Output [3]: [promotions#25, total#40, ((cast(promotions#25 as decimal(15,4)) / cast(total#40 as decimal(15,4))) * 100) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#41] +Input [2]: [promotions#25, total#40] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (84) -+- * ColumnarToRow (83) - +- CometProject (82) - +- CometFilter (81) - +- CometScan parquet spark_catalog.default.date_dim (80) +BroadcastExchange (70) ++- * ColumnarToRow (69) + +- CometProject (68) + +- CometFilter (67) + +- CometScan parquet spark_catalog.default.date_dim (66) -(80) Scan parquet spark_catalog.default.date_dim +(66) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#14, d_year#15, d_moy#16] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(81) CometFilter +(67) CometFilter Input [3]: [d_date_sk#14, d_year#15, d_moy#16] Condition : ((((isnotnull(d_year#15) AND isnotnull(d_moy#16)) AND (d_year#15 = 1998)) AND (d_moy#16 = 11)) AND isnotnull(d_date_sk#14)) -(82) CometProject +(68) CometProject Input [3]: [d_date_sk#14, d_year#15, d_moy#16] Arguments: [d_date_sk#14], [d_date_sk#14] -(83) ColumnarToRow [codegen id : 1] +(69) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(84) BroadcastExchange +(70) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt index 4001f570b..4ca1dd667 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt @@ -76,22 +76,8 @@ WholeStageCodegen (4) CometFilter [ss_store_sk,ss_customer_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #11 - CometProject [s_store_sk] - CometFilter [s_gmt_offset,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] - CometBroadcastExchange #12 - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #13 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #14 - CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #15 - CometProject [i_item_sk] - CometFilter [i_category,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_category] + ReusedExchange [s_store_sk] #3 + ReusedExchange [d_date_sk] #5 + ReusedExchange [c_customer_sk,c_current_addr_sk] #6 + ReusedExchange [ca_address_sk] #7 + ReusedExchange [i_item_sk] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt index 05610a0f1..56500b3fe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -TakeOrderedAndProject (45) -+- * Project (44) - +- * BroadcastHashJoin Inner BuildRight (43) +TakeOrderedAndProject (42) ++- * Project (41) + +- * BroadcastHashJoin Inner BuildRight (40) :- * Project (25) : +- * BroadcastHashJoin Inner BuildRight (24) : :- * Project (19) @@ -27,23 +27,20 @@ TakeOrderedAndProject (45) : +- * ColumnarToRow (22) : +- CometFilter (21) : +- CometScan parquet spark_catalog.default.item (20) - +- BroadcastExchange (42) - +- * Filter (41) - +- * HashAggregate (40) - +- Exchange (39) - +- * HashAggregate (38) - +- * HashAggregate (37) - +- Exchange (36) - +- * ColumnarToRow (35) - +- CometHashAggregate (34) - +- CometProject (33) - +- CometBroadcastHashJoin (32) + +- BroadcastExchange (39) + +- * Filter (38) + +- * HashAggregate (37) + +- Exchange (36) + +- * HashAggregate (35) + +- * HashAggregate (34) + +- Exchange (33) + +- * ColumnarToRow (32) + +- CometHashAggregate (31) + +- CometProject (30) + +- CometBroadcastHashJoin (29) :- CometFilter (27) : +- CometScan parquet spark_catalog.default.store_sales (26) - +- CometBroadcastExchange (31) - +- CometProject (30) - +- CometFilter (29) - +- CometScan parquet spark_catalog.default.date_dim (28) + +- ReusedExchange (28) (1) Scan parquet spark_catalog.default.store @@ -177,122 +174,106 @@ ReadSchema: struct Input [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] Condition : isnotnull(ss_store_sk#19) -(28) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#23, d_month_seq#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)] -ReadSchema: struct - -(29) CometFilter -Input [2]: [d_date_sk#23, d_month_seq#24] -Condition : (((isnotnull(d_month_seq#24) AND (d_month_seq#24 >= 1176)) AND (d_month_seq#24 <= 1187)) AND isnotnull(d_date_sk#23)) +(28) ReusedExchange [Reuses operator id: 9] +Output [1]: [d_date_sk#23] -(30) CometProject -Input [2]: [d_date_sk#23, d_month_seq#24] -Arguments: [d_date_sk#23], [d_date_sk#23] - -(31) CometBroadcastExchange -Input [1]: [d_date_sk#23] -Arguments: [d_date_sk#23] - -(32) CometBroadcastHashJoin +(29) CometBroadcastHashJoin Left output [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] Right output [1]: [d_date_sk#23] Arguments: [ss_sold_date_sk#21], [d_date_sk#23], Inner -(33) CometProject +(30) CometProject Input [5]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21, d_date_sk#23] Arguments: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20], [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20] -(34) CometHashAggregate +(31) CometHashAggregate Input [3]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20] Keys [2]: [ss_store_sk#19, ss_item_sk#18] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#20))] -(35) ColumnarToRow [codegen id : 4] -Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#25] +(32) ColumnarToRow [codegen id : 4] +Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#24] -(36) Exchange -Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#25] +(33) Exchange +Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#24] Arguments: hashpartitioning(ss_store_sk#19, ss_item_sk#18, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(37) HashAggregate [codegen id : 5] -Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#25] +(34) HashAggregate [codegen id : 5] +Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#24] Keys [2]: [ss_store_sk#19, ss_item_sk#18] Functions [1]: [sum(UnscaledValue(ss_sales_price#20))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#20))#26] -Results [2]: [ss_store_sk#19, MakeDecimal(sum(UnscaledValue(ss_sales_price#20))#26,17,2) AS revenue#27] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#20))#25] +Results [2]: [ss_store_sk#19, MakeDecimal(sum(UnscaledValue(ss_sales_price#20))#25,17,2) AS revenue#26] -(38) HashAggregate [codegen id : 5] -Input [2]: [ss_store_sk#19, revenue#27] +(35) HashAggregate [codegen id : 5] +Input [2]: [ss_store_sk#19, revenue#26] Keys [1]: [ss_store_sk#19] -Functions [1]: [partial_avg(revenue#27)] -Aggregate Attributes [2]: [sum#28, count#29] -Results [3]: [ss_store_sk#19, sum#30, count#31] +Functions [1]: [partial_avg(revenue#26)] +Aggregate Attributes [2]: [sum#27, count#28] +Results [3]: [ss_store_sk#19, sum#29, count#30] -(39) Exchange -Input [3]: [ss_store_sk#19, sum#30, count#31] +(36) Exchange +Input [3]: [ss_store_sk#19, sum#29, count#30] Arguments: hashpartitioning(ss_store_sk#19, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(40) HashAggregate [codegen id : 6] -Input [3]: [ss_store_sk#19, sum#30, count#31] +(37) HashAggregate [codegen id : 6] +Input [3]: [ss_store_sk#19, sum#29, count#30] Keys [1]: [ss_store_sk#19] -Functions [1]: [avg(revenue#27)] -Aggregate Attributes [1]: [avg(revenue#27)#32] -Results [2]: [ss_store_sk#19, avg(revenue#27)#32 AS ave#33] +Functions [1]: [avg(revenue#26)] +Aggregate Attributes [1]: [avg(revenue#26)#31] +Results [2]: [ss_store_sk#19, avg(revenue#26)#31 AS ave#32] -(41) Filter [codegen id : 6] -Input [2]: [ss_store_sk#19, ave#33] -Condition : isnotnull(ave#33) +(38) Filter [codegen id : 6] +Input [2]: [ss_store_sk#19, ave#32] +Condition : isnotnull(ave#32) -(42) BroadcastExchange -Input [2]: [ss_store_sk#19, ave#33] +(39) BroadcastExchange +Input [2]: [ss_store_sk#19, ave#32] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(43) BroadcastHashJoin [codegen id : 7] +(40) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ss_store_sk#4] Right keys [1]: [ss_store_sk#19] Join type: Inner -Join condition: (cast(revenue#12 as decimal(23,7)) <= (0.1 * ave#33)) +Join condition: (cast(revenue#12 as decimal(23,7)) <= (0.1 * ave#32)) -(44) Project [codegen id : 7] +(41) Project [codegen id : 7] Output [6]: [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#17] -Input [9]: [s_store_name#2, ss_store_sk#4, revenue#12, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17, ss_store_sk#19, ave#33] +Input [9]: [s_store_name#2, ss_store_sk#4, revenue#12, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17, ss_store_sk#19, ave#32] -(45) TakeOrderedAndProject +(42) TakeOrderedAndProject Input [6]: [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#17] Arguments: 100, [s_store_name#2 ASC NULLS FIRST, i_item_desc#14 ASC NULLS FIRST], [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (50) -+- * ColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan parquet spark_catalog.default.date_dim (46) +BroadcastExchange (47) ++- * ColumnarToRow (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) -(46) Scan parquet spark_catalog.default.date_dim +(43) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#8, d_month_seq#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)] ReadSchema: struct -(47) CometFilter +(44) CometFilter Input [2]: [d_date_sk#8, d_month_seq#9] Condition : (((isnotnull(d_month_seq#9) AND (d_month_seq#9 >= 1176)) AND (d_month_seq#9 <= 1187)) AND isnotnull(d_date_sk#8)) -(48) CometProject +(45) CometProject Input [2]: [d_date_sk#8, d_month_seq#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(49) ColumnarToRow [codegen id : 1] +(46) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#8] -(50) BroadcastExchange +(47) BroadcastExchange Input [1]: [d_date_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt index e66643eb0..8de564ed1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt @@ -65,7 +65,4 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes CometFilter [ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #9 - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt index 36ee57502..a180017a0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (61) -+- * HashAggregate (60) - +- Exchange (59) - +- * HashAggregate (58) - +- Union (57) +TakeOrderedAndProject (51) ++- * HashAggregate (50) + +- Exchange (49) + +- * HashAggregate (48) + +- Union (47) :- * HashAggregate (28) : +- Exchange (27) : +- * ColumnarToRow (26) @@ -32,34 +32,24 @@ TakeOrderedAndProject (61) : +- CometProject (21) : +- CometFilter (20) : +- CometScan parquet spark_catalog.default.ship_mode (19) - +- * HashAggregate (56) - +- Exchange (55) - +- * ColumnarToRow (54) - +- CometHashAggregate (53) - +- CometProject (52) - +- CometBroadcastHashJoin (51) - :- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (40) - : : +- CometBroadcastHashJoin (39) - : : :- CometProject (35) - : : : +- CometBroadcastHashJoin (34) + +- * HashAggregate (46) + +- Exchange (45) + +- * ColumnarToRow (44) + +- CometHashAggregate (43) + +- CometProject (42) + +- CometBroadcastHashJoin (41) + :- CometProject (39) + : +- CometBroadcastHashJoin (38) + : :- CometProject (36) + : : +- CometBroadcastHashJoin (35) + : : :- CometProject (33) + : : : +- CometBroadcastHashJoin (32) : : : :- CometFilter (30) : : : : +- CometScan parquet spark_catalog.default.catalog_sales (29) - : : : +- CometBroadcastExchange (33) - : : : +- CometFilter (32) - : : : +- CometScan parquet spark_catalog.default.warehouse (31) - : : +- CometBroadcastExchange (38) - : : +- CometFilter (37) - : : +- CometScan parquet spark_catalog.default.date_dim (36) - : +- CometBroadcastExchange (44) - : +- CometProject (43) - : +- CometFilter (42) - : +- CometScan parquet spark_catalog.default.time_dim (41) - +- CometBroadcastExchange (50) - +- CometProject (49) - +- CometFilter (48) - +- CometScan parquet spark_catalog.default.ship_mode (47) + : : : +- ReusedExchange (31) + : : +- ReusedExchange (34) + : +- ReusedExchange (37) + +- ReusedExchange (40) (1) Scan parquet spark_catalog.default.web_sales @@ -209,177 +199,121 @@ ReadSchema: struct -(32) CometFilter -Input [7]: [w_warehouse_sk#129, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135] -Condition : isnotnull(w_warehouse_sk#129) - -(33) CometBroadcastExchange -Input [7]: [w_warehouse_sk#129, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135] -Arguments: [w_warehouse_sk#129, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135] - -(34) CometBroadcastHashJoin +(32) CometBroadcastHashJoin Left output [7]: [cs_sold_time_sk#121, cs_ship_mode_sk#122, cs_warehouse_sk#123, cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, cs_sold_date_sk#127] Right output [7]: [w_warehouse_sk#129, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135] Arguments: [cs_warehouse_sk#123], [w_warehouse_sk#129], Inner -(35) CometProject +(33) CometProject Input [14]: [cs_sold_time_sk#121, cs_ship_mode_sk#122, cs_warehouse_sk#123, cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, cs_sold_date_sk#127, w_warehouse_sk#129, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135] Arguments: [cs_sold_time_sk#121, cs_ship_mode_sk#122, cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, cs_sold_date_sk#127, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135], [cs_sold_time_sk#121, cs_ship_mode_sk#122, cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, cs_sold_date_sk#127, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135] -(36) Scan parquet spark_catalog.default.date_dim +(34) ReusedExchange [Reuses operator id: 10] Output [3]: [d_date_sk#136, d_year#137, d_moy#138] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct -(37) CometFilter -Input [3]: [d_date_sk#136, d_year#137, d_moy#138] -Condition : ((isnotnull(d_year#137) AND (d_year#137 = 2001)) AND isnotnull(d_date_sk#136)) - -(38) CometBroadcastExchange -Input [3]: [d_date_sk#136, d_year#137, d_moy#138] -Arguments: [d_date_sk#136, d_year#137, d_moy#138] - -(39) CometBroadcastHashJoin +(35) CometBroadcastHashJoin Left output [12]: [cs_sold_time_sk#121, cs_ship_mode_sk#122, cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, cs_sold_date_sk#127, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135] Right output [3]: [d_date_sk#136, d_year#137, d_moy#138] Arguments: [cs_sold_date_sk#127], [d_date_sk#136], Inner -(40) CometProject +(36) CometProject Input [15]: [cs_sold_time_sk#121, cs_ship_mode_sk#122, cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, cs_sold_date_sk#127, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_date_sk#136, d_year#137, d_moy#138] Arguments: [cs_sold_time_sk#121, cs_ship_mode_sk#122, cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137, d_moy#138], [cs_sold_time_sk#121, cs_ship_mode_sk#122, cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137, d_moy#138] -(41) Scan parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#139, t_time#140] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_time), GreaterThanOrEqual(t_time,30838), LessThanOrEqual(t_time,59638), IsNotNull(t_time_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [t_time_sk#139, t_time#140] -Condition : (((isnotnull(t_time#140) AND (t_time#140 >= 30838)) AND (t_time#140 <= 59638)) AND isnotnull(t_time_sk#139)) +(37) ReusedExchange [Reuses operator id: 16] +Output [1]: [t_time_sk#139] -(43) CometProject -Input [2]: [t_time_sk#139, t_time#140] -Arguments: [t_time_sk#139], [t_time_sk#139] - -(44) CometBroadcastExchange -Input [1]: [t_time_sk#139] -Arguments: [t_time_sk#139] - -(45) CometBroadcastHashJoin +(38) CometBroadcastHashJoin Left output [13]: [cs_sold_time_sk#121, cs_ship_mode_sk#122, cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137, d_moy#138] Right output [1]: [t_time_sk#139] Arguments: [cs_sold_time_sk#121], [t_time_sk#139], Inner -(46) CometProject +(39) CometProject Input [14]: [cs_sold_time_sk#121, cs_ship_mode_sk#122, cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137, d_moy#138, t_time_sk#139] Arguments: [cs_ship_mode_sk#122, cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137, d_moy#138], [cs_ship_mode_sk#122, cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137, d_moy#138] -(47) Scan parquet spark_catalog.default.ship_mode -Output [2]: [sm_ship_mode_sk#141, sm_carrier#142] -Batched: true -Location [not included in comparison]/{warehouse_dir}/ship_mode] -PushedFilters: [In(sm_carrier, [BARIAN ,DHL ]), IsNotNull(sm_ship_mode_sk)] -ReadSchema: struct - -(48) CometFilter -Input [2]: [sm_ship_mode_sk#141, sm_carrier#142] -Condition : (sm_carrier#142 IN (DHL ,BARIAN ) AND isnotnull(sm_ship_mode_sk#141)) - -(49) CometProject -Input [2]: [sm_ship_mode_sk#141, sm_carrier#142] -Arguments: [sm_ship_mode_sk#141], [sm_ship_mode_sk#141] - -(50) CometBroadcastExchange -Input [1]: [sm_ship_mode_sk#141] -Arguments: [sm_ship_mode_sk#141] +(40) ReusedExchange [Reuses operator id: 22] +Output [1]: [sm_ship_mode_sk#140] -(51) CometBroadcastHashJoin +(41) CometBroadcastHashJoin Left output [12]: [cs_ship_mode_sk#122, cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137, d_moy#138] -Right output [1]: [sm_ship_mode_sk#141] -Arguments: [cs_ship_mode_sk#122], [sm_ship_mode_sk#141], Inner +Right output [1]: [sm_ship_mode_sk#140] +Arguments: [cs_ship_mode_sk#122], [sm_ship_mode_sk#140], Inner -(52) CometProject -Input [13]: [cs_ship_mode_sk#122, cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137, d_moy#138, sm_ship_mode_sk#141] +(42) CometProject +Input [13]: [cs_ship_mode_sk#122, cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137, d_moy#138, sm_ship_mode_sk#140] Arguments: [cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137, d_moy#138], [cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137, d_moy#138] -(53) CometHashAggregate +(43) CometHashAggregate Input [11]: [cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137, d_moy#138] Keys [7]: [w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137] Functions [24]: [partial_sum(CASE WHEN (d_moy#138 = 1) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 2) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 3) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 4) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 5) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 6) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 7) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 8) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 9) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 10) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 11) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 12) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 1) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 2) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 3) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 4) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 5) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 6) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 7) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 8) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 9) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 10) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 11) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 12) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)] -(54) ColumnarToRow [codegen id : 3] -Input [55]: [w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180, sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] +(44) ColumnarToRow [codegen id : 3] +Input [55]: [w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180, sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188] -(55) Exchange -Input [55]: [w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180, sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] +(45) Exchange +Input [55]: [w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180, sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188] Arguments: hashpartitioning(w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(56) HashAggregate [codegen id : 4] -Input [55]: [w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180, sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] +(46) HashAggregate [codegen id : 4] +Input [55]: [w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180, sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188] Keys [7]: [w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137] Functions [24]: [sum(CASE WHEN (d_moy#138 = 1) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 2) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 3) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 4) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 5) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 6) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 7) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 8) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 9) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 10) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 11) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 12) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 1) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 2) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 3) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 4) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 5) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 6) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 7) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 8) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 9) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 10) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 11) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 12) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)] -Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#138 = 1) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#191, sum(CASE WHEN (d_moy#138 = 2) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#192, sum(CASE WHEN (d_moy#138 = 3) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#193, sum(CASE WHEN (d_moy#138 = 4) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#194, sum(CASE WHEN (d_moy#138 = 5) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#195, sum(CASE WHEN (d_moy#138 = 6) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#196, sum(CASE WHEN (d_moy#138 = 7) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#197, sum(CASE WHEN (d_moy#138 = 8) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#198, sum(CASE WHEN (d_moy#138 = 9) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#199, sum(CASE WHEN (d_moy#138 = 10) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#200, sum(CASE WHEN (d_moy#138 = 11) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#201, sum(CASE WHEN (d_moy#138 = 12) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#202, sum(CASE WHEN (d_moy#138 = 1) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#203, sum(CASE WHEN (d_moy#138 = 2) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#204, sum(CASE WHEN (d_moy#138 = 3) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#205, sum(CASE WHEN (d_moy#138 = 4) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#206, sum(CASE WHEN (d_moy#138 = 5) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#207, sum(CASE WHEN (d_moy#138 = 6) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#208, sum(CASE WHEN (d_moy#138 = 7) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#209, sum(CASE WHEN (d_moy#138 = 8) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#210, sum(CASE WHEN (d_moy#138 = 9) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#211, sum(CASE WHEN (d_moy#138 = 10) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#212, sum(CASE WHEN (d_moy#138 = 11) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#213, sum(CASE WHEN (d_moy#138 = 12) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#214] -Results [32]: [w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, DHL,BARIAN AS ship_carriers#215, d_year#137 AS year#216, sum(CASE WHEN (d_moy#138 = 1) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#191 AS jan_sales#217, sum(CASE WHEN (d_moy#138 = 2) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#192 AS feb_sales#218, sum(CASE WHEN (d_moy#138 = 3) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#193 AS mar_sales#219, sum(CASE WHEN (d_moy#138 = 4) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#194 AS apr_sales#220, sum(CASE WHEN (d_moy#138 = 5) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#195 AS may_sales#221, sum(CASE WHEN (d_moy#138 = 6) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#196 AS jun_sales#222, sum(CASE WHEN (d_moy#138 = 7) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#197 AS jul_sales#223, sum(CASE WHEN (d_moy#138 = 8) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#198 AS aug_sales#224, sum(CASE WHEN (d_moy#138 = 9) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#199 AS sep_sales#225, sum(CASE WHEN (d_moy#138 = 10) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#200 AS oct_sales#226, sum(CASE WHEN (d_moy#138 = 11) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#201 AS nov_sales#227, sum(CASE WHEN (d_moy#138 = 12) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#202 AS dec_sales#228, sum(CASE WHEN (d_moy#138 = 1) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#203 AS jan_net#229, sum(CASE WHEN (d_moy#138 = 2) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#204 AS feb_net#230, sum(CASE WHEN (d_moy#138 = 3) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#205 AS mar_net#231, sum(CASE WHEN (d_moy#138 = 4) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#206 AS apr_net#232, sum(CASE WHEN (d_moy#138 = 5) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#207 AS may_net#233, sum(CASE WHEN (d_moy#138 = 6) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#208 AS jun_net#234, sum(CASE WHEN (d_moy#138 = 7) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#209 AS jul_net#235, sum(CASE WHEN (d_moy#138 = 8) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#210 AS aug_net#236, sum(CASE WHEN (d_moy#138 = 9) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#211 AS sep_net#237, sum(CASE WHEN (d_moy#138 = 10) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#212 AS oct_net#238, sum(CASE WHEN (d_moy#138 = 11) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#213 AS nov_net#239, sum(CASE WHEN (d_moy#138 = 12) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#214 AS dec_net#240] +Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#138 = 1) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#189, sum(CASE WHEN (d_moy#138 = 2) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#190, sum(CASE WHEN (d_moy#138 = 3) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#191, sum(CASE WHEN (d_moy#138 = 4) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#192, sum(CASE WHEN (d_moy#138 = 5) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#193, sum(CASE WHEN (d_moy#138 = 6) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#194, sum(CASE WHEN (d_moy#138 = 7) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#195, sum(CASE WHEN (d_moy#138 = 8) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#196, sum(CASE WHEN (d_moy#138 = 9) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#197, sum(CASE WHEN (d_moy#138 = 10) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#198, sum(CASE WHEN (d_moy#138 = 11) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#199, sum(CASE WHEN (d_moy#138 = 12) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#200, sum(CASE WHEN (d_moy#138 = 1) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#201, sum(CASE WHEN (d_moy#138 = 2) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#202, sum(CASE WHEN (d_moy#138 = 3) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#203, sum(CASE WHEN (d_moy#138 = 4) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#204, sum(CASE WHEN (d_moy#138 = 5) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#205, sum(CASE WHEN (d_moy#138 = 6) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#206, sum(CASE WHEN (d_moy#138 = 7) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#207, sum(CASE WHEN (d_moy#138 = 8) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#208, sum(CASE WHEN (d_moy#138 = 9) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#209, sum(CASE WHEN (d_moy#138 = 10) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#210, sum(CASE WHEN (d_moy#138 = 11) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#211, sum(CASE WHEN (d_moy#138 = 12) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#212] +Results [32]: [w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, DHL,BARIAN AS ship_carriers#213, d_year#137 AS year#214, sum(CASE WHEN (d_moy#138 = 1) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#189 AS jan_sales#215, sum(CASE WHEN (d_moy#138 = 2) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#190 AS feb_sales#216, sum(CASE WHEN (d_moy#138 = 3) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#191 AS mar_sales#217, sum(CASE WHEN (d_moy#138 = 4) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#192 AS apr_sales#218, sum(CASE WHEN (d_moy#138 = 5) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#193 AS may_sales#219, sum(CASE WHEN (d_moy#138 = 6) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#194 AS jun_sales#220, sum(CASE WHEN (d_moy#138 = 7) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#195 AS jul_sales#221, sum(CASE WHEN (d_moy#138 = 8) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#196 AS aug_sales#222, sum(CASE WHEN (d_moy#138 = 9) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#197 AS sep_sales#223, sum(CASE WHEN (d_moy#138 = 10) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#198 AS oct_sales#224, sum(CASE WHEN (d_moy#138 = 11) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#199 AS nov_sales#225, sum(CASE WHEN (d_moy#138 = 12) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#200 AS dec_sales#226, sum(CASE WHEN (d_moy#138 = 1) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#201 AS jan_net#227, sum(CASE WHEN (d_moy#138 = 2) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#202 AS feb_net#228, sum(CASE WHEN (d_moy#138 = 3) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#203 AS mar_net#229, sum(CASE WHEN (d_moy#138 = 4) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#204 AS apr_net#230, sum(CASE WHEN (d_moy#138 = 5) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#205 AS may_net#231, sum(CASE WHEN (d_moy#138 = 6) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#206 AS jun_net#232, sum(CASE WHEN (d_moy#138 = 7) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#207 AS jul_net#233, sum(CASE WHEN (d_moy#138 = 8) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#208 AS aug_net#234, sum(CASE WHEN (d_moy#138 = 9) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#209 AS sep_net#235, sum(CASE WHEN (d_moy#138 = 10) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#210 AS oct_net#236, sum(CASE WHEN (d_moy#138 = 11) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#211 AS nov_net#237, sum(CASE WHEN (d_moy#138 = 12) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#212 AS dec_net#238] -(57) Union +(47) Union -(58) HashAggregate [codegen id : 5] +(48) HashAggregate [codegen id : 5] Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#95, year#96, jan_sales#97, feb_sales#98, mar_sales#99, apr_sales#100, may_sales#101, jun_sales#102, jul_sales#103, aug_sales#104, sep_sales#105, oct_sales#106, nov_sales#107, dec_sales#108, jan_net#109, feb_net#110, mar_net#111, apr_net#112, may_net#113, jun_net#114, jul_net#115, aug_net#116, sep_net#117, oct_net#118, nov_net#119, dec_net#120] Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#95, year#96] Functions [36]: [partial_sum(jan_sales#97), partial_sum(feb_sales#98), partial_sum(mar_sales#99), partial_sum(apr_sales#100), partial_sum(may_sales#101), partial_sum(jun_sales#102), partial_sum(jul_sales#103), partial_sum(aug_sales#104), partial_sum(sep_sales#105), partial_sum(oct_sales#106), partial_sum(nov_sales#107), partial_sum(dec_sales#108), partial_sum((jan_sales#97 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((feb_sales#98 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((mar_sales#99 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((apr_sales#100 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((may_sales#101 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jun_sales#102 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jul_sales#103 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((aug_sales#104 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((sep_sales#105 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((oct_sales#106 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((nov_sales#107 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((dec_sales#108 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum(jan_net#109), partial_sum(feb_net#110), partial_sum(mar_net#111), partial_sum(apr_net#112), partial_sum(may_net#113), partial_sum(jun_net#114), partial_sum(jul_net#115), partial_sum(aug_net#116), partial_sum(sep_net#117), partial_sum(oct_net#118), partial_sum(nov_net#119), partial_sum(dec_net#120)] -Aggregate Attributes [72]: [sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284, sum#285, isEmpty#286, sum#287, isEmpty#288, sum#289, isEmpty#290, sum#291, isEmpty#292, sum#293, isEmpty#294, sum#295, isEmpty#296, sum#297, isEmpty#298, sum#299, isEmpty#300, sum#301, isEmpty#302, sum#303, isEmpty#304, sum#305, isEmpty#306, sum#307, isEmpty#308, sum#309, isEmpty#310, sum#311, isEmpty#312] -Results [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#95, year#96, sum#313, isEmpty#314, sum#315, isEmpty#316, sum#317, isEmpty#318, sum#319, isEmpty#320, sum#321, isEmpty#322, sum#323, isEmpty#324, sum#325, isEmpty#326, sum#327, isEmpty#328, sum#329, isEmpty#330, sum#331, isEmpty#332, sum#333, isEmpty#334, sum#335, isEmpty#336, sum#337, isEmpty#338, sum#339, isEmpty#340, sum#341, isEmpty#342, sum#343, isEmpty#344, sum#345, isEmpty#346, sum#347, isEmpty#348, sum#349, isEmpty#350, sum#351, isEmpty#352, sum#353, isEmpty#354, sum#355, isEmpty#356, sum#357, isEmpty#358, sum#359, isEmpty#360, sum#361, isEmpty#362, sum#363, isEmpty#364, sum#365, isEmpty#366, sum#367, isEmpty#368, sum#369, isEmpty#370, sum#371, isEmpty#372, sum#373, isEmpty#374, sum#375, isEmpty#376, sum#377, isEmpty#378, sum#379, isEmpty#380, sum#381, isEmpty#382, sum#383, isEmpty#384] +Aggregate Attributes [72]: [sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284, sum#285, isEmpty#286, sum#287, isEmpty#288, sum#289, isEmpty#290, sum#291, isEmpty#292, sum#293, isEmpty#294, sum#295, isEmpty#296, sum#297, isEmpty#298, sum#299, isEmpty#300, sum#301, isEmpty#302, sum#303, isEmpty#304, sum#305, isEmpty#306, sum#307, isEmpty#308, sum#309, isEmpty#310] +Results [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#95, year#96, sum#311, isEmpty#312, sum#313, isEmpty#314, sum#315, isEmpty#316, sum#317, isEmpty#318, sum#319, isEmpty#320, sum#321, isEmpty#322, sum#323, isEmpty#324, sum#325, isEmpty#326, sum#327, isEmpty#328, sum#329, isEmpty#330, sum#331, isEmpty#332, sum#333, isEmpty#334, sum#335, isEmpty#336, sum#337, isEmpty#338, sum#339, isEmpty#340, sum#341, isEmpty#342, sum#343, isEmpty#344, sum#345, isEmpty#346, sum#347, isEmpty#348, sum#349, isEmpty#350, sum#351, isEmpty#352, sum#353, isEmpty#354, sum#355, isEmpty#356, sum#357, isEmpty#358, sum#359, isEmpty#360, sum#361, isEmpty#362, sum#363, isEmpty#364, sum#365, isEmpty#366, sum#367, isEmpty#368, sum#369, isEmpty#370, sum#371, isEmpty#372, sum#373, isEmpty#374, sum#375, isEmpty#376, sum#377, isEmpty#378, sum#379, isEmpty#380, sum#381, isEmpty#382] -(59) Exchange -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#95, year#96, sum#313, isEmpty#314, sum#315, isEmpty#316, sum#317, isEmpty#318, sum#319, isEmpty#320, sum#321, isEmpty#322, sum#323, isEmpty#324, sum#325, isEmpty#326, sum#327, isEmpty#328, sum#329, isEmpty#330, sum#331, isEmpty#332, sum#333, isEmpty#334, sum#335, isEmpty#336, sum#337, isEmpty#338, sum#339, isEmpty#340, sum#341, isEmpty#342, sum#343, isEmpty#344, sum#345, isEmpty#346, sum#347, isEmpty#348, sum#349, isEmpty#350, sum#351, isEmpty#352, sum#353, isEmpty#354, sum#355, isEmpty#356, sum#357, isEmpty#358, sum#359, isEmpty#360, sum#361, isEmpty#362, sum#363, isEmpty#364, sum#365, isEmpty#366, sum#367, isEmpty#368, sum#369, isEmpty#370, sum#371, isEmpty#372, sum#373, isEmpty#374, sum#375, isEmpty#376, sum#377, isEmpty#378, sum#379, isEmpty#380, sum#381, isEmpty#382, sum#383, isEmpty#384] +(49) Exchange +Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#95, year#96, sum#311, isEmpty#312, sum#313, isEmpty#314, sum#315, isEmpty#316, sum#317, isEmpty#318, sum#319, isEmpty#320, sum#321, isEmpty#322, sum#323, isEmpty#324, sum#325, isEmpty#326, sum#327, isEmpty#328, sum#329, isEmpty#330, sum#331, isEmpty#332, sum#333, isEmpty#334, sum#335, isEmpty#336, sum#337, isEmpty#338, sum#339, isEmpty#340, sum#341, isEmpty#342, sum#343, isEmpty#344, sum#345, isEmpty#346, sum#347, isEmpty#348, sum#349, isEmpty#350, sum#351, isEmpty#352, sum#353, isEmpty#354, sum#355, isEmpty#356, sum#357, isEmpty#358, sum#359, isEmpty#360, sum#361, isEmpty#362, sum#363, isEmpty#364, sum#365, isEmpty#366, sum#367, isEmpty#368, sum#369, isEmpty#370, sum#371, isEmpty#372, sum#373, isEmpty#374, sum#375, isEmpty#376, sum#377, isEmpty#378, sum#379, isEmpty#380, sum#381, isEmpty#382] Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#95, year#96, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(60) HashAggregate [codegen id : 6] -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#95, year#96, sum#313, isEmpty#314, sum#315, isEmpty#316, sum#317, isEmpty#318, sum#319, isEmpty#320, sum#321, isEmpty#322, sum#323, isEmpty#324, sum#325, isEmpty#326, sum#327, isEmpty#328, sum#329, isEmpty#330, sum#331, isEmpty#332, sum#333, isEmpty#334, sum#335, isEmpty#336, sum#337, isEmpty#338, sum#339, isEmpty#340, sum#341, isEmpty#342, sum#343, isEmpty#344, sum#345, isEmpty#346, sum#347, isEmpty#348, sum#349, isEmpty#350, sum#351, isEmpty#352, sum#353, isEmpty#354, sum#355, isEmpty#356, sum#357, isEmpty#358, sum#359, isEmpty#360, sum#361, isEmpty#362, sum#363, isEmpty#364, sum#365, isEmpty#366, sum#367, isEmpty#368, sum#369, isEmpty#370, sum#371, isEmpty#372, sum#373, isEmpty#374, sum#375, isEmpty#376, sum#377, isEmpty#378, sum#379, isEmpty#380, sum#381, isEmpty#382, sum#383, isEmpty#384] +(50) HashAggregate [codegen id : 6] +Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#95, year#96, sum#311, isEmpty#312, sum#313, isEmpty#314, sum#315, isEmpty#316, sum#317, isEmpty#318, sum#319, isEmpty#320, sum#321, isEmpty#322, sum#323, isEmpty#324, sum#325, isEmpty#326, sum#327, isEmpty#328, sum#329, isEmpty#330, sum#331, isEmpty#332, sum#333, isEmpty#334, sum#335, isEmpty#336, sum#337, isEmpty#338, sum#339, isEmpty#340, sum#341, isEmpty#342, sum#343, isEmpty#344, sum#345, isEmpty#346, sum#347, isEmpty#348, sum#349, isEmpty#350, sum#351, isEmpty#352, sum#353, isEmpty#354, sum#355, isEmpty#356, sum#357, isEmpty#358, sum#359, isEmpty#360, sum#361, isEmpty#362, sum#363, isEmpty#364, sum#365, isEmpty#366, sum#367, isEmpty#368, sum#369, isEmpty#370, sum#371, isEmpty#372, sum#373, isEmpty#374, sum#375, isEmpty#376, sum#377, isEmpty#378, sum#379, isEmpty#380, sum#381, isEmpty#382] Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#95, year#96] Functions [36]: [sum(jan_sales#97), sum(feb_sales#98), sum(mar_sales#99), sum(apr_sales#100), sum(may_sales#101), sum(jun_sales#102), sum(jul_sales#103), sum(aug_sales#104), sum(sep_sales#105), sum(oct_sales#106), sum(nov_sales#107), sum(dec_sales#108), sum((jan_sales#97 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((feb_sales#98 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((mar_sales#99 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((apr_sales#100 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((may_sales#101 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jun_sales#102 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jul_sales#103 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((aug_sales#104 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((sep_sales#105 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((oct_sales#106 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((nov_sales#107 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((dec_sales#108 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum(jan_net#109), sum(feb_net#110), sum(mar_net#111), sum(apr_net#112), sum(may_net#113), sum(jun_net#114), sum(jul_net#115), sum(aug_net#116), sum(sep_net#117), sum(oct_net#118), sum(nov_net#119), sum(dec_net#120)] -Aggregate Attributes [36]: [sum(jan_sales#97)#385, sum(feb_sales#98)#386, sum(mar_sales#99)#387, sum(apr_sales#100)#388, sum(may_sales#101)#389, sum(jun_sales#102)#390, sum(jul_sales#103)#391, sum(aug_sales#104)#392, sum(sep_sales#105)#393, sum(oct_sales#106)#394, sum(nov_sales#107)#395, sum(dec_sales#108)#396, sum((jan_sales#97 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#397, sum((feb_sales#98 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#398, sum((mar_sales#99 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#399, sum((apr_sales#100 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#400, sum((may_sales#101 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#401, sum((jun_sales#102 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#402, sum((jul_sales#103 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#403, sum((aug_sales#104 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#404, sum((sep_sales#105 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#405, sum((oct_sales#106 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#406, sum((nov_sales#107 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#407, sum((dec_sales#108 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#408, sum(jan_net#109)#409, sum(feb_net#110)#410, sum(mar_net#111)#411, sum(apr_net#112)#412, sum(may_net#113)#413, sum(jun_net#114)#414, sum(jul_net#115)#415, sum(aug_net#116)#416, sum(sep_net#117)#417, sum(oct_net#118)#418, sum(nov_net#119)#419, sum(dec_net#120)#420] -Results [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#95, year#96, sum(jan_sales#97)#385 AS jan_sales#421, sum(feb_sales#98)#386 AS feb_sales#422, sum(mar_sales#99)#387 AS mar_sales#423, sum(apr_sales#100)#388 AS apr_sales#424, sum(may_sales#101)#389 AS may_sales#425, sum(jun_sales#102)#390 AS jun_sales#426, sum(jul_sales#103)#391 AS jul_sales#427, sum(aug_sales#104)#392 AS aug_sales#428, sum(sep_sales#105)#393 AS sep_sales#429, sum(oct_sales#106)#394 AS oct_sales#430, sum(nov_sales#107)#395 AS nov_sales#431, sum(dec_sales#108)#396 AS dec_sales#432, sum((jan_sales#97 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#397 AS jan_sales_per_sq_foot#433, sum((feb_sales#98 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#398 AS feb_sales_per_sq_foot#434, sum((mar_sales#99 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#399 AS mar_sales_per_sq_foot#435, sum((apr_sales#100 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#400 AS apr_sales_per_sq_foot#436, sum((may_sales#101 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#401 AS may_sales_per_sq_foot#437, sum((jun_sales#102 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#402 AS jun_sales_per_sq_foot#438, sum((jul_sales#103 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#403 AS jul_sales_per_sq_foot#439, sum((aug_sales#104 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#404 AS aug_sales_per_sq_foot#440, sum((sep_sales#105 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#405 AS sep_sales_per_sq_foot#441, sum((oct_sales#106 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#406 AS oct_sales_per_sq_foot#442, sum((nov_sales#107 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#407 AS nov_sales_per_sq_foot#443, sum((dec_sales#108 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#408 AS dec_sales_per_sq_foot#444, sum(jan_net#109)#409 AS jan_net#445, sum(feb_net#110)#410 AS feb_net#446, sum(mar_net#111)#411 AS mar_net#447, sum(apr_net#112)#412 AS apr_net#448, sum(may_net#113)#413 AS may_net#449, sum(jun_net#114)#414 AS jun_net#450, sum(jul_net#115)#415 AS jul_net#451, sum(aug_net#116)#416 AS aug_net#452, sum(sep_net#117)#417 AS sep_net#453, sum(oct_net#118)#418 AS oct_net#454, sum(nov_net#119)#419 AS nov_net#455, sum(dec_net#120)#420 AS dec_net#456] +Aggregate Attributes [36]: [sum(jan_sales#97)#383, sum(feb_sales#98)#384, sum(mar_sales#99)#385, sum(apr_sales#100)#386, sum(may_sales#101)#387, sum(jun_sales#102)#388, sum(jul_sales#103)#389, sum(aug_sales#104)#390, sum(sep_sales#105)#391, sum(oct_sales#106)#392, sum(nov_sales#107)#393, sum(dec_sales#108)#394, sum((jan_sales#97 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#395, sum((feb_sales#98 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#396, sum((mar_sales#99 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#397, sum((apr_sales#100 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#398, sum((may_sales#101 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#399, sum((jun_sales#102 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#400, sum((jul_sales#103 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#401, sum((aug_sales#104 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#402, sum((sep_sales#105 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#403, sum((oct_sales#106 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#404, sum((nov_sales#107 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#405, sum((dec_sales#108 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#406, sum(jan_net#109)#407, sum(feb_net#110)#408, sum(mar_net#111)#409, sum(apr_net#112)#410, sum(may_net#113)#411, sum(jun_net#114)#412, sum(jul_net#115)#413, sum(aug_net#116)#414, sum(sep_net#117)#415, sum(oct_net#118)#416, sum(nov_net#119)#417, sum(dec_net#120)#418] +Results [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#95, year#96, sum(jan_sales#97)#383 AS jan_sales#419, sum(feb_sales#98)#384 AS feb_sales#420, sum(mar_sales#99)#385 AS mar_sales#421, sum(apr_sales#100)#386 AS apr_sales#422, sum(may_sales#101)#387 AS may_sales#423, sum(jun_sales#102)#388 AS jun_sales#424, sum(jul_sales#103)#389 AS jul_sales#425, sum(aug_sales#104)#390 AS aug_sales#426, sum(sep_sales#105)#391 AS sep_sales#427, sum(oct_sales#106)#392 AS oct_sales#428, sum(nov_sales#107)#393 AS nov_sales#429, sum(dec_sales#108)#394 AS dec_sales#430, sum((jan_sales#97 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#395 AS jan_sales_per_sq_foot#431, sum((feb_sales#98 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#396 AS feb_sales_per_sq_foot#432, sum((mar_sales#99 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#397 AS mar_sales_per_sq_foot#433, sum((apr_sales#100 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#398 AS apr_sales_per_sq_foot#434, sum((may_sales#101 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#399 AS may_sales_per_sq_foot#435, sum((jun_sales#102 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#400 AS jun_sales_per_sq_foot#436, sum((jul_sales#103 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#401 AS jul_sales_per_sq_foot#437, sum((aug_sales#104 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#402 AS aug_sales_per_sq_foot#438, sum((sep_sales#105 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#403 AS sep_sales_per_sq_foot#439, sum((oct_sales#106 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#404 AS oct_sales_per_sq_foot#440, sum((nov_sales#107 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#405 AS nov_sales_per_sq_foot#441, sum((dec_sales#108 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#406 AS dec_sales_per_sq_foot#442, sum(jan_net#109)#407 AS jan_net#443, sum(feb_net#110)#408 AS feb_net#444, sum(mar_net#111)#409 AS mar_net#445, sum(apr_net#112)#410 AS apr_net#446, sum(may_net#113)#411 AS may_net#447, sum(jun_net#114)#412 AS jun_net#448, sum(jul_net#115)#413 AS jul_net#449, sum(aug_net#116)#414 AS aug_net#450, sum(sep_net#117)#415 AS sep_net#451, sum(oct_net#118)#416 AS oct_net#452, sum(nov_net#119)#417 AS nov_net#453, sum(dec_net#120)#418 AS dec_net#454] -(61) TakeOrderedAndProject -Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#95, year#96, jan_sales#421, feb_sales#422, mar_sales#423, apr_sales#424, may_sales#425, jun_sales#426, jul_sales#427, aug_sales#428, sep_sales#429, oct_sales#430, nov_sales#431, dec_sales#432, jan_sales_per_sq_foot#433, feb_sales_per_sq_foot#434, mar_sales_per_sq_foot#435, apr_sales_per_sq_foot#436, may_sales_per_sq_foot#437, jun_sales_per_sq_foot#438, jul_sales_per_sq_foot#439, aug_sales_per_sq_foot#440, sep_sales_per_sq_foot#441, oct_sales_per_sq_foot#442, nov_sales_per_sq_foot#443, dec_sales_per_sq_foot#444, jan_net#445, feb_net#446, mar_net#447, apr_net#448, may_net#449, jun_net#450, jul_net#451, aug_net#452, sep_net#453, oct_net#454, nov_net#455, dec_net#456] -Arguments: 100, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#95, year#96, jan_sales#421, feb_sales#422, mar_sales#423, apr_sales#424, may_sales#425, jun_sales#426, jul_sales#427, aug_sales#428, sep_sales#429, oct_sales#430, nov_sales#431, dec_sales#432, jan_sales_per_sq_foot#433, feb_sales_per_sq_foot#434, mar_sales_per_sq_foot#435, apr_sales_per_sq_foot#436, may_sales_per_sq_foot#437, jun_sales_per_sq_foot#438, jul_sales_per_sq_foot#439, aug_sales_per_sq_foot#440, sep_sales_per_sq_foot#441, oct_sales_per_sq_foot#442, nov_sales_per_sq_foot#443, dec_sales_per_sq_foot#444, jan_net#445, feb_net#446, mar_net#447, apr_net#448, may_net#449, jun_net#450, jul_net#451, aug_net#452, sep_net#453, oct_net#454, nov_net#455, dec_net#456] +(51) TakeOrderedAndProject +Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#95, year#96, jan_sales#419, feb_sales#420, mar_sales#421, apr_sales#422, may_sales#423, jun_sales#424, jul_sales#425, aug_sales#426, sep_sales#427, oct_sales#428, nov_sales#429, dec_sales#430, jan_sales_per_sq_foot#431, feb_sales_per_sq_foot#432, mar_sales_per_sq_foot#433, apr_sales_per_sq_foot#434, may_sales_per_sq_foot#435, jun_sales_per_sq_foot#436, jul_sales_per_sq_foot#437, aug_sales_per_sq_foot#438, sep_sales_per_sq_foot#439, oct_sales_per_sq_foot#440, nov_sales_per_sq_foot#441, dec_sales_per_sq_foot#442, jan_net#443, feb_net#444, mar_net#445, apr_net#446, may_net#447, jun_net#448, jul_net#449, aug_net#450, sep_net#451, oct_net#452, nov_net#453, dec_net#454] +Arguments: 100, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#95, year#96, jan_sales#419, feb_sales#420, mar_sales#421, apr_sales#422, may_sales#423, jun_sales#424, jul_sales#425, aug_sales#426, sep_sales#427, oct_sales#428, nov_sales#429, dec_sales#430, jan_sales_per_sq_foot#431, feb_sales_per_sq_foot#432, mar_sales_per_sq_foot#433, apr_sales_per_sq_foot#434, may_sales_per_sq_foot#435, jun_sales_per_sq_foot#436, jul_sales_per_sq_foot#437, aug_sales_per_sq_foot#438, sep_sales_per_sq_foot#439, oct_sales_per_sq_foot#440, nov_sales_per_sq_foot#441, dec_sales_per_sq_foot#442, jan_net#443, feb_net#444, mar_net#445, apr_net#446, may_net#447, jun_net#448, jul_net#449, aug_net#450, sep_net#451, oct_net#452, nov_net#453, dec_net#454] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (65) -+- * ColumnarToRow (64) - +- CometFilter (63) - +- CometScan parquet spark_catalog.default.date_dim (62) +BroadcastExchange (55) ++- * ColumnarToRow (54) + +- CometFilter (53) + +- CometScan parquet spark_catalog.default.date_dim (52) -(62) Scan parquet spark_catalog.default.date_dim +(52) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#16, d_year#17, d_moy#18] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(63) CometFilter +(53) CometFilter Input [3]: [d_date_sk#16, d_year#17, d_moy#18] Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) -(64) ColumnarToRow [codegen id : 1] +(54) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#16, d_year#17, d_moy#18] -(65) BroadcastExchange +(55) BroadcastExchange Input [3]: [d_date_sk#16, d_year#17, d_moy#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt index 7685723ce..d746739b3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt @@ -65,17 +65,7 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat CometFilter [cs_warehouse_sk,cs_sold_time_sk,cs_ship_mode_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #9 - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastExchange #10 - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #11 - CometProject [t_time_sk] - CometFilter [t_time,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] - CometBroadcastExchange #12 - CometProject [sm_ship_mode_sk] - CometFilter [sm_carrier,sm_ship_mode_sk] - CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] + ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 + ReusedExchange [d_date_sk,d_year,d_moy] #5 + ReusedExchange [t_time_sk] #6 + ReusedExchange [sm_ship_mode_sk] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt index df40e56fe..7d7b9efad 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt @@ -1,16 +1,16 @@ == Physical Plan == -TakeOrderedAndProject (46) -+- * HashAggregate (45) - +- Exchange (44) - +- * ColumnarToRow (43) - +- CometHashAggregate (42) - +- CometProject (41) - +- CometBroadcastHashJoin (40) - :- CometProject (36) - : +- CometBroadcastHashJoin (35) - : :- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometBroadcastHashJoin (20) +TakeOrderedAndProject (40) ++- * HashAggregate (39) + +- Exchange (38) + +- * ColumnarToRow (37) + +- CometHashAggregate (36) + +- CometProject (35) + +- CometBroadcastHashJoin (34) + :- CometProject (30) + : +- CometBroadcastHashJoin (29) + : :- CometProject (24) + : : +- CometBroadcastHashJoin (23) + : : :- CometBroadcastHashJoin (17) : : : :- CometBroadcastHashJoin (11) : : : : :- CometFilter (2) : : : : : +- CometScan parquet spark_catalog.default.customer (1) @@ -22,29 +22,23 @@ TakeOrderedAndProject (46) : : : : +- CometProject (6) : : : : +- CometFilter (5) : : : : +- CometScan parquet spark_catalog.default.date_dim (4) - : : : +- CometBroadcastExchange (19) - : : : +- CometProject (18) - : : : +- CometBroadcastHashJoin (17) + : : : +- CometBroadcastExchange (16) + : : : +- CometProject (15) + : : : +- CometBroadcastHashJoin (14) : : : :- CometScan parquet spark_catalog.default.web_sales (12) - : : : +- CometBroadcastExchange (16) - : : : +- CometProject (15) - : : : +- CometFilter (14) - : : : +- CometScan parquet spark_catalog.default.date_dim (13) - : : +- CometBroadcastExchange (28) - : : +- CometProject (27) - : : +- CometBroadcastHashJoin (26) - : : :- CometScan parquet spark_catalog.default.catalog_sales (21) - : : +- CometBroadcastExchange (25) - : : +- CometProject (24) - : : +- CometFilter (23) - : : +- CometScan parquet spark_catalog.default.date_dim (22) - : +- CometBroadcastExchange (34) - : +- CometProject (33) - : +- CometFilter (32) - : +- CometScan parquet spark_catalog.default.customer_address (31) - +- CometBroadcastExchange (39) - +- CometFilter (38) - +- CometScan parquet spark_catalog.default.customer_demographics (37) + : : : +- ReusedExchange (13) + : : +- CometBroadcastExchange (22) + : : +- CometProject (21) + : : +- CometBroadcastHashJoin (20) + : : :- CometScan parquet spark_catalog.default.catalog_sales (18) + : : +- ReusedExchange (19) + : +- CometBroadcastExchange (28) + : +- CometProject (27) + : +- CometFilter (26) + : +- CometScan parquet spark_catalog.default.customer_address (25) + +- CometBroadcastExchange (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.customer_demographics (31) (1) Scan parquet spark_catalog.default.customer @@ -109,200 +103,168 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] ReadSchema: struct -(13) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : (((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 >= 4)) AND (d_moy#15 <= 6)) AND isnotnull(d_date_sk#13)) - -(15) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(16) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] +(13) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#13] -(17) CometBroadcastHashJoin +(14) CometBroadcastHashJoin Left output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] Right output [1]: [d_date_sk#13] Arguments: [ws_sold_date_sk#11], [d_date_sk#13], Inner -(18) CometProject +(15) CometProject Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] Arguments: [ws_bill_customer_sk#10], [ws_bill_customer_sk#10] -(19) CometBroadcastExchange +(16) CometBroadcastExchange Input [1]: [ws_bill_customer_sk#10] Arguments: [ws_bill_customer_sk#10] -(20) CometBroadcastHashJoin +(17) CometBroadcastHashJoin Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Right output [1]: [ws_bill_customer_sk#10] Arguments: [c_customer_sk#1], [ws_bill_customer_sk#10], LeftAnti -(21) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +(18) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] +PartitionFilters: [isnotnull(cs_sold_date_sk#15), dynamicpruningexpression(cs_sold_date_sk#15 IN dynamicpruning#16)] ReadSchema: struct -(22) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#20, d_moy#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] -ReadSchema: struct - -(23) CometFilter -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Condition : (((((isnotnull(d_year#20) AND isnotnull(d_moy#21)) AND (d_year#20 = 2001)) AND (d_moy#21 >= 4)) AND (d_moy#21 <= 6)) AND isnotnull(d_date_sk#19)) - -(24) CometProject -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(25) CometBroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: [d_date_sk#19] +(19) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#17] -(26) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#19] -Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner +(20) CometBroadcastHashJoin +Left output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] +Right output [1]: [d_date_sk#17] +Arguments: [cs_sold_date_sk#15], [d_date_sk#17], Inner -(27) CometProject -Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] -Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] +(21) CometProject +Input [3]: [cs_ship_customer_sk#14, cs_sold_date_sk#15, d_date_sk#17] +Arguments: [cs_ship_customer_sk#14], [cs_ship_customer_sk#14] -(28) CometBroadcastExchange -Input [1]: [cs_ship_customer_sk#16] -Arguments: [cs_ship_customer_sk#16] +(22) CometBroadcastExchange +Input [1]: [cs_ship_customer_sk#14] +Arguments: [cs_ship_customer_sk#14] -(29) CometBroadcastHashJoin +(23) CometBroadcastHashJoin Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [cs_ship_customer_sk#16] -Arguments: [c_customer_sk#1], [cs_ship_customer_sk#16], LeftAnti +Right output [1]: [cs_ship_customer_sk#14] +Arguments: [c_customer_sk#1], [cs_ship_customer_sk#14], LeftAnti -(30) CometProject +(24) CometProject Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Arguments: [c_current_cdemo_sk#2, c_current_addr_sk#3], [c_current_cdemo_sk#2, c_current_addr_sk#3] -(31) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#22, ca_state#23] +(25) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#18, ca_state#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [GA,KY,NM]), IsNotNull(ca_address_sk)] ReadSchema: struct -(32) CometFilter -Input [2]: [ca_address_sk#22, ca_state#23] -Condition : (ca_state#23 IN (KY,GA,NM) AND isnotnull(ca_address_sk#22)) +(26) CometFilter +Input [2]: [ca_address_sk#18, ca_state#19] +Condition : (ca_state#19 IN (KY,GA,NM) AND isnotnull(ca_address_sk#18)) -(33) CometProject -Input [2]: [ca_address_sk#22, ca_state#23] -Arguments: [ca_address_sk#22], [ca_address_sk#22] +(27) CometProject +Input [2]: [ca_address_sk#18, ca_state#19] +Arguments: [ca_address_sk#18], [ca_address_sk#18] -(34) CometBroadcastExchange -Input [1]: [ca_address_sk#22] -Arguments: [ca_address_sk#22] +(28) CometBroadcastExchange +Input [1]: [ca_address_sk#18] +Arguments: [ca_address_sk#18] -(35) CometBroadcastHashJoin +(29) CometBroadcastHashJoin Left output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [ca_address_sk#22] -Arguments: [c_current_addr_sk#3], [ca_address_sk#22], Inner +Right output [1]: [ca_address_sk#18] +Arguments: [c_current_addr_sk#3], [ca_address_sk#18], Inner -(36) CometProject -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#22] +(30) CometProject +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#18] Arguments: [c_current_cdemo_sk#2], [c_current_cdemo_sk#2] -(37) Scan parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29] +(31) Scan parquet spark_catalog.default.customer_demographics +Output [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(38) CometFilter -Input [6]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29] -Condition : isnotnull(cd_demo_sk#24) +(32) CometFilter +Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] +Condition : isnotnull(cd_demo_sk#20) -(39) CometBroadcastExchange -Input [6]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29] -Arguments: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29] +(33) CometBroadcastExchange +Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] +Arguments: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] -(40) CometBroadcastHashJoin +(34) CometBroadcastHashJoin Left output [1]: [c_current_cdemo_sk#2] -Right output [6]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29] -Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#24], Inner +Right output [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] +Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#20], Inner -(41) CometProject -Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29] -Arguments: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29], [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29] +(35) CometProject +Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] +Arguments: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25], [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] -(42) CometHashAggregate -Input [5]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29] -Keys [5]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29] +(36) CometHashAggregate +Input [5]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] +Keys [5]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] Functions [1]: [partial_count(1)] -(43) ColumnarToRow [codegen id : 1] -Input [6]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, count#30] +(37) ColumnarToRow [codegen id : 1] +Input [6]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, count#26] -(44) Exchange -Input [6]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, count#30] -Arguments: hashpartitioning(cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, 5), ENSURE_REQUIREMENTS, [plan_id=1] +(38) Exchange +Input [6]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, count#26] +Arguments: hashpartitioning(cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(45) HashAggregate [codegen id : 2] -Input [6]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, count#30] -Keys [5]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29] +(39) HashAggregate [codegen id : 2] +Input [6]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, count#26] +Keys [5]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#31] -Results [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, count(1)#31 AS cnt1#32, cd_purchase_estimate#28, count(1)#31 AS cnt2#33, cd_credit_rating#29, count(1)#31 AS cnt3#34] +Aggregate Attributes [1]: [count(1)#27] +Results [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, count(1)#27 AS cnt1#28, cd_purchase_estimate#24, count(1)#27 AS cnt2#29, cd_credit_rating#25, count(1)#27 AS cnt3#30] -(46) TakeOrderedAndProject -Input [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#32, cd_purchase_estimate#28, cnt2#33, cd_credit_rating#29, cnt3#34] -Arguments: 100, [cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_education_status#27 ASC NULLS FIRST, cd_purchase_estimate#28 ASC NULLS FIRST, cd_credit_rating#29 ASC NULLS FIRST], [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#32, cd_purchase_estimate#28, cnt2#33, cd_credit_rating#29, cnt3#34] +(40) TakeOrderedAndProject +Input [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#28, cd_purchase_estimate#24, cnt2#29, cd_credit_rating#25, cnt3#30] +Arguments: 100, [cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_education_status#23 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#25 ASC NULLS FIRST], [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#28, cd_purchase_estimate#24, cnt2#29, cd_credit_rating#25, cnt3#30] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (51) -+- * ColumnarToRow (50) - +- CometProject (49) - +- CometFilter (48) - +- CometScan parquet spark_catalog.default.date_dim (47) +BroadcastExchange (45) ++- * ColumnarToRow (44) + +- CometProject (43) + +- CometFilter (42) + +- CometScan parquet spark_catalog.default.date_dim (41) -(47) Scan parquet spark_catalog.default.date_dim +(41) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#7, d_year#8, d_moy#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] ReadSchema: struct -(48) CometFilter +(42) CometFilter Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2001)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 6)) AND isnotnull(d_date_sk#7)) -(49) CometProject +(43) CometProject Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(50) ColumnarToRow [codegen id : 1] +(44) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(51) BroadcastExchange +(45) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt index fffd06c01..a9bd92acf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt @@ -38,23 +38,17 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #6 - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #7 + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange #6 CometProject [cs_ship_customer_sk] CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #8 - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #9 + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange #7 CometProject [ca_address_sk] CometFilter [ca_state,ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange #10 + CometBroadcastExchange #8 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt index 7329fe6c0..f3229b97a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt @@ -1,15 +1,15 @@ == Physical Plan == -TakeOrderedAndProject (47) -+- * Project (46) - +- Window (45) - +- * Sort (44) - +- Exchange (43) - +- * HashAggregate (42) - +- Exchange (41) - +- * HashAggregate (40) - +- * Expand (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) +TakeOrderedAndProject (44) ++- * Project (43) + +- Window (42) + +- * Sort (41) + +- Exchange (40) + +- * HashAggregate (39) + +- Exchange (38) + +- * HashAggregate (37) + +- * Expand (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) :- * ColumnarToRow (9) : +- CometProject (8) : +- CometBroadcastHashJoin (7) @@ -19,22 +19,22 @@ TakeOrderedAndProject (47) : +- CometProject (5) : +- CometFilter (4) : +- CometScan parquet spark_catalog.default.date_dim (3) - +- BroadcastExchange (36) - +- * BroadcastHashJoin LeftSemi BuildRight (35) + +- BroadcastExchange (33) + +- * BroadcastHashJoin LeftSemi BuildRight (32) :- * ColumnarToRow (12) : +- CometFilter (11) : +- CometScan parquet spark_catalog.default.store (10) - +- BroadcastExchange (34) - +- * Project (33) - +- * Filter (32) - +- Window (31) - +- * Sort (30) - +- * HashAggregate (29) - +- Exchange (28) - +- * ColumnarToRow (27) - +- CometHashAggregate (26) - +- CometProject (25) - +- CometBroadcastHashJoin (24) + +- BroadcastExchange (31) + +- * Project (30) + +- * Filter (29) + +- Window (28) + +- * Sort (27) + +- * HashAggregate (26) + +- Exchange (25) + +- * ColumnarToRow (24) + +- CometHashAggregate (23) + +- CometProject (22) + +- CometBroadcastHashJoin (21) :- CometProject (19) : +- CometBroadcastHashJoin (18) : :- CometFilter (14) @@ -42,10 +42,7 @@ TakeOrderedAndProject (47) : +- CometBroadcastExchange (17) : +- CometFilter (16) : +- CometScan parquet spark_catalog.default.store (15) - +- CometBroadcastExchange (23) - +- CometProject (22) - +- CometFilter (21) - +- CometScan parquet spark_catalog.default.date_dim (20) + +- ReusedExchange (20) (1) Scan parquet spark_catalog.default.store_sales @@ -141,164 +138,148 @@ Arguments: [ss_store_sk#10], [s_store_sk#14], Inner Input [5]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12, s_store_sk#14, s_state#15] Arguments: [ss_net_profit#11, ss_sold_date_sk#12, s_state#15], [ss_net_profit#11, ss_sold_date_sk#12, s_state#15] -(20) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#16, d_month_seq#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [d_date_sk#16, d_month_seq#17] -Condition : (((isnotnull(d_month_seq#17) AND (d_month_seq#17 >= 1200)) AND (d_month_seq#17 <= 1211)) AND isnotnull(d_date_sk#16)) +(20) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#16] -(22) CometProject -Input [2]: [d_date_sk#16, d_month_seq#17] -Arguments: [d_date_sk#16], [d_date_sk#16] - -(23) CometBroadcastExchange -Input [1]: [d_date_sk#16] -Arguments: [d_date_sk#16] - -(24) CometBroadcastHashJoin +(21) CometBroadcastHashJoin Left output [3]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#15] Right output [1]: [d_date_sk#16] Arguments: [ss_sold_date_sk#12], [d_date_sk#16], Inner -(25) CometProject +(22) CometProject Input [4]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#15, d_date_sk#16] Arguments: [ss_net_profit#11, s_state#15], [ss_net_profit#11, s_state#15] -(26) CometHashAggregate +(23) CometHashAggregate Input [2]: [ss_net_profit#11, s_state#15] Keys [1]: [s_state#15] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] -(27) ColumnarToRow [codegen id : 1] -Input [2]: [s_state#15, sum#18] +(24) ColumnarToRow [codegen id : 1] +Input [2]: [s_state#15, sum#17] -(28) Exchange -Input [2]: [s_state#15, sum#18] +(25) Exchange +Input [2]: [s_state#15, sum#17] Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(29) HashAggregate [codegen id : 2] -Input [2]: [s_state#15, sum#18] +(26) HashAggregate [codegen id : 2] +Input [2]: [s_state#15, sum#17] Keys [1]: [s_state#15] Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#11))#19] -Results [3]: [s_state#15, MakeDecimal(sum(UnscaledValue(ss_net_profit#11))#19,17,2) AS _w0#20, s_state#15] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#11))#18] +Results [3]: [s_state#15, MakeDecimal(sum(UnscaledValue(ss_net_profit#11))#18,17,2) AS _w0#19, s_state#15] -(30) Sort [codegen id : 2] -Input [3]: [s_state#15, _w0#20, s_state#15] -Arguments: [s_state#15 ASC NULLS FIRST, _w0#20 DESC NULLS LAST], false, 0 +(27) Sort [codegen id : 2] +Input [3]: [s_state#15, _w0#19, s_state#15] +Arguments: [s_state#15 ASC NULLS FIRST, _w0#19 DESC NULLS LAST], false, 0 -(31) Window -Input [3]: [s_state#15, _w0#20, s_state#15] -Arguments: [rank(_w0#20) windowspecdefinition(s_state#15, _w0#20 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#21], [s_state#15], [_w0#20 DESC NULLS LAST] +(28) Window +Input [3]: [s_state#15, _w0#19, s_state#15] +Arguments: [rank(_w0#19) windowspecdefinition(s_state#15, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#15], [_w0#19 DESC NULLS LAST] -(32) Filter [codegen id : 3] -Input [4]: [s_state#15, _w0#20, s_state#15, ranking#21] -Condition : (ranking#21 <= 5) +(29) Filter [codegen id : 3] +Input [4]: [s_state#15, _w0#19, s_state#15, ranking#20] +Condition : (ranking#20 <= 5) -(33) Project [codegen id : 3] +(30) Project [codegen id : 3] Output [1]: [s_state#15] -Input [4]: [s_state#15, _w0#20, s_state#15, ranking#21] +Input [4]: [s_state#15, _w0#19, s_state#15, ranking#20] -(34) BroadcastExchange +(31) BroadcastExchange Input [1]: [s_state#15] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] -(35) BroadcastHashJoin [codegen id : 4] +(32) BroadcastHashJoin [codegen id : 4] Left keys [1]: [s_state#9] Right keys [1]: [s_state#15] Join type: LeftSemi Join condition: None -(36) BroadcastExchange +(33) BroadcastExchange Input [3]: [s_store_sk#7, s_county#8, s_state#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(37) BroadcastHashJoin [codegen id : 5] +(34) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_store_sk#1] Right keys [1]: [s_store_sk#7] Join type: Inner Join condition: None -(38) Project [codegen id : 5] +(35) Project [codegen id : 5] Output [3]: [ss_net_profit#2, s_state#9, s_county#8] Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#9] -(39) Expand [codegen id : 5] +(36) Expand [codegen id : 5] Input [3]: [ss_net_profit#2, s_state#9, s_county#8] -Arguments: [[ss_net_profit#2, s_state#9, s_county#8, 0], [ss_net_profit#2, s_state#9, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] +Arguments: [[ss_net_profit#2, s_state#9, s_county#8, 0], [ss_net_profit#2, s_state#9, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#21, s_county#22, spark_grouping_id#23] -(40) HashAggregate [codegen id : 5] -Input [4]: [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] -Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] +(37) HashAggregate [codegen id : 5] +Input [4]: [ss_net_profit#2, s_state#21, s_county#22, spark_grouping_id#23] +Keys [3]: [s_state#21, s_county#22, spark_grouping_id#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#25] -Results [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] +Aggregate Attributes [1]: [sum#24] +Results [4]: [s_state#21, s_county#22, spark_grouping_id#23, sum#25] -(41) Exchange -Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] -Arguments: hashpartitioning(s_state#22, s_county#23, spark_grouping_id#24, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(38) Exchange +Input [4]: [s_state#21, s_county#22, spark_grouping_id#23, sum#25] +Arguments: hashpartitioning(s_state#21, s_county#22, spark_grouping_id#23, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(42) HashAggregate [codegen id : 6] -Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] -Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] +(39) HashAggregate [codegen id : 6] +Input [4]: [s_state#21, s_county#22, spark_grouping_id#23, sum#25] +Keys [3]: [s_state#21, s_county#22, spark_grouping_id#23] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#27] -Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#27,17,2) AS total_sum#28, s_state#22, s_county#23, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS lochierarchy#29, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#27,17,2) AS _w0#30, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS _w1#31, CASE WHEN (cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint) = 0) THEN s_state#22 END AS _w2#32] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#26] +Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#26,17,2) AS total_sum#27, s_state#21, s_county#22, (cast((shiftright(spark_grouping_id#23, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#23, 0) & 1) as tinyint)) AS lochierarchy#28, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#26,17,2) AS _w0#29, (cast((shiftright(spark_grouping_id#23, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#23, 0) & 1) as tinyint)) AS _w1#30, CASE WHEN (cast((shiftright(spark_grouping_id#23, 0) & 1) as tinyint) = 0) THEN s_state#21 END AS _w2#31] -(43) Exchange -Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] -Arguments: hashpartitioning(_w1#31, _w2#32, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(40) Exchange +Input [7]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, _w0#29, _w1#30, _w2#31] +Arguments: hashpartitioning(_w1#30, _w2#31, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(44) Sort [codegen id : 7] -Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] -Arguments: [_w1#31 ASC NULLS FIRST, _w2#32 ASC NULLS FIRST, _w0#30 DESC NULLS LAST], false, 0 +(41) Sort [codegen id : 7] +Input [7]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, _w0#29, _w1#30, _w2#31] +Arguments: [_w1#30 ASC NULLS FIRST, _w2#31 ASC NULLS FIRST, _w0#29 DESC NULLS LAST], false, 0 -(45) Window -Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] -Arguments: [rank(_w0#30) windowspecdefinition(_w1#31, _w2#32, _w0#30 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#33], [_w1#31, _w2#32], [_w0#30 DESC NULLS LAST] +(42) Window +Input [7]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, _w0#29, _w1#30, _w2#31] +Arguments: [rank(_w0#29) windowspecdefinition(_w1#30, _w2#31, _w0#29 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#32], [_w1#30, _w2#31], [_w0#29 DESC NULLS LAST] -(46) Project [codegen id : 8] -Output [5]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, rank_within_parent#33] -Input [8]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32, rank_within_parent#33] +(43) Project [codegen id : 8] +Output [5]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, rank_within_parent#32] +Input [8]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, _w0#29, _w1#30, _w2#31, rank_within_parent#32] -(47) TakeOrderedAndProject -Input [5]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, rank_within_parent#33] -Arguments: 100, [lochierarchy#29 DESC NULLS LAST, CASE WHEN (lochierarchy#29 = 0) THEN s_state#22 END ASC NULLS FIRST, rank_within_parent#33 ASC NULLS FIRST], [total_sum#28, s_state#22, s_county#23, lochierarchy#29, rank_within_parent#33] +(44) TakeOrderedAndProject +Input [5]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, rank_within_parent#32] +Arguments: 100, [lochierarchy#28 DESC NULLS LAST, CASE WHEN (lochierarchy#28 = 0) THEN s_state#21 END ASC NULLS FIRST, rank_within_parent#32 ASC NULLS FIRST], [total_sum#27, s_state#21, s_county#22, lochierarchy#28, rank_within_parent#32] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (52) -+- * ColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan parquet spark_catalog.default.date_dim (48) +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan parquet spark_catalog.default.date_dim (45) -(48) Scan parquet spark_catalog.default.date_dim +(45) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#5, d_month_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(49) CometFilter +(46) CometFilter Input [2]: [d_date_sk#5, d_month_seq#6] Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) -(50) CometProject +(47) CometProject Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(51) ColumnarToRow [codegen id : 1] +(48) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(52) BroadcastExchange +(49) BroadcastExchange Input [1]: [d_date_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt index 57f21a2a4..f4a03a2d2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt @@ -68,7 +68,4 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count CometBroadcastExchange #8 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange #9 - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt index 27cab8cf8..2fa583dc2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt @@ -1,19 +1,19 @@ == Physical Plan == -* Sort (43) -+- Exchange (42) - +- * HashAggregate (41) - +- Exchange (40) - +- * ColumnarToRow (39) - +- CometHashAggregate (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (31) - : +- CometBroadcastHashJoin (30) +* Sort (37) ++- Exchange (36) + +- * HashAggregate (35) + +- Exchange (34) + +- * ColumnarToRow (33) + +- CometHashAggregate (32) + +- CometProject (31) + +- CometBroadcastHashJoin (30) + :- CometProject (25) + : +- CometBroadcastHashJoin (24) : :- CometBroadcastExchange (4) : : +- CometProject (3) : : +- CometFilter (2) : : +- CometScan parquet spark_catalog.default.item (1) - : +- CometUnion (29) + : +- CometUnion (23) : :- CometProject (12) : : +- CometBroadcastHashJoin (11) : : :- CometFilter (6) @@ -22,26 +22,20 @@ : : +- CometProject (9) : : +- CometFilter (8) : : +- CometScan parquet spark_catalog.default.date_dim (7) - : :- CometProject (20) - : : +- CometBroadcastHashJoin (19) + : :- CometProject (17) + : : +- CometBroadcastHashJoin (16) : : :- CometFilter (14) : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) - : : +- CometBroadcastExchange (18) - : : +- CometProject (17) - : : +- CometFilter (16) - : : +- CometScan parquet spark_catalog.default.date_dim (15) - : +- CometProject (28) - : +- CometBroadcastHashJoin (27) - : :- CometFilter (22) - : : +- CometScan parquet spark_catalog.default.store_sales (21) - : +- CometBroadcastExchange (26) - : +- CometProject (25) - : +- CometFilter (24) - : +- CometScan parquet spark_catalog.default.date_dim (23) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan parquet spark_catalog.default.time_dim (32) + : : +- ReusedExchange (15) + : +- CometProject (22) + : +- CometBroadcastHashJoin (21) + : :- CometFilter (19) + : : +- CometScan parquet spark_catalog.default.store_sales (18) + : +- ReusedExchange (20) + +- CometBroadcastExchange (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan parquet spark_catalog.default.time_dim (26) (1) Scan parquet spark_catalog.default.item @@ -115,177 +109,145 @@ ReadSchema: struct - -(16) CometFilter -Input [3]: [d_date_sk#21, d_year#22, d_moy#23] -Condition : ((((isnotnull(d_moy#23) AND isnotnull(d_year#22)) AND (d_moy#23 = 11)) AND (d_year#22 = 1999)) AND isnotnull(d_date_sk#21)) - -(17) CometProject -Input [3]: [d_date_sk#21, d_year#22, d_moy#23] -Arguments: [d_date_sk#21], [d_date_sk#21] - -(18) CometBroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: [d_date_sk#21] +(15) ReusedExchange [Reuses operator id: 10] +Output [1]: [d_date_sk#21] -(19) CometBroadcastHashJoin +(16) CometBroadcastHashJoin Left output [4]: [cs_sold_time_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] Right output [1]: [d_date_sk#21] Arguments: [cs_sold_date_sk#19], [d_date_sk#21], Inner -(20) CometProject +(17) CometProject Input [5]: [cs_sold_time_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19, d_date_sk#21] -Arguments: [ext_price#24, sold_item_sk#25, time_sk#26], [cs_ext_sales_price#18 AS ext_price#24, cs_item_sk#17 AS sold_item_sk#25, cs_sold_time_sk#16 AS time_sk#26] +Arguments: [ext_price#22, sold_item_sk#23, time_sk#24], [cs_ext_sales_price#18 AS ext_price#22, cs_item_sk#17 AS sold_item_sk#23, cs_sold_time_sk#16 AS time_sk#24] -(21) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#27, ss_item_sk#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +(18) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#30), dynamicpruningexpression(ss_sold_date_sk#30 IN dynamicpruning#31)] +PartitionFilters: [isnotnull(ss_sold_date_sk#28), dynamicpruningexpression(ss_sold_date_sk#28 IN dynamicpruning#29)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_time_sk)] ReadSchema: struct -(22) CometFilter -Input [4]: [ss_sold_time_sk#27, ss_item_sk#28, ss_ext_sales_price#29, ss_sold_date_sk#30] -Condition : (isnotnull(ss_item_sk#28) AND isnotnull(ss_sold_time_sk#27)) - -(23) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#32, d_year#33, d_moy#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(24) CometFilter -Input [3]: [d_date_sk#32, d_year#33, d_moy#34] -Condition : ((((isnotnull(d_moy#34) AND isnotnull(d_year#33)) AND (d_moy#34 = 11)) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) +(19) CometFilter +Input [4]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] +Condition : (isnotnull(ss_item_sk#26) AND isnotnull(ss_sold_time_sk#25)) -(25) CometProject -Input [3]: [d_date_sk#32, d_year#33, d_moy#34] -Arguments: [d_date_sk#32], [d_date_sk#32] - -(26) CometBroadcastExchange -Input [1]: [d_date_sk#32] -Arguments: [d_date_sk#32] +(20) ReusedExchange [Reuses operator id: 10] +Output [1]: [d_date_sk#30] -(27) CometBroadcastHashJoin -Left output [4]: [ss_sold_time_sk#27, ss_item_sk#28, ss_ext_sales_price#29, ss_sold_date_sk#30] -Right output [1]: [d_date_sk#32] -Arguments: [ss_sold_date_sk#30], [d_date_sk#32], Inner +(21) CometBroadcastHashJoin +Left output [4]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] +Right output [1]: [d_date_sk#30] +Arguments: [ss_sold_date_sk#28], [d_date_sk#30], Inner -(28) CometProject -Input [5]: [ss_sold_time_sk#27, ss_item_sk#28, ss_ext_sales_price#29, ss_sold_date_sk#30, d_date_sk#32] -Arguments: [ext_price#35, sold_item_sk#36, time_sk#37], [ss_ext_sales_price#29 AS ext_price#35, ss_item_sk#28 AS sold_item_sk#36, ss_sold_time_sk#27 AS time_sk#37] +(22) CometProject +Input [5]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#30] +Arguments: [ext_price#31, sold_item_sk#32, time_sk#33], [ss_ext_sales_price#27 AS ext_price#31, ss_item_sk#26 AS sold_item_sk#32, ss_sold_time_sk#25 AS time_sk#33] -(29) CometUnion +(23) CometUnion Child 0 Input [3]: [ext_price#13, sold_item_sk#14, time_sk#15] -Child 1 Input [3]: [ext_price#24, sold_item_sk#25, time_sk#26] -Child 2 Input [3]: [ext_price#35, sold_item_sk#36, time_sk#37] +Child 1 Input [3]: [ext_price#22, sold_item_sk#23, time_sk#24] +Child 2 Input [3]: [ext_price#31, sold_item_sk#32, time_sk#33] -(30) CometBroadcastHashJoin +(24) CometBroadcastHashJoin Left output [3]: [i_item_sk#1, i_brand_id#2, i_brand#3] Right output [3]: [ext_price#13, sold_item_sk#14, time_sk#15] Arguments: [i_item_sk#1], [sold_item_sk#14], Inner -(31) CometProject +(25) CometProject Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#3, ext_price#13, sold_item_sk#14, time_sk#15] Arguments: [i_brand_id#2, i_brand#3, ext_price#13, time_sk#15], [i_brand_id#2, i_brand#3, ext_price#13, time_sk#15] -(32) Scan parquet spark_catalog.default.time_dim -Output [4]: [t_time_sk#38, t_hour#39, t_minute#40, t_meal_time#41] +(26) Scan parquet spark_catalog.default.time_dim +Output [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [Or(EqualTo(t_meal_time,breakfast ),EqualTo(t_meal_time,dinner )), IsNotNull(t_time_sk)] ReadSchema: struct -(33) CometFilter -Input [4]: [t_time_sk#38, t_hour#39, t_minute#40, t_meal_time#41] -Condition : (((t_meal_time#41 = breakfast ) OR (t_meal_time#41 = dinner )) AND isnotnull(t_time_sk#38)) +(27) CometFilter +Input [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] +Condition : (((t_meal_time#37 = breakfast ) OR (t_meal_time#37 = dinner )) AND isnotnull(t_time_sk#34)) -(34) CometProject -Input [4]: [t_time_sk#38, t_hour#39, t_minute#40, t_meal_time#41] -Arguments: [t_time_sk#38, t_hour#39, t_minute#40], [t_time_sk#38, t_hour#39, t_minute#40] +(28) CometProject +Input [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] +Arguments: [t_time_sk#34, t_hour#35, t_minute#36], [t_time_sk#34, t_hour#35, t_minute#36] -(35) CometBroadcastExchange -Input [3]: [t_time_sk#38, t_hour#39, t_minute#40] -Arguments: [t_time_sk#38, t_hour#39, t_minute#40] +(29) CometBroadcastExchange +Input [3]: [t_time_sk#34, t_hour#35, t_minute#36] +Arguments: [t_time_sk#34, t_hour#35, t_minute#36] -(36) CometBroadcastHashJoin +(30) CometBroadcastHashJoin Left output [4]: [i_brand_id#2, i_brand#3, ext_price#13, time_sk#15] -Right output [3]: [t_time_sk#38, t_hour#39, t_minute#40] -Arguments: [time_sk#15], [t_time_sk#38], Inner +Right output [3]: [t_time_sk#34, t_hour#35, t_minute#36] +Arguments: [time_sk#15], [t_time_sk#34], Inner -(37) CometProject -Input [7]: [i_brand_id#2, i_brand#3, ext_price#13, time_sk#15, t_time_sk#38, t_hour#39, t_minute#40] -Arguments: [i_brand_id#2, i_brand#3, ext_price#13, t_hour#39, t_minute#40], [i_brand_id#2, i_brand#3, ext_price#13, t_hour#39, t_minute#40] +(31) CometProject +Input [7]: [i_brand_id#2, i_brand#3, ext_price#13, time_sk#15, t_time_sk#34, t_hour#35, t_minute#36] +Arguments: [i_brand_id#2, i_brand#3, ext_price#13, t_hour#35, t_minute#36], [i_brand_id#2, i_brand#3, ext_price#13, t_hour#35, t_minute#36] -(38) CometHashAggregate -Input [5]: [i_brand_id#2, i_brand#3, ext_price#13, t_hour#39, t_minute#40] -Keys [4]: [i_brand#3, i_brand_id#2, t_hour#39, t_minute#40] +(32) CometHashAggregate +Input [5]: [i_brand_id#2, i_brand#3, ext_price#13, t_hour#35, t_minute#36] +Keys [4]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36] Functions [1]: [partial_sum(UnscaledValue(ext_price#13))] -(39) ColumnarToRow [codegen id : 1] -Input [5]: [i_brand#3, i_brand_id#2, t_hour#39, t_minute#40, sum#42] +(33) ColumnarToRow [codegen id : 1] +Input [5]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, sum#38] -(40) Exchange -Input [5]: [i_brand#3, i_brand_id#2, t_hour#39, t_minute#40, sum#42] -Arguments: hashpartitioning(i_brand#3, i_brand_id#2, t_hour#39, t_minute#40, 5), ENSURE_REQUIREMENTS, [plan_id=1] +(34) Exchange +Input [5]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, sum#38] +Arguments: hashpartitioning(i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(41) HashAggregate [codegen id : 2] -Input [5]: [i_brand#3, i_brand_id#2, t_hour#39, t_minute#40, sum#42] -Keys [4]: [i_brand#3, i_brand_id#2, t_hour#39, t_minute#40] +(35) HashAggregate [codegen id : 2] +Input [5]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, sum#38] +Keys [4]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36] Functions [1]: [sum(UnscaledValue(ext_price#13))] -Aggregate Attributes [1]: [sum(UnscaledValue(ext_price#13))#43] -Results [5]: [i_brand_id#2 AS brand_id#44, i_brand#3 AS brand#45, t_hour#39, t_minute#40, MakeDecimal(sum(UnscaledValue(ext_price#13))#43,17,2) AS ext_price#46] +Aggregate Attributes [1]: [sum(UnscaledValue(ext_price#13))#39] +Results [5]: [i_brand_id#2 AS brand_id#40, i_brand#3 AS brand#41, t_hour#35, t_minute#36, MakeDecimal(sum(UnscaledValue(ext_price#13))#39,17,2) AS ext_price#42] -(42) Exchange -Input [5]: [brand_id#44, brand#45, t_hour#39, t_minute#40, ext_price#46] -Arguments: rangepartitioning(ext_price#46 DESC NULLS LAST, brand_id#44 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(36) Exchange +Input [5]: [brand_id#40, brand#41, t_hour#35, t_minute#36, ext_price#42] +Arguments: rangepartitioning(ext_price#42 DESC NULLS LAST, brand_id#40 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(43) Sort [codegen id : 3] -Input [5]: [brand_id#44, brand#45, t_hour#39, t_minute#40, ext_price#46] -Arguments: [ext_price#46 DESC NULLS LAST, brand_id#44 ASC NULLS FIRST], true, 0 +(37) Sort [codegen id : 3] +Input [5]: [brand_id#40, brand#41, t_hour#35, t_minute#36, ext_price#42] +Arguments: [ext_price#42 DESC NULLS LAST, brand_id#40 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 5 Hosting Expression = ws_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (48) -+- * ColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan parquet spark_catalog.default.date_dim (44) +BroadcastExchange (42) ++- * ColumnarToRow (41) + +- CometProject (40) + +- CometFilter (39) + +- CometScan parquet spark_catalog.default.date_dim (38) -(44) Scan parquet spark_catalog.default.date_dim +(38) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#10, d_year#11, d_moy#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct -(45) CometFilter +(39) CometFilter Input [3]: [d_date_sk#10, d_year#11, d_moy#12] Condition : ((((isnotnull(d_moy#12) AND isnotnull(d_year#11)) AND (d_moy#12 = 11)) AND (d_year#11 = 1999)) AND isnotnull(d_date_sk#10)) -(46) CometProject +(40) CometProject Input [3]: [d_date_sk#10, d_year#11, d_moy#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(47) ColumnarToRow [codegen id : 1] +(41) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(48) BroadcastExchange +(42) BroadcastExchange Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] Subquery:2 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 18 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt index 782fdff4c..fd6777886 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt @@ -40,20 +40,14 @@ WholeStageCodegen (3) CometFilter [cs_item_sk,cs_sold_time_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #6 - CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + ReusedExchange [d_date_sk] #5 CometProject [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] [ext_price,sold_item_sk,time_sk] CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_item_sk,ss_sold_time_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #7 - CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #8 + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange #6 CometProject [t_time_sk,t_hour,t_minute] CometFilter [t_meal_time,t_time_sk] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt index 432ade2eb..b786a6896 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (75) -+- * Project (74) - +- * BroadcastHashJoin Inner BuildRight (73) - :- * Project (55) - : +- * BroadcastHashJoin Inner BuildRight (54) +TakeOrderedAndProject (71) ++- * Project (70) + +- * BroadcastHashJoin Inner BuildRight (69) + :- * Project (53) + : +- * BroadcastHashJoin Inner BuildRight (52) : :- * BroadcastHashJoin Inner BuildRight (35) : : :- * Filter (17) : : : +- * HashAggregate (16) @@ -39,14 +39,14 @@ TakeOrderedAndProject (75) : : +- CometBroadcastExchange (27) : : +- CometFilter (26) : : +- CometScan parquet spark_catalog.default.date_dim (25) - : +- BroadcastExchange (53) - : +- * Filter (52) - : +- * HashAggregate (51) - : +- Exchange (50) - : +- * ColumnarToRow (49) - : +- CometHashAggregate (48) - : +- CometProject (47) - : +- CometBroadcastHashJoin (46) + : +- BroadcastExchange (51) + : +- * Filter (50) + : +- * HashAggregate (49) + : +- Exchange (48) + : +- * ColumnarToRow (47) + : +- CometHashAggregate (46) + : +- CometProject (45) + : +- CometBroadcastHashJoin (44) : :- CometProject (42) : : +- CometBroadcastHashJoin (41) : : :- CometFilter (37) @@ -54,26 +54,22 @@ TakeOrderedAndProject (75) : : +- CometBroadcastExchange (40) : : +- CometFilter (39) : : +- CometScan parquet spark_catalog.default.web_sales (38) - : +- CometBroadcastExchange (45) - : +- CometFilter (44) - : +- CometScan parquet spark_catalog.default.date_dim (43) - +- BroadcastExchange (72) - +- * HashAggregate (71) - +- Exchange (70) - +- * ColumnarToRow (69) - +- CometHashAggregate (68) - +- CometProject (67) - +- CometBroadcastHashJoin (66) - :- CometProject (62) - : +- CometBroadcastHashJoin (61) - : :- CometFilter (57) - : : +- CometScan parquet spark_catalog.default.customer (56) - : +- CometBroadcastExchange (60) - : +- CometFilter (59) - : +- CometScan parquet spark_catalog.default.web_sales (58) - +- CometBroadcastExchange (65) - +- CometFilter (64) - +- CometScan parquet spark_catalog.default.date_dim (63) + : +- ReusedExchange (43) + +- BroadcastExchange (68) + +- * HashAggregate (67) + +- Exchange (66) + +- * ColumnarToRow (65) + +- CometHashAggregate (64) + +- CometProject (63) + +- CometBroadcastHashJoin (62) + :- CometProject (60) + : +- CometBroadcastHashJoin (59) + : :- CometFilter (55) + : : +- CometScan parquet spark_catalog.default.customer (54) + : +- CometBroadcastExchange (58) + : +- CometFilter (57) + : +- CometScan parquet spark_catalog.default.web_sales (56) + +- ReusedExchange (61) (1) Scan parquet spark_catalog.default.customer @@ -284,79 +280,67 @@ Arguments: [c_customer_sk#30], [ws_bill_customer_sk#34], Inner Input [7]: [c_customer_sk#30, c_customer_id#31, c_first_name#32, c_last_name#33, ws_bill_customer_sk#34, ws_net_paid#35, ws_sold_date_sk#36] Arguments: [c_customer_id#31, c_first_name#32, c_last_name#33, ws_net_paid#35, ws_sold_date_sk#36], [c_customer_id#31, c_first_name#32, c_last_name#33, ws_net_paid#35, ws_sold_date_sk#36] -(43) Scan parquet spark_catalog.default.date_dim +(43) ReusedExchange [Reuses operator id: 10] Output [2]: [d_date_sk#38, d_year#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct -(44) CometFilter -Input [2]: [d_date_sk#38, d_year#39] -Condition : (((isnotnull(d_year#39) AND (d_year#39 = 2001)) AND d_year#39 IN (2001,2002)) AND isnotnull(d_date_sk#38)) - -(45) CometBroadcastExchange -Input [2]: [d_date_sk#38, d_year#39] -Arguments: [d_date_sk#38, d_year#39] - -(46) CometBroadcastHashJoin +(44) CometBroadcastHashJoin Left output [5]: [c_customer_id#31, c_first_name#32, c_last_name#33, ws_net_paid#35, ws_sold_date_sk#36] Right output [2]: [d_date_sk#38, d_year#39] Arguments: [ws_sold_date_sk#36], [d_date_sk#38], Inner -(47) CometProject +(45) CometProject Input [7]: [c_customer_id#31, c_first_name#32, c_last_name#33, ws_net_paid#35, ws_sold_date_sk#36, d_date_sk#38, d_year#39] Arguments: [c_customer_id#31, c_first_name#32, c_last_name#33, ws_net_paid#35, d_year#39], [c_customer_id#31, c_first_name#32, c_last_name#33, ws_net_paid#35, d_year#39] -(48) CometHashAggregate +(46) CometHashAggregate Input [5]: [c_customer_id#31, c_first_name#32, c_last_name#33, ws_net_paid#35, d_year#39] Keys [4]: [c_customer_id#31, c_first_name#32, c_last_name#33, d_year#39] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#35))] -(49) ColumnarToRow [codegen id : 4] +(47) ColumnarToRow [codegen id : 4] Input [5]: [c_customer_id#31, c_first_name#32, c_last_name#33, d_year#39, sum#40] -(50) Exchange +(48) Exchange Input [5]: [c_customer_id#31, c_first_name#32, c_last_name#33, d_year#39, sum#40] Arguments: hashpartitioning(c_customer_id#31, c_first_name#32, c_last_name#33, d_year#39, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(51) HashAggregate [codegen id : 5] +(49) HashAggregate [codegen id : 5] Input [5]: [c_customer_id#31, c_first_name#32, c_last_name#33, d_year#39, sum#40] Keys [4]: [c_customer_id#31, c_first_name#32, c_last_name#33, d_year#39] Functions [1]: [sum(UnscaledValue(ws_net_paid#35))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#35))#41] Results [2]: [c_customer_id#31 AS customer_id#42, MakeDecimal(sum(UnscaledValue(ws_net_paid#35))#41,17,2) AS year_total#43] -(52) Filter [codegen id : 5] +(50) Filter [codegen id : 5] Input [2]: [customer_id#42, year_total#43] Condition : (isnotnull(year_total#43) AND (year_total#43 > 0.00)) -(53) BroadcastExchange +(51) BroadcastExchange Input [2]: [customer_id#42, year_total#43] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(54) BroadcastHashJoin [codegen id : 8] +(52) BroadcastHashJoin [codegen id : 8] Left keys [1]: [customer_id#13] Right keys [1]: [customer_id#42] Join type: Inner Join condition: None -(55) Project [codegen id : 8] +(53) Project [codegen id : 8] Output [7]: [customer_id#13, year_total#14, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, year_total#43] Input [8]: [customer_id#13, year_total#14, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, customer_id#42, year_total#43] -(56) Scan parquet spark_catalog.default.customer +(54) Scan parquet spark_catalog.default.customer Output [4]: [c_customer_sk#44, c_customer_id#45, c_first_name#46, c_last_name#47] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(57) CometFilter +(55) CometFilter Input [4]: [c_customer_sk#44, c_customer_id#45, c_first_name#46, c_last_name#47] Condition : (isnotnull(c_customer_sk#44) AND isnotnull(c_customer_id#45)) -(58) Scan parquet spark_catalog.default.web_sales +(56) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] Batched: true Location: InMemoryFileIndex [] @@ -364,138 +348,126 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#50), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(59) CometFilter +(57) CometFilter Input [3]: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] Condition : isnotnull(ws_bill_customer_sk#48) -(60) CometBroadcastExchange +(58) CometBroadcastExchange Input [3]: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] Arguments: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] -(61) CometBroadcastHashJoin +(59) CometBroadcastHashJoin Left output [4]: [c_customer_sk#44, c_customer_id#45, c_first_name#46, c_last_name#47] Right output [3]: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] Arguments: [c_customer_sk#44], [ws_bill_customer_sk#48], Inner -(62) CometProject +(60) CometProject Input [7]: [c_customer_sk#44, c_customer_id#45, c_first_name#46, c_last_name#47, ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] Arguments: [c_customer_id#45, c_first_name#46, c_last_name#47, ws_net_paid#49, ws_sold_date_sk#50], [c_customer_id#45, c_first_name#46, c_last_name#47, ws_net_paid#49, ws_sold_date_sk#50] -(63) Scan parquet spark_catalog.default.date_dim +(61) ReusedExchange [Reuses operator id: 27] Output [2]: [d_date_sk#52, d_year#53] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct -(64) CometFilter -Input [2]: [d_date_sk#52, d_year#53] -Condition : (((isnotnull(d_year#53) AND (d_year#53 = 2002)) AND d_year#53 IN (2001,2002)) AND isnotnull(d_date_sk#52)) - -(65) CometBroadcastExchange -Input [2]: [d_date_sk#52, d_year#53] -Arguments: [d_date_sk#52, d_year#53] - -(66) CometBroadcastHashJoin +(62) CometBroadcastHashJoin Left output [5]: [c_customer_id#45, c_first_name#46, c_last_name#47, ws_net_paid#49, ws_sold_date_sk#50] Right output [2]: [d_date_sk#52, d_year#53] Arguments: [ws_sold_date_sk#50], [d_date_sk#52], Inner -(67) CometProject +(63) CometProject Input [7]: [c_customer_id#45, c_first_name#46, c_last_name#47, ws_net_paid#49, ws_sold_date_sk#50, d_date_sk#52, d_year#53] Arguments: [c_customer_id#45, c_first_name#46, c_last_name#47, ws_net_paid#49, d_year#53], [c_customer_id#45, c_first_name#46, c_last_name#47, ws_net_paid#49, d_year#53] -(68) CometHashAggregate +(64) CometHashAggregate Input [5]: [c_customer_id#45, c_first_name#46, c_last_name#47, ws_net_paid#49, d_year#53] Keys [4]: [c_customer_id#45, c_first_name#46, c_last_name#47, d_year#53] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#49))] -(69) ColumnarToRow [codegen id : 6] +(65) ColumnarToRow [codegen id : 6] Input [5]: [c_customer_id#45, c_first_name#46, c_last_name#47, d_year#53, sum#54] -(70) Exchange +(66) Exchange Input [5]: [c_customer_id#45, c_first_name#46, c_last_name#47, d_year#53, sum#54] Arguments: hashpartitioning(c_customer_id#45, c_first_name#46, c_last_name#47, d_year#53, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(71) HashAggregate [codegen id : 7] +(67) HashAggregate [codegen id : 7] Input [5]: [c_customer_id#45, c_first_name#46, c_last_name#47, d_year#53, sum#54] Keys [4]: [c_customer_id#45, c_first_name#46, c_last_name#47, d_year#53] Functions [1]: [sum(UnscaledValue(ws_net_paid#49))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#49))#41] Results [2]: [c_customer_id#45 AS customer_id#55, MakeDecimal(sum(UnscaledValue(ws_net_paid#49))#41,17,2) AS year_total#56] -(72) BroadcastExchange +(68) BroadcastExchange Input [2]: [customer_id#55, year_total#56] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] -(73) BroadcastHashJoin [codegen id : 8] +(69) BroadcastHashJoin [codegen id : 8] Left keys [1]: [customer_id#13] Right keys [1]: [customer_id#55] Join type: Inner Join condition: (CASE WHEN (year_total#43 > 0.00) THEN (year_total#56 / year_total#43) END > CASE WHEN (year_total#14 > 0.00) THEN (year_total#29 / year_total#14) END) -(74) Project [codegen id : 8] +(70) Project [codegen id : 8] Output [3]: [customer_id#26, customer_first_name#27, customer_last_name#28] Input [9]: [customer_id#13, year_total#14, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, year_total#43, customer_id#55, year_total#56] -(75) TakeOrderedAndProject +(71) TakeOrderedAndProject Input [3]: [customer_id#26, customer_first_name#27, customer_last_name#28] Arguments: 100, [customer_id#26 ASC NULLS FIRST, customer_id#26 ASC NULLS FIRST, customer_id#26 ASC NULLS FIRST], [customer_id#26, customer_first_name#27, customer_last_name#28] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (79) -+- * ColumnarToRow (78) - +- CometFilter (77) - +- CometScan parquet spark_catalog.default.date_dim (76) +BroadcastExchange (75) ++- * ColumnarToRow (74) + +- CometFilter (73) + +- CometScan parquet spark_catalog.default.date_dim (72) -(76) Scan parquet spark_catalog.default.date_dim +(72) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#9, d_year#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct -(77) CometFilter +(73) CometFilter Input [2]: [d_date_sk#9, d_year#10] Condition : (((isnotnull(d_year#10) AND (d_year#10 = 2001)) AND d_year#10 IN (2001,2002)) AND isnotnull(d_date_sk#9)) -(78) ColumnarToRow [codegen id : 1] +(74) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_year#10] -(79) BroadcastExchange +(75) BroadcastExchange Input [2]: [d_date_sk#9, d_year#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#21 IN dynamicpruning#22 -BroadcastExchange (83) -+- * ColumnarToRow (82) - +- CometFilter (81) - +- CometScan parquet spark_catalog.default.date_dim (80) +BroadcastExchange (79) ++- * ColumnarToRow (78) + +- CometFilter (77) + +- CometScan parquet spark_catalog.default.date_dim (76) -(80) Scan parquet spark_catalog.default.date_dim +(76) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#23, d_year#24] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct -(81) CometFilter +(77) CometFilter Input [2]: [d_date_sk#23, d_year#24] Condition : (((isnotnull(d_year#24) AND (d_year#24 = 2002)) AND d_year#24 IN (2001,2002)) AND isnotnull(d_date_sk#23)) -(82) ColumnarToRow [codegen id : 1] +(78) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#23, d_year#24] -(83) BroadcastExchange +(79) BroadcastExchange Input [2]: [d_date_sk#23, d_year#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] Subquery:3 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#8 -Subquery:4 Hosting operator id = 58 Hosting Expression = ws_sold_date_sk#50 IN dynamicpruning#22 +Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#50 IN dynamicpruning#22 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt index d39cd9415..2e6286d93 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt @@ -82,15 +82,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] CometFilter [ws_bill_customer_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #13 - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk,d_year] #4 InputAdapter - BroadcastExchange #14 + BroadcastExchange #13 WholeStageCodegen (7) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year] #15 + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #14 WholeStageCodegen (6) ColumnarToRow InputAdapter @@ -101,10 +99,8 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #16 + CometBroadcastExchange #15 CometFilter [ws_bill_customer_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #17 - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk,d_year] #9 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 89a6b21db..d3fd3d97c 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,17 +1,17 @@ == Physical Plan == -TakeOrderedAndProject (155) -+- * Project (154) - +- * SortMergeJoin Inner (153) - :- * Sort (82) - : +- Exchange (81) - : +- * Filter (80) - : +- * HashAggregate (79) - : +- Exchange (78) - : +- * HashAggregate (77) - : +- * HashAggregate (76) - : +- Exchange (75) - : +- * HashAggregate (74) - : +- Union (73) +TakeOrderedAndProject (132) ++- * Project (131) + +- * SortMergeJoin Inner (130) + :- * Sort (72) + : +- Exchange (71) + : +- * Filter (70) + : +- * HashAggregate (69) + : +- Exchange (68) + : +- * HashAggregate (67) + : +- * HashAggregate (66) + : +- Exchange (65) + : +- * HashAggregate (64) + : +- Union (63) : :- * Project (24) : : +- * SortMergeJoin LeftOuter (23) : : :- * Sort (16) @@ -36,124 +36,101 @@ TakeOrderedAndProject (155) : : +- CometProject (19) : : +- CometFilter (18) : : +- CometScan parquet spark_catalog.default.catalog_returns (17) - : :- * Project (48) - : : +- * SortMergeJoin LeftOuter (47) - : : :- * Sort (40) - : : : +- Exchange (39) - : : : +- * ColumnarToRow (38) - : : : +- CometProject (37) - : : : +- CometBroadcastHashJoin (36) - : : : :- CometProject (32) - : : : : +- CometBroadcastHashJoin (31) + : :- * Project (43) + : : +- * SortMergeJoin LeftOuter (42) + : : :- * Sort (35) + : : : +- Exchange (34) + : : : +- * ColumnarToRow (33) + : : : +- CometProject (32) + : : : +- CometBroadcastHashJoin (31) + : : : :- CometProject (29) + : : : : +- CometBroadcastHashJoin (28) : : : : :- CometFilter (26) : : : : : +- CometScan parquet spark_catalog.default.store_sales (25) - : : : : +- CometBroadcastExchange (30) - : : : : +- CometProject (29) - : : : : +- CometFilter (28) - : : : : +- CometScan parquet spark_catalog.default.item (27) - : : : +- CometBroadcastExchange (35) - : : : +- CometFilter (34) - : : : +- CometScan parquet spark_catalog.default.date_dim (33) - : : +- * Sort (46) - : : +- Exchange (45) - : : +- * ColumnarToRow (44) - : : +- CometProject (43) - : : +- CometFilter (42) - : : +- CometScan parquet spark_catalog.default.store_returns (41) - : +- * Project (72) - : +- * SortMergeJoin LeftOuter (71) - : :- * Sort (64) - : : +- Exchange (63) - : : +- * ColumnarToRow (62) - : : +- CometProject (61) - : : +- CometBroadcastHashJoin (60) - : : :- CometProject (56) - : : : +- CometBroadcastHashJoin (55) - : : : :- CometFilter (50) - : : : : +- CometScan parquet spark_catalog.default.web_sales (49) - : : : +- CometBroadcastExchange (54) - : : : +- CometProject (53) - : : : +- CometFilter (52) - : : : +- CometScan parquet spark_catalog.default.item (51) - : : +- CometBroadcastExchange (59) - : : +- CometFilter (58) - : : +- CometScan parquet spark_catalog.default.date_dim (57) - : +- * Sort (70) - : +- Exchange (69) - : +- * ColumnarToRow (68) - : +- CometProject (67) - : +- CometFilter (66) - : +- CometScan parquet spark_catalog.default.web_returns (65) - +- * Sort (152) - +- Exchange (151) - +- * Filter (150) - +- * HashAggregate (149) - +- Exchange (148) - +- * HashAggregate (147) - +- * HashAggregate (146) - +- Exchange (145) - +- * HashAggregate (144) - +- Union (143) - :- * Project (102) - : +- * SortMergeJoin LeftOuter (101) - : :- * Sort (98) - : : +- Exchange (97) - : : +- * ColumnarToRow (96) - : : +- CometProject (95) - : : +- CometBroadcastHashJoin (94) - : : :- CometProject (90) - : : : +- CometBroadcastHashJoin (89) - : : : :- CometFilter (84) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (83) - : : : +- CometBroadcastExchange (88) - : : : +- CometProject (87) - : : : +- CometFilter (86) - : : : +- CometScan parquet spark_catalog.default.item (85) - : : +- CometBroadcastExchange (93) - : : +- CometFilter (92) - : : +- CometScan parquet spark_catalog.default.date_dim (91) - : +- * Sort (100) - : +- ReusedExchange (99) - :- * Project (122) - : +- * SortMergeJoin LeftOuter (121) - : :- * Sort (118) - : : +- Exchange (117) - : : +- * ColumnarToRow (116) - : : +- CometProject (115) - : : +- CometBroadcastHashJoin (114) - : : :- CometProject (110) - : : : +- CometBroadcastHashJoin (109) - : : : :- CometFilter (104) - : : : : +- CometScan parquet spark_catalog.default.store_sales (103) - : : : +- CometBroadcastExchange (108) - : : : +- CometProject (107) - : : : +- CometFilter (106) - : : : +- CometScan parquet spark_catalog.default.item (105) - : : +- CometBroadcastExchange (113) - : : +- CometFilter (112) - : : +- CometScan parquet spark_catalog.default.date_dim (111) - : +- * Sort (120) - : +- ReusedExchange (119) - +- * Project (142) - +- * SortMergeJoin LeftOuter (141) - :- * Sort (138) - : +- Exchange (137) - : +- * ColumnarToRow (136) - : +- CometProject (135) - : +- CometBroadcastHashJoin (134) - : :- CometProject (130) - : : +- CometBroadcastHashJoin (129) - : : :- CometFilter (124) - : : : +- CometScan parquet spark_catalog.default.web_sales (123) - : : +- CometBroadcastExchange (128) - : : +- CometProject (127) - : : +- CometFilter (126) - : : +- CometScan parquet spark_catalog.default.item (125) - : +- CometBroadcastExchange (133) - : +- CometFilter (132) - : +- CometScan parquet spark_catalog.default.date_dim (131) - +- * Sort (140) - +- ReusedExchange (139) + : : : : +- ReusedExchange (27) + : : : +- ReusedExchange (30) + : : +- * Sort (41) + : : +- Exchange (40) + : : +- * ColumnarToRow (39) + : : +- CometProject (38) + : : +- CometFilter (37) + : : +- CometScan parquet spark_catalog.default.store_returns (36) + : +- * Project (62) + : +- * SortMergeJoin LeftOuter (61) + : :- * Sort (54) + : : +- Exchange (53) + : : +- * ColumnarToRow (52) + : : +- CometProject (51) + : : +- CometBroadcastHashJoin (50) + : : :- CometProject (48) + : : : +- CometBroadcastHashJoin (47) + : : : :- CometFilter (45) + : : : : +- CometScan parquet spark_catalog.default.web_sales (44) + : : : +- ReusedExchange (46) + : : +- ReusedExchange (49) + : +- * Sort (60) + : +- Exchange (59) + : +- * ColumnarToRow (58) + : +- CometProject (57) + : +- CometFilter (56) + : +- CometScan parquet spark_catalog.default.web_returns (55) + +- * Sort (129) + +- Exchange (128) + +- * Filter (127) + +- * HashAggregate (126) + +- Exchange (125) + +- * HashAggregate (124) + +- * HashAggregate (123) + +- Exchange (122) + +- * HashAggregate (121) + +- Union (120) + :- * Project (89) + : +- * SortMergeJoin LeftOuter (88) + : :- * Sort (85) + : : +- Exchange (84) + : : +- * ColumnarToRow (83) + : : +- CometProject (82) + : : +- CometBroadcastHashJoin (81) + : : :- CometProject (77) + : : : +- CometBroadcastHashJoin (76) + : : : :- CometFilter (74) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (73) + : : : +- ReusedExchange (75) + : : +- CometBroadcastExchange (80) + : : +- CometFilter (79) + : : +- CometScan parquet spark_catalog.default.date_dim (78) + : +- * Sort (87) + : +- ReusedExchange (86) + :- * Project (104) + : +- * SortMergeJoin LeftOuter (103) + : :- * Sort (100) + : : +- Exchange (99) + : : +- * ColumnarToRow (98) + : : +- CometProject (97) + : : +- CometBroadcastHashJoin (96) + : : :- CometProject (94) + : : : +- CometBroadcastHashJoin (93) + : : : :- CometFilter (91) + : : : : +- CometScan parquet spark_catalog.default.store_sales (90) + : : : +- ReusedExchange (92) + : : +- ReusedExchange (95) + : +- * Sort (102) + : +- ReusedExchange (101) + +- * Project (119) + +- * SortMergeJoin LeftOuter (118) + :- * Sort (115) + : +- Exchange (114) + : +- * ColumnarToRow (113) + : +- CometProject (112) + : +- CometBroadcastHashJoin (111) + : :- CometProject (109) + : : +- CometBroadcastHashJoin (108) + : : :- CometFilter (106) + : : : +- CometScan parquet spark_catalog.default.web_sales (105) + : : +- ReusedExchange (107) + : +- ReusedExchange (110) + +- * Sort (117) + +- ReusedExchange (116) (1) Scan parquet spark_catalog.default.catalog_sales @@ -279,664 +256,536 @@ ReadSchema: struct +(27) ReusedExchange [Reuses operator id: 6] +Output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -(28) CometFilter -Input [6]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_category#32, i_manufact_id#33] -Condition : ((((((isnotnull(i_category#32) AND (i_category#32 = Books )) AND isnotnull(i_item_sk#28)) AND isnotnull(i_brand_id#29)) AND isnotnull(i_class_id#30)) AND isnotnull(i_category_id#31)) AND isnotnull(i_manufact_id#33)) +(28) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] +Right output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] +Arguments: [ss_item_sk#22], [i_item_sk#28], Inner (29) CometProject -Input [6]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_category#32, i_manufact_id#33] -Arguments: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#33], [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#33] +Input [10]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] +Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -(30) CometBroadcastExchange -Input [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#33] -Arguments: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#33] +(30) ReusedExchange [Reuses operator id: 11] +Output [2]: [d_date_sk#33, d_year#34] (31) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Right output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#33] -Arguments: [ss_item_sk#22], [i_item_sk#28], Inner +Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] +Right output [2]: [d_date_sk#33, d_year#34] +Arguments: [ss_sold_date_sk#26], [d_date_sk#33], Inner (32) CometProject -Input [10]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#33] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#33], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#33] - -(33) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#34, d_year#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [d_date_sk#34, d_year#35] -Condition : ((isnotnull(d_year#35) AND (d_year#35 = 2002)) AND isnotnull(d_date_sk#34)) - -(35) CometBroadcastExchange -Input [2]: [d_date_sk#34, d_year#35] -Arguments: [d_date_sk#34, d_year#35] - -(36) CometBroadcastHashJoin -Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#33] -Right output [2]: [d_date_sk#34, d_year#35] -Arguments: [ss_sold_date_sk#26], [d_date_sk#34], Inner - -(37) CometProject -Input [11]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#33, d_date_sk#34, d_year#35] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#33, d_year#35], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#33, d_year#35] +Input [11]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_date_sk#33, d_year#34] +Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -(38) ColumnarToRow [codegen id : 6] -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#33, d_year#35] +(33) ColumnarToRow [codegen id : 6] +Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -(39) Exchange -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#33, d_year#35] +(34) Exchange +Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] Arguments: hashpartitioning(ss_ticket_number#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(40) Sort [codegen id : 7] -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#33, d_year#35] +(35) Sort [codegen id : 7] +Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] Arguments: [ss_ticket_number#23 ASC NULLS FIRST, ss_item_sk#22 ASC NULLS FIRST], false, 0 -(41) Scan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#36, sr_ticket_number#37, sr_return_quantity#38, sr_return_amt#39, sr_returned_date_sk#40] +(36) Scan parquet spark_catalog.default.store_returns +Output [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(42) CometFilter -Input [5]: [sr_item_sk#36, sr_ticket_number#37, sr_return_quantity#38, sr_return_amt#39, sr_returned_date_sk#40] -Condition : (isnotnull(sr_ticket_number#37) AND isnotnull(sr_item_sk#36)) +(37) CometFilter +Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] +Condition : (isnotnull(sr_ticket_number#36) AND isnotnull(sr_item_sk#35)) -(43) CometProject -Input [5]: [sr_item_sk#36, sr_ticket_number#37, sr_return_quantity#38, sr_return_amt#39, sr_returned_date_sk#40] -Arguments: [sr_item_sk#36, sr_ticket_number#37, sr_return_quantity#38, sr_return_amt#39], [sr_item_sk#36, sr_ticket_number#37, sr_return_quantity#38, sr_return_amt#39] +(38) CometProject +Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] +Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -(44) ColumnarToRow [codegen id : 8] -Input [4]: [sr_item_sk#36, sr_ticket_number#37, sr_return_quantity#38, sr_return_amt#39] +(39) ColumnarToRow [codegen id : 8] +Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -(45) Exchange -Input [4]: [sr_item_sk#36, sr_ticket_number#37, sr_return_quantity#38, sr_return_amt#39] -Arguments: hashpartitioning(sr_ticket_number#37, sr_item_sk#36, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(40) Exchange +Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] +Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(46) Sort [codegen id : 9] -Input [4]: [sr_item_sk#36, sr_ticket_number#37, sr_return_quantity#38, sr_return_amt#39] -Arguments: [sr_ticket_number#37 ASC NULLS FIRST, sr_item_sk#36 ASC NULLS FIRST], false, 0 +(41) Sort [codegen id : 9] +Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] +Arguments: [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST], false, 0 -(47) SortMergeJoin [codegen id : 10] +(42) SortMergeJoin [codegen id : 10] Left keys [2]: [ss_ticket_number#23, ss_item_sk#22] -Right keys [2]: [sr_ticket_number#37, sr_item_sk#36] +Right keys [2]: [sr_ticket_number#36, sr_item_sk#35] Join type: LeftOuter Join condition: None -(48) Project [codegen id : 10] -Output [7]: [d_year#35, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#33, (ss_quantity#24 - coalesce(sr_return_quantity#38, 0)) AS sales_cnt#41, (ss_ext_sales_price#25 - coalesce(sr_return_amt#39, 0.00)) AS sales_amt#42] -Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#33, d_year#35, sr_item_sk#36, sr_ticket_number#37, sr_return_quantity#38, sr_return_amt#39] +(43) Project [codegen id : 10] +Output [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, (ss_quantity#24 - coalesce(sr_return_quantity#37, 0)) AS sales_cnt#40, (ss_ext_sales_price#25 - coalesce(sr_return_amt#38, 0.00)) AS sales_amt#41] +Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34, sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -(49) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#43, ws_order_number#44, ws_quantity#45, ws_ext_sales_price#46, ws_sold_date_sk#47] +(44) Scan parquet spark_catalog.default.web_sales +Output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#48)] +PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_sold_date_sk#46 IN dynamicpruning#47)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(50) CometFilter -Input [5]: [ws_item_sk#43, ws_order_number#44, ws_quantity#45, ws_ext_sales_price#46, ws_sold_date_sk#47] -Condition : isnotnull(ws_item_sk#43) +(45) CometFilter +Input [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] +Condition : isnotnull(ws_item_sk#42) -(51) Scan parquet spark_catalog.default.item -Output [6]: [i_item_sk#49, i_brand_id#50, i_class_id#51, i_category_id#52, i_category#53, i_manufact_id#54] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books ), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] -ReadSchema: struct - -(52) CometFilter -Input [6]: [i_item_sk#49, i_brand_id#50, i_class_id#51, i_category_id#52, i_category#53, i_manufact_id#54] -Condition : ((((((isnotnull(i_category#53) AND (i_category#53 = Books )) AND isnotnull(i_item_sk#49)) AND isnotnull(i_brand_id#50)) AND isnotnull(i_class_id#51)) AND isnotnull(i_category_id#52)) AND isnotnull(i_manufact_id#54)) - -(53) CometProject -Input [6]: [i_item_sk#49, i_brand_id#50, i_class_id#51, i_category_id#52, i_category#53, i_manufact_id#54] -Arguments: [i_item_sk#49, i_brand_id#50, i_class_id#51, i_category_id#52, i_manufact_id#54], [i_item_sk#49, i_brand_id#50, i_class_id#51, i_category_id#52, i_manufact_id#54] - -(54) CometBroadcastExchange -Input [5]: [i_item_sk#49, i_brand_id#50, i_class_id#51, i_category_id#52, i_manufact_id#54] -Arguments: [i_item_sk#49, i_brand_id#50, i_class_id#51, i_category_id#52, i_manufact_id#54] - -(55) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#43, ws_order_number#44, ws_quantity#45, ws_ext_sales_price#46, ws_sold_date_sk#47] -Right output [5]: [i_item_sk#49, i_brand_id#50, i_class_id#51, i_category_id#52, i_manufact_id#54] -Arguments: [ws_item_sk#43], [i_item_sk#49], Inner +(46) ReusedExchange [Reuses operator id: 6] +Output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -(56) CometProject -Input [10]: [ws_item_sk#43, ws_order_number#44, ws_quantity#45, ws_ext_sales_price#46, ws_sold_date_sk#47, i_item_sk#49, i_brand_id#50, i_class_id#51, i_category_id#52, i_manufact_id#54] -Arguments: [ws_item_sk#43, ws_order_number#44, ws_quantity#45, ws_ext_sales_price#46, ws_sold_date_sk#47, i_brand_id#50, i_class_id#51, i_category_id#52, i_manufact_id#54], [ws_item_sk#43, ws_order_number#44, ws_quantity#45, ws_ext_sales_price#46, ws_sold_date_sk#47, i_brand_id#50, i_class_id#51, i_category_id#52, i_manufact_id#54] +(47) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] +Right output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] +Arguments: [ws_item_sk#42], [i_item_sk#48], Inner -(57) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#55, d_year#56] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct +(48) CometProject +Input [10]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] +Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -(58) CometFilter -Input [2]: [d_date_sk#55, d_year#56] -Condition : ((isnotnull(d_year#56) AND (d_year#56 = 2002)) AND isnotnull(d_date_sk#55)) +(49) ReusedExchange [Reuses operator id: 11] +Output [2]: [d_date_sk#53, d_year#54] -(59) CometBroadcastExchange -Input [2]: [d_date_sk#55, d_year#56] -Arguments: [d_date_sk#55, d_year#56] +(50) CometBroadcastHashJoin +Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] +Right output [2]: [d_date_sk#53, d_year#54] +Arguments: [ws_sold_date_sk#46], [d_date_sk#53], Inner -(60) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#43, ws_order_number#44, ws_quantity#45, ws_ext_sales_price#46, ws_sold_date_sk#47, i_brand_id#50, i_class_id#51, i_category_id#52, i_manufact_id#54] -Right output [2]: [d_date_sk#55, d_year#56] -Arguments: [ws_sold_date_sk#47], [d_date_sk#55], Inner +(51) CometProject +Input [11]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_date_sk#53, d_year#54] +Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -(61) CometProject -Input [11]: [ws_item_sk#43, ws_order_number#44, ws_quantity#45, ws_ext_sales_price#46, ws_sold_date_sk#47, i_brand_id#50, i_class_id#51, i_category_id#52, i_manufact_id#54, d_date_sk#55, d_year#56] -Arguments: [ws_item_sk#43, ws_order_number#44, ws_quantity#45, ws_ext_sales_price#46, i_brand_id#50, i_class_id#51, i_category_id#52, i_manufact_id#54, d_year#56], [ws_item_sk#43, ws_order_number#44, ws_quantity#45, ws_ext_sales_price#46, i_brand_id#50, i_class_id#51, i_category_id#52, i_manufact_id#54, d_year#56] +(52) ColumnarToRow [codegen id : 11] +Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -(62) ColumnarToRow [codegen id : 11] -Input [9]: [ws_item_sk#43, ws_order_number#44, ws_quantity#45, ws_ext_sales_price#46, i_brand_id#50, i_class_id#51, i_category_id#52, i_manufact_id#54, d_year#56] +(53) Exchange +Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] +Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(63) Exchange -Input [9]: [ws_item_sk#43, ws_order_number#44, ws_quantity#45, ws_ext_sales_price#46, i_brand_id#50, i_class_id#51, i_category_id#52, i_manufact_id#54, d_year#56] -Arguments: hashpartitioning(ws_order_number#44, ws_item_sk#43, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(54) Sort [codegen id : 12] +Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] +Arguments: [ws_order_number#43 ASC NULLS FIRST, ws_item_sk#42 ASC NULLS FIRST], false, 0 -(64) Sort [codegen id : 12] -Input [9]: [ws_item_sk#43, ws_order_number#44, ws_quantity#45, ws_ext_sales_price#46, i_brand_id#50, i_class_id#51, i_category_id#52, i_manufact_id#54, d_year#56] -Arguments: [ws_order_number#44 ASC NULLS FIRST, ws_item_sk#43 ASC NULLS FIRST], false, 0 - -(65) Scan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#57, wr_order_number#58, wr_return_quantity#59, wr_return_amt#60, wr_returned_date_sk#61] +(55) Scan parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct -(66) CometFilter -Input [5]: [wr_item_sk#57, wr_order_number#58, wr_return_quantity#59, wr_return_amt#60, wr_returned_date_sk#61] -Condition : (isnotnull(wr_order_number#58) AND isnotnull(wr_item_sk#57)) +(56) CometFilter +Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] +Condition : (isnotnull(wr_order_number#56) AND isnotnull(wr_item_sk#55)) -(67) CometProject -Input [5]: [wr_item_sk#57, wr_order_number#58, wr_return_quantity#59, wr_return_amt#60, wr_returned_date_sk#61] -Arguments: [wr_item_sk#57, wr_order_number#58, wr_return_quantity#59, wr_return_amt#60], [wr_item_sk#57, wr_order_number#58, wr_return_quantity#59, wr_return_amt#60] +(57) CometProject +Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] +Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -(68) ColumnarToRow [codegen id : 13] -Input [4]: [wr_item_sk#57, wr_order_number#58, wr_return_quantity#59, wr_return_amt#60] +(58) ColumnarToRow [codegen id : 13] +Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -(69) Exchange -Input [4]: [wr_item_sk#57, wr_order_number#58, wr_return_quantity#59, wr_return_amt#60] -Arguments: hashpartitioning(wr_order_number#58, wr_item_sk#57, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(59) Exchange +Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] +Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(70) Sort [codegen id : 14] -Input [4]: [wr_item_sk#57, wr_order_number#58, wr_return_quantity#59, wr_return_amt#60] -Arguments: [wr_order_number#58 ASC NULLS FIRST, wr_item_sk#57 ASC NULLS FIRST], false, 0 +(60) Sort [codegen id : 14] +Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] +Arguments: [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST], false, 0 -(71) SortMergeJoin [codegen id : 15] -Left keys [2]: [ws_order_number#44, ws_item_sk#43] -Right keys [2]: [wr_order_number#58, wr_item_sk#57] +(61) SortMergeJoin [codegen id : 15] +Left keys [2]: [ws_order_number#43, ws_item_sk#42] +Right keys [2]: [wr_order_number#56, wr_item_sk#55] Join type: LeftOuter Join condition: None -(72) Project [codegen id : 15] -Output [7]: [d_year#56, i_brand_id#50, i_class_id#51, i_category_id#52, i_manufact_id#54, (ws_quantity#45 - coalesce(wr_return_quantity#59, 0)) AS sales_cnt#62, (ws_ext_sales_price#46 - coalesce(wr_return_amt#60, 0.00)) AS sales_amt#63] -Input [13]: [ws_item_sk#43, ws_order_number#44, ws_quantity#45, ws_ext_sales_price#46, i_brand_id#50, i_class_id#51, i_category_id#52, i_manufact_id#54, d_year#56, wr_item_sk#57, wr_order_number#58, wr_return_quantity#59, wr_return_amt#60] +(62) Project [codegen id : 15] +Output [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, (ws_quantity#44 - coalesce(wr_return_quantity#57, 0)) AS sales_cnt#60, (ws_ext_sales_price#45 - coalesce(wr_return_amt#58, 0.00)) AS sales_amt#61] +Input [13]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54, wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -(73) Union +(63) Union -(74) HashAggregate [codegen id : 16] +(64) HashAggregate [codegen id : 16] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Functions: [] Aggregate Attributes: [] Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -(75) Exchange +(65) Exchange Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(76) HashAggregate [codegen id : 17] +(66) HashAggregate [codegen id : 17] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Functions: [] Aggregate Attributes: [] Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -(77) HashAggregate [codegen id : 17] +(67) HashAggregate [codegen id : 17] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] -Aggregate Attributes [2]: [sum#64, sum#65] -Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#66, sum#67] +Aggregate Attributes [2]: [sum#62, sum#63] +Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] -(78) Exchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#66, sum#67] +(68) Exchange +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(79) HashAggregate [codegen id : 18] -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#66, sum#67] +(69) HashAggregate [codegen id : 18] +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] -Aggregate Attributes [2]: [sum(sales_cnt#20)#68, sum(UnscaledValue(sales_amt#21))#69] -Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#20)#68 AS sales_cnt#70, MakeDecimal(sum(UnscaledValue(sales_amt#21))#69,18,2) AS sales_amt#71] +Aggregate Attributes [2]: [sum(sales_cnt#20)#66, sum(UnscaledValue(sales_amt#21))#67] +Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#20)#66 AS sales_cnt#68, MakeDecimal(sum(UnscaledValue(sales_amt#21))#67,18,2) AS sales_amt#69] -(80) Filter [codegen id : 18] -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#70, sales_amt#71] -Condition : isnotnull(sales_cnt#70) +(70) Filter [codegen id : 18] +Input [7]: [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] +Condition : isnotnull(sales_cnt#68) -(81) Exchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#70, sales_amt#71] +(71) Exchange +Input [7]: [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] Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(82) Sort [codegen id : 19] -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#70, sales_amt#71] +(72) Sort [codegen id : 19] +Input [7]: [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] Arguments: [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST], false, 0 -(83) Scan parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#72, cs_order_number#73, cs_quantity#74, cs_ext_sales_price#75, cs_sold_date_sk#76] +(73) Scan parquet spark_catalog.default.catalog_sales +Output [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#76), dynamicpruningexpression(cs_sold_date_sk#76 IN dynamicpruning#77)] +PartitionFilters: [isnotnull(cs_sold_date_sk#74), dynamicpruningexpression(cs_sold_date_sk#74 IN dynamicpruning#75)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(84) CometFilter -Input [5]: [cs_item_sk#72, cs_order_number#73, cs_quantity#74, cs_ext_sales_price#75, cs_sold_date_sk#76] -Condition : isnotnull(cs_item_sk#72) +(74) CometFilter +Input [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] +Condition : isnotnull(cs_item_sk#70) -(85) Scan parquet spark_catalog.default.item -Output [6]: [i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81, i_category#82, i_manufact_id#83] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books ), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] -ReadSchema: struct - -(86) CometFilter -Input [6]: [i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81, i_category#82, i_manufact_id#83] -Condition : ((((((isnotnull(i_category#82) AND (i_category#82 = Books )) AND isnotnull(i_item_sk#78)) AND isnotnull(i_brand_id#79)) AND isnotnull(i_class_id#80)) AND isnotnull(i_category_id#81)) AND isnotnull(i_manufact_id#83)) - -(87) CometProject -Input [6]: [i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81, i_category#82, i_manufact_id#83] -Arguments: [i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83], [i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83] - -(88) CometBroadcastExchange -Input [5]: [i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83] -Arguments: [i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83] +(75) ReusedExchange [Reuses operator id: 6] +Output [5]: [i_item_sk#76, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] -(89) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#72, cs_order_number#73, cs_quantity#74, cs_ext_sales_price#75, cs_sold_date_sk#76] -Right output [5]: [i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83] -Arguments: [cs_item_sk#72], [i_item_sk#78], Inner +(76) CometBroadcastHashJoin +Left output [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] +Right output [5]: [i_item_sk#76, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +Arguments: [cs_item_sk#70], [i_item_sk#76], Inner -(90) CometProject -Input [10]: [cs_item_sk#72, cs_order_number#73, cs_quantity#74, cs_ext_sales_price#75, cs_sold_date_sk#76, i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83] -Arguments: [cs_item_sk#72, cs_order_number#73, cs_quantity#74, cs_ext_sales_price#75, cs_sold_date_sk#76, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83], [cs_item_sk#72, cs_order_number#73, cs_quantity#74, cs_ext_sales_price#75, cs_sold_date_sk#76, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83] +(77) CometProject +Input [10]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_item_sk#76, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +Arguments: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80], [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] -(91) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#84, d_year#85] +(78) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#81, d_year#82] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(92) CometFilter -Input [2]: [d_date_sk#84, d_year#85] -Condition : ((isnotnull(d_year#85) AND (d_year#85 = 2001)) AND isnotnull(d_date_sk#84)) +(79) 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)) -(93) CometBroadcastExchange -Input [2]: [d_date_sk#84, d_year#85] -Arguments: [d_date_sk#84, d_year#85] +(80) CometBroadcastExchange +Input [2]: [d_date_sk#81, d_year#82] +Arguments: [d_date_sk#81, d_year#82] -(94) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#72, cs_order_number#73, cs_quantity#74, cs_ext_sales_price#75, cs_sold_date_sk#76, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83] -Right output [2]: [d_date_sk#84, d_year#85] -Arguments: [cs_sold_date_sk#76], [d_date_sk#84], Inner +(81) CometBroadcastHashJoin +Left output [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +Right output [2]: [d_date_sk#81, d_year#82] +Arguments: [cs_sold_date_sk#74], [d_date_sk#81], Inner -(95) CometProject -Input [11]: [cs_item_sk#72, cs_order_number#73, cs_quantity#74, cs_ext_sales_price#75, cs_sold_date_sk#76, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, d_date_sk#84, d_year#85] -Arguments: [cs_item_sk#72, cs_order_number#73, cs_quantity#74, cs_ext_sales_price#75, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, d_year#85], [cs_item_sk#72, cs_order_number#73, cs_quantity#74, cs_ext_sales_price#75, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, d_year#85] +(82) CometProject +Input [11]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_date_sk#81, d_year#82] +Arguments: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82], [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] -(96) ColumnarToRow [codegen id : 20] -Input [9]: [cs_item_sk#72, cs_order_number#73, cs_quantity#74, cs_ext_sales_price#75, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, d_year#85] +(83) ColumnarToRow [codegen id : 20] +Input [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] -(97) Exchange -Input [9]: [cs_item_sk#72, cs_order_number#73, cs_quantity#74, cs_ext_sales_price#75, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, d_year#85] -Arguments: hashpartitioning(cs_order_number#73, cs_item_sk#72, 5), ENSURE_REQUIREMENTS, [plan_id=10] +(84) Exchange +Input [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] +Arguments: hashpartitioning(cs_order_number#71, cs_item_sk#70, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(98) Sort [codegen id : 21] -Input [9]: [cs_item_sk#72, cs_order_number#73, cs_quantity#74, cs_ext_sales_price#75, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, d_year#85] -Arguments: [cs_order_number#73 ASC NULLS FIRST, cs_item_sk#72 ASC NULLS FIRST], false, 0 +(85) Sort [codegen id : 21] +Input [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] +Arguments: [cs_order_number#71 ASC NULLS FIRST, cs_item_sk#70 ASC NULLS FIRST], false, 0 -(99) ReusedExchange [Reuses operator id: 21] -Output [4]: [cr_item_sk#86, cr_order_number#87, cr_return_quantity#88, cr_return_amount#89] +(86) ReusedExchange [Reuses operator id: 21] +Output [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] -(100) Sort [codegen id : 23] -Input [4]: [cr_item_sk#86, cr_order_number#87, cr_return_quantity#88, cr_return_amount#89] -Arguments: [cr_order_number#87 ASC NULLS FIRST, cr_item_sk#86 ASC NULLS FIRST], false, 0 +(87) Sort [codegen id : 23] +Input [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] +Arguments: [cr_order_number#84 ASC NULLS FIRST, cr_item_sk#83 ASC NULLS FIRST], false, 0 -(101) SortMergeJoin [codegen id : 24] -Left keys [2]: [cs_order_number#73, cs_item_sk#72] -Right keys [2]: [cr_order_number#87, cr_item_sk#86] +(88) SortMergeJoin [codegen id : 24] +Left keys [2]: [cs_order_number#71, cs_item_sk#70] +Right keys [2]: [cr_order_number#84, cr_item_sk#83] Join type: LeftOuter Join condition: None -(102) Project [codegen id : 24] -Output [7]: [d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, (cs_quantity#74 - coalesce(cr_return_quantity#88, 0)) AS sales_cnt#20, (cs_ext_sales_price#75 - coalesce(cr_return_amount#89, 0.00)) AS sales_amt#21] -Input [13]: [cs_item_sk#72, cs_order_number#73, cs_quantity#74, cs_ext_sales_price#75, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, d_year#85, cr_item_sk#86, cr_order_number#87, cr_return_quantity#88, cr_return_amount#89] +(89) Project [codegen id : 24] +Output [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, (cs_quantity#72 - coalesce(cr_return_quantity#85, 0)) AS sales_cnt#20, (cs_ext_sales_price#73 - coalesce(cr_return_amount#86, 0.00)) AS sales_amt#21] +Input [13]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82, cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] -(103) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#90, ss_ticket_number#91, ss_quantity#92, ss_ext_sales_price#93, ss_sold_date_sk#94] +(90) Scan parquet spark_catalog.default.store_sales +Output [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#94), dynamicpruningexpression(ss_sold_date_sk#94 IN dynamicpruning#95)] +PartitionFilters: [isnotnull(ss_sold_date_sk#91), dynamicpruningexpression(ss_sold_date_sk#91 IN dynamicpruning#92)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(104) CometFilter -Input [5]: [ss_item_sk#90, ss_ticket_number#91, ss_quantity#92, ss_ext_sales_price#93, ss_sold_date_sk#94] -Condition : isnotnull(ss_item_sk#90) - -(105) Scan parquet spark_catalog.default.item -Output [6]: [i_item_sk#96, i_brand_id#97, i_class_id#98, i_category_id#99, i_category#100, i_manufact_id#101] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books ), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] -ReadSchema: struct - -(106) CometFilter -Input [6]: [i_item_sk#96, i_brand_id#97, i_class_id#98, i_category_id#99, i_category#100, i_manufact_id#101] -Condition : ((((((isnotnull(i_category#100) AND (i_category#100 = Books )) AND isnotnull(i_item_sk#96)) AND isnotnull(i_brand_id#97)) AND isnotnull(i_class_id#98)) AND isnotnull(i_category_id#99)) AND isnotnull(i_manufact_id#101)) - -(107) CometProject -Input [6]: [i_item_sk#96, i_brand_id#97, i_class_id#98, i_category_id#99, i_category#100, i_manufact_id#101] -Arguments: [i_item_sk#96, i_brand_id#97, i_class_id#98, i_category_id#99, i_manufact_id#101], [i_item_sk#96, i_brand_id#97, i_class_id#98, i_category_id#99, i_manufact_id#101] +(91) 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) -(108) CometBroadcastExchange -Input [5]: [i_item_sk#96, i_brand_id#97, i_class_id#98, i_category_id#99, i_manufact_id#101] -Arguments: [i_item_sk#96, i_brand_id#97, i_class_id#98, i_category_id#99, i_manufact_id#101] +(92) ReusedExchange [Reuses operator id: 6] +Output [5]: [i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] -(109) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#90, ss_ticket_number#91, ss_quantity#92, ss_ext_sales_price#93, ss_sold_date_sk#94] -Right output [5]: [i_item_sk#96, i_brand_id#97, i_class_id#98, i_category_id#99, i_manufact_id#101] -Arguments: [ss_item_sk#90], [i_item_sk#96], Inner - -(110) CometProject -Input [10]: [ss_item_sk#90, ss_ticket_number#91, ss_quantity#92, ss_ext_sales_price#93, ss_sold_date_sk#94, i_item_sk#96, i_brand_id#97, i_class_id#98, i_category_id#99, i_manufact_id#101] -Arguments: [ss_item_sk#90, ss_ticket_number#91, ss_quantity#92, ss_ext_sales_price#93, ss_sold_date_sk#94, i_brand_id#97, i_class_id#98, i_category_id#99, i_manufact_id#101], [ss_item_sk#90, ss_ticket_number#91, ss_quantity#92, ss_ext_sales_price#93, ss_sold_date_sk#94, i_brand_id#97, i_class_id#98, i_category_id#99, i_manufact_id#101] - -(111) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#102, d_year#103] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct +(93) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] +Right output [5]: [i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] +Arguments: [ss_item_sk#87], [i_item_sk#93], Inner -(112) CometFilter -Input [2]: [d_date_sk#102, d_year#103] -Condition : ((isnotnull(d_year#103) AND (d_year#103 = 2001)) AND isnotnull(d_date_sk#102)) +(94) CometProject +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] +Arguments: [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], [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] -(113) CometBroadcastExchange -Input [2]: [d_date_sk#102, d_year#103] -Arguments: [d_date_sk#102, d_year#103] +(95) ReusedExchange [Reuses operator id: 80] +Output [2]: [d_date_sk#98, d_year#99] -(114) CometBroadcastHashJoin -Left output [9]: [ss_item_sk#90, ss_ticket_number#91, ss_quantity#92, ss_ext_sales_price#93, ss_sold_date_sk#94, i_brand_id#97, i_class_id#98, i_category_id#99, i_manufact_id#101] -Right output [2]: [d_date_sk#102, d_year#103] -Arguments: [ss_sold_date_sk#94], [d_date_sk#102], Inner +(96) CometBroadcastHashJoin +Left 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] +Right output [2]: [d_date_sk#98, d_year#99] +Arguments: [ss_sold_date_sk#91], [d_date_sk#98], Inner -(115) CometProject -Input [11]: [ss_item_sk#90, ss_ticket_number#91, ss_quantity#92, ss_ext_sales_price#93, ss_sold_date_sk#94, i_brand_id#97, i_class_id#98, i_category_id#99, i_manufact_id#101, d_date_sk#102, d_year#103] -Arguments: [ss_item_sk#90, ss_ticket_number#91, ss_quantity#92, ss_ext_sales_price#93, i_brand_id#97, i_class_id#98, i_category_id#99, i_manufact_id#101, d_year#103], [ss_item_sk#90, ss_ticket_number#91, ss_quantity#92, ss_ext_sales_price#93, i_brand_id#97, i_class_id#98, i_category_id#99, i_manufact_id#101, d_year#103] +(97) CometProject +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] +Arguments: [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], [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] -(116) ColumnarToRow [codegen id : 25] -Input [9]: [ss_item_sk#90, ss_ticket_number#91, ss_quantity#92, ss_ext_sales_price#93, i_brand_id#97, i_class_id#98, i_category_id#99, i_manufact_id#101, d_year#103] +(98) ColumnarToRow [codegen id : 25] +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] -(117) Exchange -Input [9]: [ss_item_sk#90, ss_ticket_number#91, ss_quantity#92, ss_ext_sales_price#93, i_brand_id#97, i_class_id#98, i_category_id#99, i_manufact_id#101, d_year#103] -Arguments: hashpartitioning(ss_ticket_number#91, ss_item_sk#90, 5), ENSURE_REQUIREMENTS, [plan_id=11] +(99) 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=11] -(118) Sort [codegen id : 26] -Input [9]: [ss_item_sk#90, ss_ticket_number#91, ss_quantity#92, ss_ext_sales_price#93, i_brand_id#97, i_class_id#98, i_category_id#99, i_manufact_id#101, d_year#103] -Arguments: [ss_ticket_number#91 ASC NULLS FIRST, ss_item_sk#90 ASC NULLS FIRST], false, 0 +(100) Sort [codegen id : 26] +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 -(119) ReusedExchange [Reuses operator id: 45] -Output [4]: [sr_item_sk#104, sr_ticket_number#105, sr_return_quantity#106, sr_return_amt#107] +(101) ReusedExchange [Reuses operator id: 40] +Output [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] -(120) Sort [codegen id : 28] -Input [4]: [sr_item_sk#104, sr_ticket_number#105, sr_return_quantity#106, sr_return_amt#107] -Arguments: [sr_ticket_number#105 ASC NULLS FIRST, sr_item_sk#104 ASC NULLS FIRST], false, 0 +(102) Sort [codegen id : 28] +Input [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] +Arguments: [sr_ticket_number#101 ASC NULLS FIRST, sr_item_sk#100 ASC NULLS FIRST], false, 0 -(121) SortMergeJoin [codegen id : 29] -Left keys [2]: [ss_ticket_number#91, ss_item_sk#90] -Right keys [2]: [sr_ticket_number#105, sr_item_sk#104] +(103) SortMergeJoin [codegen id : 29] +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 -(122) Project [codegen id : 29] -Output [7]: [d_year#103, i_brand_id#97, i_class_id#98, i_category_id#99, i_manufact_id#101, (ss_quantity#92 - coalesce(sr_return_quantity#106, 0)) AS sales_cnt#41, (ss_ext_sales_price#93 - coalesce(sr_return_amt#107, 0.00)) AS sales_amt#42] -Input [13]: [ss_item_sk#90, ss_ticket_number#91, ss_quantity#92, ss_ext_sales_price#93, i_brand_id#97, i_class_id#98, i_category_id#99, i_manufact_id#101, d_year#103, sr_item_sk#104, sr_ticket_number#105, sr_return_quantity#106, sr_return_amt#107] +(104) Project [codegen id : 29] +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] -(123) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112] +(105) Scan parquet spark_catalog.default.web_sales +Output [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#112), dynamicpruningexpression(ws_sold_date_sk#112 IN dynamicpruning#113)] +PartitionFilters: [isnotnull(ws_sold_date_sk#108), dynamicpruningexpression(ws_sold_date_sk#108 IN dynamicpruning#109)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(124) CometFilter -Input [5]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112] -Condition : isnotnull(ws_item_sk#108) - -(125) Scan parquet spark_catalog.default.item -Output [6]: [i_item_sk#114, i_brand_id#115, i_class_id#116, i_category_id#117, i_category#118, i_manufact_id#119] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books ), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] -ReadSchema: struct - -(126) CometFilter -Input [6]: [i_item_sk#114, i_brand_id#115, i_class_id#116, i_category_id#117, i_category#118, i_manufact_id#119] -Condition : ((((((isnotnull(i_category#118) AND (i_category#118 = Books )) AND isnotnull(i_item_sk#114)) AND isnotnull(i_brand_id#115)) AND isnotnull(i_class_id#116)) AND isnotnull(i_category_id#117)) AND isnotnull(i_manufact_id#119)) - -(127) CometProject -Input [6]: [i_item_sk#114, i_brand_id#115, i_class_id#116, i_category_id#117, i_category#118, i_manufact_id#119] -Arguments: [i_item_sk#114, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#119], [i_item_sk#114, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#119] - -(128) CometBroadcastExchange -Input [5]: [i_item_sk#114, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#119] -Arguments: [i_item_sk#114, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#119] - -(129) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112] -Right output [5]: [i_item_sk#114, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#119] -Arguments: [ws_item_sk#108], [i_item_sk#114], Inner +(106) 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) -(130) CometProject -Input [10]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112, i_item_sk#114, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#119] -Arguments: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#119], [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#119] +(107) ReusedExchange [Reuses operator id: 6] +Output [5]: [i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -(131) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#120, d_year#121] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct +(108) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] +Right output [5]: [i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] +Arguments: [ws_item_sk#104], [i_item_sk#110], Inner -(132) CometFilter -Input [2]: [d_date_sk#120, d_year#121] -Condition : ((isnotnull(d_year#121) AND (d_year#121 = 2001)) AND isnotnull(d_date_sk#120)) +(109) CometProject +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] +Arguments: [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], [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] -(133) CometBroadcastExchange -Input [2]: [d_date_sk#120, d_year#121] -Arguments: [d_date_sk#120, d_year#121] +(110) ReusedExchange [Reuses operator id: 80] +Output [2]: [d_date_sk#115, d_year#116] -(134) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#119] -Right output [2]: [d_date_sk#120, d_year#121] -Arguments: [ws_sold_date_sk#112], [d_date_sk#120], Inner +(111) CometBroadcastHashJoin +Left 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] +Right output [2]: [d_date_sk#115, d_year#116] +Arguments: [ws_sold_date_sk#108], [d_date_sk#115], Inner -(135) CometProject -Input [11]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#119, d_date_sk#120, d_year#121] -Arguments: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#119, d_year#121], [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#119, d_year#121] +(112) CometProject +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] +Arguments: [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], [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] -(136) ColumnarToRow [codegen id : 30] -Input [9]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#119, d_year#121] +(113) ColumnarToRow [codegen id : 30] +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] -(137) Exchange -Input [9]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#119, d_year#121] -Arguments: hashpartitioning(ws_order_number#109, ws_item_sk#108, 5), ENSURE_REQUIREMENTS, [plan_id=12] +(114) 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=12] -(138) Sort [codegen id : 31] -Input [9]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#119, d_year#121] -Arguments: [ws_order_number#109 ASC NULLS FIRST, ws_item_sk#108 ASC NULLS FIRST], false, 0 +(115) Sort [codegen id : 31] +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 -(139) ReusedExchange [Reuses operator id: 69] -Output [4]: [wr_item_sk#122, wr_order_number#123, wr_return_quantity#124, wr_return_amt#125] +(116) ReusedExchange [Reuses operator id: 59] +Output [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] -(140) Sort [codegen id : 33] -Input [4]: [wr_item_sk#122, wr_order_number#123, wr_return_quantity#124, wr_return_amt#125] -Arguments: [wr_order_number#123 ASC NULLS FIRST, wr_item_sk#122 ASC NULLS FIRST], false, 0 +(117) Sort [codegen id : 33] +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 -(141) SortMergeJoin [codegen id : 34] -Left keys [2]: [ws_order_number#109, ws_item_sk#108] -Right keys [2]: [wr_order_number#123, wr_item_sk#122] +(118) SortMergeJoin [codegen id : 34] +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 -(142) Project [codegen id : 34] -Output [7]: [d_year#121, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#119, (ws_quantity#110 - coalesce(wr_return_quantity#124, 0)) AS sales_cnt#62, (ws_ext_sales_price#111 - coalesce(wr_return_amt#125, 0.00)) AS sales_amt#63] -Input [13]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#119, d_year#121, wr_item_sk#122, wr_order_number#123, wr_return_quantity#124, wr_return_amt#125] +(119) Project [codegen id : 34] +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] -(143) Union +(120) Union -(144) HashAggregate [codegen id : 35] -Input [7]: [d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, sales_cnt#20, sales_amt#21] +(121) HashAggregate [codegen id : 35] +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#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, sales_cnt#20, sales_amt#21] +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] -(145) Exchange -Input [7]: [d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, [plan_id=13] +(122) 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=13] -(146) HashAggregate [codegen id : 36] -Input [7]: [d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, sales_cnt#20, sales_amt#21] +(123) HashAggregate [codegen id : 36] +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#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, sales_cnt#20, sales_amt#21] +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] -(147) HashAggregate [codegen id : 36] -Input [7]: [d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83] +(124) HashAggregate [codegen id : 36] +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#64, sum#126] -Results [7]: [d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, sum#66, sum#127] +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] -(148) Exchange -Input [7]: [d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, sum#66, sum#127] -Arguments: hashpartitioning(d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, 5), ENSURE_REQUIREMENTS, [plan_id=14] +(125) 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=14] -(149) HashAggregate [codegen id : 37] -Input [7]: [d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, sum#66, sum#127] -Keys [5]: [d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83] +(126) HashAggregate [codegen id : 37] +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)#68, sum(UnscaledValue(sales_amt#21))#69] -Results [7]: [d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, sum(sales_cnt#20)#68 AS sales_cnt#128, MakeDecimal(sum(UnscaledValue(sales_amt#21))#69,18,2) AS sales_amt#129] +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] -(150) Filter [codegen id : 37] -Input [7]: [d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, sales_cnt#128, sales_amt#129] -Condition : isnotnull(sales_cnt#128) +(127) Filter [codegen id : 37] +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) -(151) Exchange -Input [7]: [d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, sales_cnt#128, sales_amt#129] -Arguments: hashpartitioning(i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, 5), ENSURE_REQUIREMENTS, [plan_id=15] +(128) 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=15] -(152) Sort [codegen id : 38] -Input [7]: [d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, sales_cnt#128, sales_amt#129] -Arguments: [i_brand_id#79 ASC NULLS FIRST, i_class_id#80 ASC NULLS FIRST, i_category_id#81 ASC NULLS FIRST, i_manufact_id#83 ASC NULLS FIRST], false, 0 +(129) Sort [codegen id : 38] +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 -(153) SortMergeJoin [codegen id : 39] +(130) SortMergeJoin [codegen id : 39] 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#79, i_class_id#80, i_category_id#81, i_manufact_id#83] +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#70 as decimal(17,2)) / cast(sales_cnt#128 as decimal(17,2))) < 0.90000000000000000000) +Join condition: ((cast(sales_cnt#68 as decimal(17,2)) / cast(sales_cnt#123 as decimal(17,2))) < 0.90000000000000000000) -(154) Project [codegen id : 39] -Output [10]: [d_year#85 AS prev_year#130, d_year#14 AS year#131, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#128 AS prev_yr_cnt#132, sales_cnt#70 AS curr_yr_cnt#133, (sales_cnt#70 - sales_cnt#128) AS sales_cnt_diff#134, (sales_amt#71 - sales_amt#129) AS sales_amt_diff#135] -Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#70, sales_amt#71, d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, sales_cnt#128, sales_amt#129] +(131) Project [codegen id : 39] +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] -(155) TakeOrderedAndProject -Input [10]: [prev_year#130, year#131, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#132, curr_yr_cnt#133, sales_cnt_diff#134, sales_amt_diff#135] -Arguments: 100, [sales_cnt_diff#134 ASC NULLS FIRST], [prev_year#130, year#131, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#132, curr_yr_cnt#133, sales_cnt_diff#134, sales_amt_diff#135] +(132) 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 (159) -+- * ColumnarToRow (158) - +- CometFilter (157) - +- CometScan parquet spark_catalog.default.date_dim (156) +BroadcastExchange (136) ++- * ColumnarToRow (135) + +- CometFilter (134) + +- CometScan parquet spark_catalog.default.date_dim (133) -(156) Scan parquet spark_catalog.default.date_dim +(133) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#13, d_year#14] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(157) CometFilter +(134) 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)) -(158) ColumnarToRow [codegen id : 1] +(135) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] -(159) BroadcastExchange +(136) BroadcastExchange Input [2]: [d_date_sk#13, d_year#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 49 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 83 Hosting Expression = cs_sold_date_sk#76 IN dynamicpruning#77 -BroadcastExchange (163) -+- * ColumnarToRow (162) - +- CometFilter (161) - +- CometScan parquet spark_catalog.default.date_dim (160) +Subquery:4 Hosting operator id = 73 Hosting Expression = cs_sold_date_sk#74 IN dynamicpruning#75 +BroadcastExchange (140) ++- * ColumnarToRow (139) + +- CometFilter (138) + +- CometScan parquet spark_catalog.default.date_dim (137) -(160) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#84, d_year#85] +(137) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#81, d_year#82] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(161) CometFilter -Input [2]: [d_date_sk#84, d_year#85] -Condition : ((isnotnull(d_year#85) AND (d_year#85 = 2001)) AND isnotnull(d_date_sk#84)) +(138) 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)) -(162) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#84, d_year#85] +(139) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#81, d_year#82] -(163) BroadcastExchange -Input [2]: [d_date_sk#84, d_year#85] +(140) BroadcastExchange +Input [2]: [d_date_sk#81, d_year#82] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] -Subquery:5 Hosting operator id = 103 Hosting Expression = ss_sold_date_sk#94 IN dynamicpruning#77 +Subquery:5 Hosting operator id = 90 Hosting Expression = ss_sold_date_sk#91 IN dynamicpruning#75 -Subquery:6 Hosting operator id = 123 Hosting Expression = ws_sold_date_sk#112 IN dynamicpruning#77 +Subquery:6 Hosting operator id = 105 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 944b48590..34866bc37 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 @@ -81,18 +81,13 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #10 - CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange #11 - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #7 InputAdapter WholeStageCodegen (9) Sort [sr_ticket_number,sr_item_sk] InputAdapter - Exchange [sr_ticket_number,sr_item_sk] #12 + Exchange [sr_ticket_number,sr_item_sk] #10 WholeStageCodegen (8) ColumnarToRow InputAdapter @@ -106,7 +101,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat WholeStageCodegen (12) Sort [ws_order_number,ws_item_sk] InputAdapter - Exchange [ws_order_number,ws_item_sk] #13 + Exchange [ws_order_number,ws_item_sk] #11 WholeStageCodegen (11) ColumnarToRow InputAdapter @@ -117,18 +112,13 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #14 - CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange #15 - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #7 InputAdapter WholeStageCodegen (14) Sort [wr_order_number,wr_item_sk] InputAdapter - Exchange [wr_order_number,wr_item_sk] #16 + Exchange [wr_order_number,wr_item_sk] #12 WholeStageCodegen (13) ColumnarToRow InputAdapter @@ -139,17 +129,17 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat WholeStageCodegen (38) 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] #17 + Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 WholeStageCodegen (37) 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] #18 + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 WholeStageCodegen (36) 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] #19 + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 WholeStageCodegen (35) HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter @@ -161,7 +151,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat WholeStageCodegen (21) Sort [cs_order_number,cs_item_sk] InputAdapter - Exchange [cs_order_number,cs_item_sk] #20 + Exchange [cs_order_number,cs_item_sk] #16 WholeStageCodegen (20) ColumnarToRow InputAdapter @@ -172,17 +162,14 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #21 + BroadcastExchange #17 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #22 - CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange #23 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + CometBroadcastExchange #18 CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter @@ -197,7 +184,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat WholeStageCodegen (26) Sort [ss_ticket_number,ss_item_sk] InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #24 + Exchange [ss_ticket_number,ss_item_sk] #19 WholeStageCodegen (25) ColumnarToRow InputAdapter @@ -208,18 +195,13 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #25 - CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange #26 - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #18 InputAdapter WholeStageCodegen (28) Sort [sr_ticket_number,sr_item_sk] InputAdapter - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #12 + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #10 WholeStageCodegen (34) 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] @@ -227,7 +209,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat WholeStageCodegen (31) Sort [ws_order_number,ws_item_sk] InputAdapter - Exchange [ws_order_number,ws_item_sk] #27 + Exchange [ws_order_number,ws_item_sk] #20 WholeStageCodegen (30) ColumnarToRow InputAdapter @@ -238,15 +220,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #28 - CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange #29 - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #18 InputAdapter WholeStageCodegen (33) Sort [wr_order_number,wr_item_sk] InputAdapter - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #16 + ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt index ed486d8af..61f6a472c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * HashAggregate (41) - +- Exchange (40) - +- * ColumnarToRow (39) - +- CometHashAggregate (38) - +- CometUnion (37) +TakeOrderedAndProject (34) ++- * HashAggregate (33) + +- Exchange (32) + +- * ColumnarToRow (31) + +- CometHashAggregate (30) + +- CometUnion (29) :- CometProject (12) : +- CometBroadcastHashJoin (11) : :- CometProject (7) @@ -17,30 +17,22 @@ TakeOrderedAndProject (42) : +- CometBroadcastExchange (10) : +- CometFilter (9) : +- CometScan parquet spark_catalog.default.date_dim (8) - :- CometProject (24) - : +- CometBroadcastHashJoin (23) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) + :- CometProject (20) + : +- CometBroadcastHashJoin (19) + : :- CometProject (17) + : : +- CometBroadcastHashJoin (16) : : :- CometFilter (14) : : : +- CometScan parquet spark_catalog.default.web_sales (13) - : : +- CometBroadcastExchange (17) - : : +- CometFilter (16) - : : +- CometScan parquet spark_catalog.default.item (15) - : +- CometBroadcastExchange (22) - : +- CometFilter (21) - : +- CometScan parquet spark_catalog.default.date_dim (20) - +- CometProject (36) - +- CometBroadcastHashJoin (35) - :- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometFilter (26) - : : +- CometScan parquet spark_catalog.default.catalog_sales (25) - : +- CometBroadcastExchange (29) - : +- CometFilter (28) - : +- CometScan parquet spark_catalog.default.item (27) - +- CometBroadcastExchange (34) - +- CometFilter (33) - +- CometScan parquet spark_catalog.default.date_dim (32) + : : +- ReusedExchange (15) + : +- ReusedExchange (18) + +- CometProject (28) + +- CometBroadcastHashJoin (27) + :- CometProject (25) + : +- CometBroadcastHashJoin (24) + : :- CometFilter (22) + : : +- CometScan parquet spark_catalog.default.catalog_sales (21) + : +- ReusedExchange (23) + +- ReusedExchange (26) (1) Scan parquet spark_catalog.default.store_sales @@ -115,55 +107,31 @@ ReadSchema: struct - -(16) CometFilter -Input [2]: [i_item_sk#17, i_category#18] -Condition : isnotnull(i_item_sk#17) - -(17) CometBroadcastExchange -Input [2]: [i_item_sk#17, i_category#18] -Arguments: [i_item_sk#17, i_category#18] -(18) CometBroadcastHashJoin +(16) CometBroadcastHashJoin Left output [4]: [ws_item_sk#13, ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16] Right output [2]: [i_item_sk#17, i_category#18] Arguments: [ws_item_sk#13], [i_item_sk#17], Inner -(19) CometProject +(17) CometProject Input [6]: [ws_item_sk#13, ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16, i_item_sk#17, i_category#18] Arguments: [ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16, i_category#18], [ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16, i_category#18] -(20) Scan parquet spark_catalog.default.date_dim +(18) ReusedExchange [Reuses operator id: 10] Output [3]: [d_date_sk#19, d_year#20, d_qoy#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [d_date_sk#19, d_year#20, d_qoy#21] -Condition : isnotnull(d_date_sk#19) - -(22) CometBroadcastExchange -Input [3]: [d_date_sk#19, d_year#20, d_qoy#21] -Arguments: [d_date_sk#19, d_year#20, d_qoy#21] -(23) CometBroadcastHashJoin +(19) CometBroadcastHashJoin Left output [4]: [ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16, i_category#18] Right output [3]: [d_date_sk#19, d_year#20, d_qoy#21] Arguments: [ws_sold_date_sk#16], [d_date_sk#19], Inner -(24) CometProject +(20) CometProject Input [7]: [ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16, i_category#18, d_date_sk#19, d_year#20, d_qoy#21] Arguments: [channel#22, col_name#23, d_year#20, d_qoy#21, i_category#18, ext_sales_price#24], [web AS channel#22, ws_ship_customer_sk#14 AS col_name#23, d_year#20, d_qoy#21, i_category#18, ws_ext_sales_price#15 AS ext_sales_price#24] -(25) Scan parquet spark_catalog.default.catalog_sales +(21) Scan parquet spark_catalog.default.catalog_sales Output [4]: [cs_ship_addr_sk#25, cs_item_sk#26, cs_ext_sales_price#27, cs_sold_date_sk#28] Batched: true Location: InMemoryFileIndex [] @@ -171,83 +139,59 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#28)] PushedFilters: [IsNull(cs_ship_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(26) CometFilter +(22) CometFilter Input [4]: [cs_ship_addr_sk#25, cs_item_sk#26, cs_ext_sales_price#27, cs_sold_date_sk#28] Condition : (isnull(cs_ship_addr_sk#25) AND isnotnull(cs_item_sk#26)) -(27) Scan parquet spark_catalog.default.item +(23) ReusedExchange [Reuses operator id: 5] Output [2]: [i_item_sk#29, i_category#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [i_item_sk#29, i_category#30] -Condition : isnotnull(i_item_sk#29) - -(29) CometBroadcastExchange -Input [2]: [i_item_sk#29, i_category#30] -Arguments: [i_item_sk#29, i_category#30] -(30) CometBroadcastHashJoin +(24) CometBroadcastHashJoin Left output [4]: [cs_ship_addr_sk#25, cs_item_sk#26, cs_ext_sales_price#27, cs_sold_date_sk#28] Right output [2]: [i_item_sk#29, i_category#30] Arguments: [cs_item_sk#26], [i_item_sk#29], Inner -(31) CometProject +(25) CometProject Input [6]: [cs_ship_addr_sk#25, cs_item_sk#26, cs_ext_sales_price#27, cs_sold_date_sk#28, i_item_sk#29, i_category#30] Arguments: [cs_ship_addr_sk#25, cs_ext_sales_price#27, cs_sold_date_sk#28, i_category#30], [cs_ship_addr_sk#25, cs_ext_sales_price#27, cs_sold_date_sk#28, i_category#30] -(32) Scan parquet spark_catalog.default.date_dim +(26) ReusedExchange [Reuses operator id: 10] Output [3]: [d_date_sk#31, d_year#32, d_qoy#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [d_date_sk#31, d_year#32, d_qoy#33] -Condition : isnotnull(d_date_sk#31) - -(34) CometBroadcastExchange -Input [3]: [d_date_sk#31, d_year#32, d_qoy#33] -Arguments: [d_date_sk#31, d_year#32, d_qoy#33] -(35) CometBroadcastHashJoin +(27) CometBroadcastHashJoin Left output [4]: [cs_ship_addr_sk#25, cs_ext_sales_price#27, cs_sold_date_sk#28, i_category#30] Right output [3]: [d_date_sk#31, d_year#32, d_qoy#33] Arguments: [cs_sold_date_sk#28], [d_date_sk#31], Inner -(36) CometProject +(28) CometProject Input [7]: [cs_ship_addr_sk#25, cs_ext_sales_price#27, cs_sold_date_sk#28, i_category#30, d_date_sk#31, d_year#32, d_qoy#33] Arguments: [channel#34, col_name#35, d_year#32, d_qoy#33, i_category#30, ext_sales_price#36], [catalog AS channel#34, cs_ship_addr_sk#25 AS col_name#35, d_year#32, d_qoy#33, i_category#30, cs_ext_sales_price#27 AS ext_sales_price#36] -(37) CometUnion +(29) CometUnion Child 0 Input [6]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, ext_sales_price#12] Child 1 Input [6]: [channel#22, col_name#23, d_year#20, d_qoy#21, i_category#18, ext_sales_price#24] Child 2 Input [6]: [channel#34, col_name#35, d_year#32, d_qoy#33, i_category#30, ext_sales_price#36] -(38) CometHashAggregate +(30) CometHashAggregate Input [6]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, ext_sales_price#12] Keys [5]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6] Functions [2]: [partial_count(1), partial_sum(UnscaledValue(ext_sales_price#12))] -(39) ColumnarToRow [codegen id : 1] +(31) ColumnarToRow [codegen id : 1] Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count#37, sum#38] -(40) Exchange +(32) Exchange Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count#37, sum#38] Arguments: hashpartitioning(channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(41) HashAggregate [codegen id : 2] +(33) HashAggregate [codegen id : 2] Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count#37, sum#38] Keys [5]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6] Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#12))] Aggregate Attributes [2]: [count(1)#39, sum(UnscaledValue(ext_sales_price#12))#40] Results [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count(1)#39 AS sales_cnt#41, MakeDecimal(sum(UnscaledValue(ext_sales_price#12))#40,17,2) AS sales_amt#42] -(42) TakeOrderedAndProject +(34) TakeOrderedAndProject Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#41, sales_amt#42] Arguments: 100, [channel#10 ASC NULLS FIRST, col_name#11 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#9 ASC NULLS FIRST, i_category#6 ASC NULLS FIRST], [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#41, sales_amt#42] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt index 9d582612a..473eef9e4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt @@ -26,21 +26,13 @@ TakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_ CometBroadcastHashJoin [ws_item_sk,i_item_sk] CometFilter [ws_ship_customer_sk,ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] - CometBroadcastExchange #4 - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_category] - CometBroadcastExchange #5 - CometFilter [d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + ReusedExchange [i_item_sk,i_category] #2 + ReusedExchange [d_date_sk,d_year,d_qoy] #3 CometProject [cs_ship_addr_sk,cs_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category] CometBroadcastHashJoin [cs_item_sk,i_item_sk] CometFilter [cs_ship_addr_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometBroadcastExchange #6 - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_category] - CometBroadcastExchange #7 - CometFilter [d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + ReusedExchange [i_item_sk,i_category] #2 + ReusedExchange [d_date_sk,d_year,d_qoy] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt index 423f4e2cc..60b5fd411 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt @@ -1,12 +1,12 @@ == Physical Plan == -TakeOrderedAndProject (105) -+- * HashAggregate (104) - +- Exchange (103) - +- * HashAggregate (102) - +- * Expand (101) - +- Union (100) - :- * Project (37) - : +- * BroadcastHashJoin LeftOuter BuildRight (36) +TakeOrderedAndProject (86) ++- * HashAggregate (85) + +- Exchange (84) + +- * HashAggregate (83) + +- * Expand (82) + +- Union (81) + :- * Project (32) + : +- * BroadcastHashJoin LeftOuter BuildRight (31) : :- * HashAggregate (17) : : +- Exchange (16) : : +- * ColumnarToRow (15) @@ -24,86 +24,67 @@ TakeOrderedAndProject (105) : : +- CometBroadcastExchange (11) : : +- CometFilter (10) : : +- CometScan parquet spark_catalog.default.store (9) - : +- BroadcastExchange (35) - : +- * HashAggregate (34) - : +- Exchange (33) - : +- * ColumnarToRow (32) - : +- CometHashAggregate (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometProject (25) - : : +- CometBroadcastHashJoin (24) + : +- BroadcastExchange (30) + : +- * HashAggregate (29) + : +- Exchange (28) + : +- * ColumnarToRow (27) + : +- CometHashAggregate (26) + : +- CometProject (25) + : +- CometBroadcastHashJoin (24) + : :- CometProject (22) + : : +- CometBroadcastHashJoin (21) : : :- CometFilter (19) : : : +- CometScan parquet spark_catalog.default.store_returns (18) - : : +- CometBroadcastExchange (23) - : : +- CometProject (22) - : : +- CometFilter (21) - : : +- CometScan parquet spark_catalog.default.date_dim (20) - : +- CometBroadcastExchange (28) - : +- CometFilter (27) - : +- CometScan parquet spark_catalog.default.store (26) - :- * Project (62) - : +- * BroadcastNestedLoopJoin Inner BuildLeft (61) - : :- BroadcastExchange (49) - : : +- * HashAggregate (48) - : : +- Exchange (47) - : : +- * ColumnarToRow (46) - : : +- CometHashAggregate (45) - : : +- CometProject (44) - : : +- CometBroadcastHashJoin (43) - : : :- CometScan parquet spark_catalog.default.catalog_sales (38) - : : +- CometBroadcastExchange (42) - : : +- CometProject (41) - : : +- CometFilter (40) - : : +- CometScan parquet spark_catalog.default.date_dim (39) - : +- * HashAggregate (60) - : +- Exchange (59) - : +- * ColumnarToRow (58) - : +- CometHashAggregate (57) - : +- CometProject (56) - : +- CometBroadcastHashJoin (55) - : :- CometScan parquet spark_catalog.default.catalog_returns (50) - : +- CometBroadcastExchange (54) - : +- CometProject (53) - : +- CometFilter (52) - : +- CometScan parquet spark_catalog.default.date_dim (51) - +- * Project (99) - +- * BroadcastHashJoin LeftOuter BuildRight (98) - :- * HashAggregate (79) - : +- Exchange (78) - : +- * ColumnarToRow (77) - : +- CometHashAggregate (76) - : +- CometProject (75) - : +- CometBroadcastHashJoin (74) - : :- CometProject (70) - : : +- CometBroadcastHashJoin (69) - : : :- CometFilter (64) - : : : +- CometScan parquet spark_catalog.default.web_sales (63) - : : +- CometBroadcastExchange (68) - : : +- CometProject (67) - : : +- CometFilter (66) - : : +- CometScan parquet spark_catalog.default.date_dim (65) - : +- CometBroadcastExchange (73) - : +- CometFilter (72) - : +- CometScan parquet spark_catalog.default.web_page (71) - +- BroadcastExchange (97) - +- * HashAggregate (96) - +- Exchange (95) - +- * ColumnarToRow (94) - +- CometHashAggregate (93) - +- CometProject (92) - +- CometBroadcastHashJoin (91) - :- CometProject (87) - : +- CometBroadcastHashJoin (86) - : :- CometFilter (81) - : : +- CometScan parquet spark_catalog.default.web_returns (80) - : +- CometBroadcastExchange (85) - : +- CometProject (84) - : +- CometFilter (83) - : +- CometScan parquet spark_catalog.default.date_dim (82) - +- CometBroadcastExchange (90) - +- CometFilter (89) - +- CometScan parquet spark_catalog.default.web_page (88) + : : +- ReusedExchange (20) + : +- ReusedExchange (23) + :- * Project (51) + : +- * BroadcastNestedLoopJoin Inner BuildLeft (50) + : :- BroadcastExchange (41) + : : +- * HashAggregate (40) + : : +- Exchange (39) + : : +- * ColumnarToRow (38) + : : +- CometHashAggregate (37) + : : +- CometProject (36) + : : +- CometBroadcastHashJoin (35) + : : :- CometScan parquet spark_catalog.default.catalog_sales (33) + : : +- ReusedExchange (34) + : +- * HashAggregate (49) + : +- Exchange (48) + : +- * ColumnarToRow (47) + : +- CometHashAggregate (46) + : +- CometProject (45) + : +- CometBroadcastHashJoin (44) + : :- CometScan parquet spark_catalog.default.catalog_returns (42) + : +- ReusedExchange (43) + +- * Project (80) + +- * BroadcastHashJoin LeftOuter BuildRight (79) + :- * HashAggregate (65) + : +- Exchange (64) + : +- * ColumnarToRow (63) + : +- CometHashAggregate (62) + : +- CometProject (61) + : +- CometBroadcastHashJoin (60) + : :- CometProject (56) + : : +- CometBroadcastHashJoin (55) + : : :- CometFilter (53) + : : : +- CometScan parquet spark_catalog.default.web_sales (52) + : : +- ReusedExchange (54) + : +- CometBroadcastExchange (59) + : +- CometFilter (58) + : +- CometScan parquet spark_catalog.default.web_page (57) + +- BroadcastExchange (78) + +- * HashAggregate (77) + +- Exchange (76) + +- * ColumnarToRow (75) + +- CometHashAggregate (74) + +- CometProject (73) + +- CometBroadcastHashJoin (72) + :- CometProject (70) + : +- CometBroadcastHashJoin (69) + : :- CometFilter (67) + : : +- CometScan parquet spark_catalog.default.web_returns (66) + : +- ReusedExchange (68) + +- ReusedExchange (71) (1) Scan parquet spark_catalog.default.store_sales @@ -201,459 +182,355 @@ ReadSchema: struct - -(21) CometFilter -Input [2]: [d_date_sk#20, d_date#21] -Condition : (((isnotnull(d_date#21) AND (d_date#21 >= 2000-08-03)) AND (d_date#21 <= 2000-09-02)) AND isnotnull(d_date_sk#20)) +(20) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#20] -(22) CometProject -Input [2]: [d_date_sk#20, d_date#21] -Arguments: [d_date_sk#20], [d_date_sk#20] - -(23) CometBroadcastExchange -Input [1]: [d_date_sk#20] -Arguments: [d_date_sk#20] - -(24) CometBroadcastHashJoin +(21) CometBroadcastHashJoin Left output [4]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, sr_returned_date_sk#18] Right output [1]: [d_date_sk#20] Arguments: [sr_returned_date_sk#18], [d_date_sk#20], Inner -(25) CometProject +(22) CometProject Input [5]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, sr_returned_date_sk#18, d_date_sk#20] Arguments: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17], [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17] -(26) Scan parquet spark_catalog.default.store -Output [1]: [s_store_sk#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(27) CometFilter -Input [1]: [s_store_sk#22] -Condition : isnotnull(s_store_sk#22) +(23) ReusedExchange [Reuses operator id: 11] +Output [1]: [s_store_sk#21] -(28) CometBroadcastExchange -Input [1]: [s_store_sk#22] -Arguments: [s_store_sk#22] - -(29) CometBroadcastHashJoin +(24) CometBroadcastHashJoin Left output [3]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17] -Right output [1]: [s_store_sk#22] -Arguments: [sr_store_sk#15], [s_store_sk#22], Inner +Right output [1]: [s_store_sk#21] +Arguments: [sr_store_sk#15], [s_store_sk#21], Inner -(30) CometProject -Input [4]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, s_store_sk#22] -Arguments: [sr_return_amt#16, sr_net_loss#17, s_store_sk#22], [sr_return_amt#16, sr_net_loss#17, s_store_sk#22] +(25) CometProject +Input [4]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, s_store_sk#21] +Arguments: [sr_return_amt#16, sr_net_loss#17, s_store_sk#21], [sr_return_amt#16, sr_net_loss#17, s_store_sk#21] -(31) CometHashAggregate -Input [3]: [sr_return_amt#16, sr_net_loss#17, s_store_sk#22] -Keys [1]: [s_store_sk#22] +(26) CometHashAggregate +Input [3]: [sr_return_amt#16, sr_net_loss#17, s_store_sk#21] +Keys [1]: [s_store_sk#21] Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#16)), partial_sum(UnscaledValue(sr_net_loss#17))] -(32) ColumnarToRow [codegen id : 2] -Input [3]: [s_store_sk#22, sum#23, sum#24] +(27) ColumnarToRow [codegen id : 2] +Input [3]: [s_store_sk#21, sum#22, sum#23] -(33) Exchange -Input [3]: [s_store_sk#22, sum#23, sum#24] -Arguments: hashpartitioning(s_store_sk#22, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(28) Exchange +Input [3]: [s_store_sk#21, sum#22, sum#23] +Arguments: hashpartitioning(s_store_sk#21, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(34) HashAggregate [codegen id : 3] -Input [3]: [s_store_sk#22, sum#23, sum#24] -Keys [1]: [s_store_sk#22] +(29) HashAggregate [codegen id : 3] +Input [3]: [s_store_sk#21, sum#22, sum#23] +Keys [1]: [s_store_sk#21] Functions [2]: [sum(UnscaledValue(sr_return_amt#16)), sum(UnscaledValue(sr_net_loss#17))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#16))#25, sum(UnscaledValue(sr_net_loss#17))#26] -Results [3]: [s_store_sk#22, MakeDecimal(sum(UnscaledValue(sr_return_amt#16))#25,17,2) AS returns#27, MakeDecimal(sum(UnscaledValue(sr_net_loss#17))#26,17,2) AS profit_loss#28] +Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#16))#24, sum(UnscaledValue(sr_net_loss#17))#25] +Results [3]: [s_store_sk#21, MakeDecimal(sum(UnscaledValue(sr_return_amt#16))#24,17,2) AS returns#26, MakeDecimal(sum(UnscaledValue(sr_net_loss#17))#25,17,2) AS profit_loss#27] -(35) BroadcastExchange -Input [3]: [s_store_sk#22, returns#27, profit_loss#28] +(30) BroadcastExchange +Input [3]: [s_store_sk#21, returns#26, profit_loss#27] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(36) BroadcastHashJoin [codegen id : 4] +(31) BroadcastHashJoin [codegen id : 4] Left keys [1]: [s_store_sk#8] -Right keys [1]: [s_store_sk#22] +Right keys [1]: [s_store_sk#21] Join type: LeftOuter Join condition: None -(37) Project [codegen id : 4] -Output [5]: [sales#13, coalesce(returns#27, 0.00) AS returns#29, (profit#14 - coalesce(profit_loss#28, 0.00)) AS profit#30, store channel AS channel#31, s_store_sk#8 AS id#32] -Input [6]: [s_store_sk#8, sales#13, profit#14, s_store_sk#22, returns#27, profit_loss#28] +(32) Project [codegen id : 4] +Output [5]: [sales#13, coalesce(returns#26, 0.00) AS returns#28, (profit#14 - coalesce(profit_loss#27, 0.00)) AS profit#29, store channel AS channel#30, s_store_sk#8 AS id#31] +Input [6]: [s_store_sk#8, sales#13, profit#14, s_store_sk#21, returns#26, profit_loss#27] -(38) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#33, cs_ext_sales_price#34, cs_net_profit#35, cs_sold_date_sk#36] +(33) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34, cs_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#36), dynamicpruningexpression(cs_sold_date_sk#36 IN dynamicpruning#37)] +PartitionFilters: [isnotnull(cs_sold_date_sk#35), dynamicpruningexpression(cs_sold_date_sk#35 IN dynamicpruning#36)] ReadSchema: struct -(39) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#38, d_date#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-03), LessThanOrEqual(d_date,2000-09-02), IsNotNull(d_date_sk)] -ReadSchema: struct +(34) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#37] -(40) CometFilter -Input [2]: [d_date_sk#38, d_date#39] -Condition : (((isnotnull(d_date#39) AND (d_date#39 >= 2000-08-03)) AND (d_date#39 <= 2000-09-02)) AND isnotnull(d_date_sk#38)) +(35) CometBroadcastHashJoin +Left output [4]: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34, cs_sold_date_sk#35] +Right output [1]: [d_date_sk#37] +Arguments: [cs_sold_date_sk#35], [d_date_sk#37], Inner -(41) CometProject -Input [2]: [d_date_sk#38, d_date#39] -Arguments: [d_date_sk#38], [d_date_sk#38] +(36) CometProject +Input [5]: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34, cs_sold_date_sk#35, d_date_sk#37] +Arguments: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34], [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34] -(42) CometBroadcastExchange -Input [1]: [d_date_sk#38] -Arguments: [d_date_sk#38] +(37) CometHashAggregate +Input [3]: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34] +Keys [1]: [cs_call_center_sk#32] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#33)), partial_sum(UnscaledValue(cs_net_profit#34))] -(43) CometBroadcastHashJoin -Left output [4]: [cs_call_center_sk#33, cs_ext_sales_price#34, cs_net_profit#35, cs_sold_date_sk#36] -Right output [1]: [d_date_sk#38] -Arguments: [cs_sold_date_sk#36], [d_date_sk#38], Inner +(38) ColumnarToRow [codegen id : 5] +Input [3]: [cs_call_center_sk#32, sum#38, sum#39] -(44) CometProject -Input [5]: [cs_call_center_sk#33, cs_ext_sales_price#34, cs_net_profit#35, cs_sold_date_sk#36, d_date_sk#38] -Arguments: [cs_call_center_sk#33, cs_ext_sales_price#34, cs_net_profit#35], [cs_call_center_sk#33, cs_ext_sales_price#34, cs_net_profit#35] +(39) Exchange +Input [3]: [cs_call_center_sk#32, sum#38, sum#39] +Arguments: hashpartitioning(cs_call_center_sk#32, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(45) CometHashAggregate -Input [3]: [cs_call_center_sk#33, cs_ext_sales_price#34, cs_net_profit#35] -Keys [1]: [cs_call_center_sk#33] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#34)), partial_sum(UnscaledValue(cs_net_profit#35))] +(40) HashAggregate [codegen id : 6] +Input [3]: [cs_call_center_sk#32, sum#38, sum#39] +Keys [1]: [cs_call_center_sk#32] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#33)), sum(UnscaledValue(cs_net_profit#34))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#33))#40, sum(UnscaledValue(cs_net_profit#34))#41] +Results [3]: [cs_call_center_sk#32, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#33))#40,17,2) AS sales#42, MakeDecimal(sum(UnscaledValue(cs_net_profit#34))#41,17,2) AS profit#43] -(46) ColumnarToRow [codegen id : 5] -Input [3]: [cs_call_center_sk#33, sum#40, sum#41] - -(47) Exchange -Input [3]: [cs_call_center_sk#33, sum#40, sum#41] -Arguments: hashpartitioning(cs_call_center_sk#33, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(48) HashAggregate [codegen id : 6] -Input [3]: [cs_call_center_sk#33, sum#40, sum#41] -Keys [1]: [cs_call_center_sk#33] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#34)), sum(UnscaledValue(cs_net_profit#35))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#34))#42, sum(UnscaledValue(cs_net_profit#35))#43] -Results [3]: [cs_call_center_sk#33, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#34))#42,17,2) AS sales#44, MakeDecimal(sum(UnscaledValue(cs_net_profit#35))#43,17,2) AS profit#45] - -(49) BroadcastExchange -Input [3]: [cs_call_center_sk#33, sales#44, profit#45] +(41) BroadcastExchange +Input [3]: [cs_call_center_sk#32, sales#42, profit#43] Arguments: IdentityBroadcastMode, [plan_id=5] -(50) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] +(42) Scan parquet spark_catalog.default.catalog_returns +Output [3]: [cr_return_amount#44, cr_net_loss#45, cr_returned_date_sk#46] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#48), dynamicpruningexpression(cr_returned_date_sk#48 IN dynamicpruning#49)] +PartitionFilters: [isnotnull(cr_returned_date_sk#46), dynamicpruningexpression(cr_returned_date_sk#46 IN dynamicpruning#47)] ReadSchema: struct -(51) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#50, d_date#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-03), LessThanOrEqual(d_date,2000-09-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(52) CometFilter -Input [2]: [d_date_sk#50, d_date#51] -Condition : (((isnotnull(d_date#51) AND (d_date#51 >= 2000-08-03)) AND (d_date#51 <= 2000-09-02)) AND isnotnull(d_date_sk#50)) +(43) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#48] -(53) CometProject -Input [2]: [d_date_sk#50, d_date#51] -Arguments: [d_date_sk#50], [d_date_sk#50] +(44) CometBroadcastHashJoin +Left output [3]: [cr_return_amount#44, cr_net_loss#45, cr_returned_date_sk#46] +Right output [1]: [d_date_sk#48] +Arguments: [cr_returned_date_sk#46], [d_date_sk#48], Inner -(54) CometBroadcastExchange -Input [1]: [d_date_sk#50] -Arguments: [d_date_sk#50] - -(55) CometBroadcastHashJoin -Left output [3]: [cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] -Right output [1]: [d_date_sk#50] -Arguments: [cr_returned_date_sk#48], [d_date_sk#50], Inner - -(56) CometProject -Input [4]: [cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48, d_date_sk#50] -Arguments: [cr_return_amount#46, cr_net_loss#47], [cr_return_amount#46, cr_net_loss#47] +(45) CometProject +Input [4]: [cr_return_amount#44, cr_net_loss#45, cr_returned_date_sk#46, d_date_sk#48] +Arguments: [cr_return_amount#44, cr_net_loss#45], [cr_return_amount#44, cr_net_loss#45] -(57) CometHashAggregate -Input [2]: [cr_return_amount#46, cr_net_loss#47] +(46) CometHashAggregate +Input [2]: [cr_return_amount#44, cr_net_loss#45] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#46)), partial_sum(UnscaledValue(cr_net_loss#47))] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#44)), partial_sum(UnscaledValue(cr_net_loss#45))] -(58) ColumnarToRow [codegen id : 7] -Input [2]: [sum#52, sum#53] +(47) ColumnarToRow [codegen id : 7] +Input [2]: [sum#49, sum#50] -(59) Exchange -Input [2]: [sum#52, sum#53] +(48) Exchange +Input [2]: [sum#49, sum#50] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(60) HashAggregate -Input [2]: [sum#52, sum#53] +(49) HashAggregate +Input [2]: [sum#49, sum#50] Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#46)), sum(UnscaledValue(cr_net_loss#47))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#46))#54, sum(UnscaledValue(cr_net_loss#47))#55] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#46))#54,17,2) AS returns#56, MakeDecimal(sum(UnscaledValue(cr_net_loss#47))#55,17,2) AS profit_loss#57] +Functions [2]: [sum(UnscaledValue(cr_return_amount#44)), sum(UnscaledValue(cr_net_loss#45))] +Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#44))#51, sum(UnscaledValue(cr_net_loss#45))#52] +Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#44))#51,17,2) AS returns#53, MakeDecimal(sum(UnscaledValue(cr_net_loss#45))#52,17,2) AS profit_loss#54] -(61) BroadcastNestedLoopJoin [codegen id : 8] +(50) BroadcastNestedLoopJoin [codegen id : 8] Join type: Inner Join condition: None -(62) Project [codegen id : 8] -Output [5]: [sales#44, returns#56, (profit#45 - profit_loss#57) AS profit#58, catalog channel AS channel#59, cs_call_center_sk#33 AS id#60] -Input [5]: [cs_call_center_sk#33, sales#44, profit#45, returns#56, profit_loss#57] +(51) Project [codegen id : 8] +Output [5]: [sales#42, returns#53, (profit#43 - profit_loss#54) AS profit#55, catalog channel AS channel#56, cs_call_center_sk#32 AS id#57] +Input [5]: [cs_call_center_sk#32, sales#42, profit#43, returns#53, profit_loss#54] -(63) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#61, ws_ext_sales_price#62, ws_net_profit#63, ws_sold_date_sk#64] +(52) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, ws_sold_date_sk#61] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#64), dynamicpruningexpression(ws_sold_date_sk#64 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(ws_sold_date_sk#61), dynamicpruningexpression(ws_sold_date_sk#61 IN dynamicpruning#62)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct -(64) CometFilter -Input [4]: [ws_web_page_sk#61, ws_ext_sales_price#62, ws_net_profit#63, ws_sold_date_sk#64] -Condition : isnotnull(ws_web_page_sk#61) +(53) CometFilter +Input [4]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, ws_sold_date_sk#61] +Condition : isnotnull(ws_web_page_sk#58) -(65) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#66, d_date#67] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-03), LessThanOrEqual(d_date,2000-09-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(66) CometFilter -Input [2]: [d_date_sk#66, d_date#67] -Condition : (((isnotnull(d_date#67) AND (d_date#67 >= 2000-08-03)) AND (d_date#67 <= 2000-09-02)) AND isnotnull(d_date_sk#66)) +(54) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#63] -(67) CometProject -Input [2]: [d_date_sk#66, d_date#67] -Arguments: [d_date_sk#66], [d_date_sk#66] - -(68) CometBroadcastExchange -Input [1]: [d_date_sk#66] -Arguments: [d_date_sk#66] - -(69) CometBroadcastHashJoin -Left output [4]: [ws_web_page_sk#61, ws_ext_sales_price#62, ws_net_profit#63, ws_sold_date_sk#64] -Right output [1]: [d_date_sk#66] -Arguments: [ws_sold_date_sk#64], [d_date_sk#66], Inner +(55) CometBroadcastHashJoin +Left output [4]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, ws_sold_date_sk#61] +Right output [1]: [d_date_sk#63] +Arguments: [ws_sold_date_sk#61], [d_date_sk#63], Inner -(70) CometProject -Input [5]: [ws_web_page_sk#61, ws_ext_sales_price#62, ws_net_profit#63, ws_sold_date_sk#64, d_date_sk#66] -Arguments: [ws_web_page_sk#61, ws_ext_sales_price#62, ws_net_profit#63], [ws_web_page_sk#61, ws_ext_sales_price#62, ws_net_profit#63] +(56) CometProject +Input [5]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, ws_sold_date_sk#61, d_date_sk#63] +Arguments: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60], [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60] -(71) Scan parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#68] +(57) Scan parquet spark_catalog.default.web_page +Output [1]: [wp_web_page_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct -(72) CometFilter -Input [1]: [wp_web_page_sk#68] -Condition : isnotnull(wp_web_page_sk#68) +(58) CometFilter +Input [1]: [wp_web_page_sk#64] +Condition : isnotnull(wp_web_page_sk#64) -(73) CometBroadcastExchange -Input [1]: [wp_web_page_sk#68] -Arguments: [wp_web_page_sk#68] +(59) CometBroadcastExchange +Input [1]: [wp_web_page_sk#64] +Arguments: [wp_web_page_sk#64] -(74) CometBroadcastHashJoin -Left output [3]: [ws_web_page_sk#61, ws_ext_sales_price#62, ws_net_profit#63] -Right output [1]: [wp_web_page_sk#68] -Arguments: [ws_web_page_sk#61], [wp_web_page_sk#68], Inner +(60) CometBroadcastHashJoin +Left output [3]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60] +Right output [1]: [wp_web_page_sk#64] +Arguments: [ws_web_page_sk#58], [wp_web_page_sk#64], Inner -(75) CometProject -Input [4]: [ws_web_page_sk#61, ws_ext_sales_price#62, ws_net_profit#63, wp_web_page_sk#68] -Arguments: [ws_ext_sales_price#62, ws_net_profit#63, wp_web_page_sk#68], [ws_ext_sales_price#62, ws_net_profit#63, wp_web_page_sk#68] +(61) CometProject +Input [4]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, wp_web_page_sk#64] +Arguments: [ws_ext_sales_price#59, ws_net_profit#60, wp_web_page_sk#64], [ws_ext_sales_price#59, ws_net_profit#60, wp_web_page_sk#64] -(76) CometHashAggregate -Input [3]: [ws_ext_sales_price#62, ws_net_profit#63, wp_web_page_sk#68] -Keys [1]: [wp_web_page_sk#68] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#62)), partial_sum(UnscaledValue(ws_net_profit#63))] +(62) CometHashAggregate +Input [3]: [ws_ext_sales_price#59, ws_net_profit#60, wp_web_page_sk#64] +Keys [1]: [wp_web_page_sk#64] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#59)), partial_sum(UnscaledValue(ws_net_profit#60))] -(77) ColumnarToRow [codegen id : 9] -Input [3]: [wp_web_page_sk#68, sum#69, sum#70] +(63) ColumnarToRow [codegen id : 9] +Input [3]: [wp_web_page_sk#64, sum#65, sum#66] -(78) Exchange -Input [3]: [wp_web_page_sk#68, sum#69, sum#70] -Arguments: hashpartitioning(wp_web_page_sk#68, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(64) Exchange +Input [3]: [wp_web_page_sk#64, sum#65, sum#66] +Arguments: hashpartitioning(wp_web_page_sk#64, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(79) HashAggregate [codegen id : 12] -Input [3]: [wp_web_page_sk#68, sum#69, sum#70] -Keys [1]: [wp_web_page_sk#68] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#62)), sum(UnscaledValue(ws_net_profit#63))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#62))#71, sum(UnscaledValue(ws_net_profit#63))#72] -Results [3]: [wp_web_page_sk#68, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#62))#71,17,2) AS sales#73, MakeDecimal(sum(UnscaledValue(ws_net_profit#63))#72,17,2) AS profit#74] +(65) HashAggregate [codegen id : 12] +Input [3]: [wp_web_page_sk#64, sum#65, sum#66] +Keys [1]: [wp_web_page_sk#64] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#59)), sum(UnscaledValue(ws_net_profit#60))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#59))#67, sum(UnscaledValue(ws_net_profit#60))#68] +Results [3]: [wp_web_page_sk#64, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#59))#67,17,2) AS sales#69, MakeDecimal(sum(UnscaledValue(ws_net_profit#60))#68,17,2) AS profit#70] -(80) Scan parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#75, wr_return_amt#76, wr_net_loss#77, wr_returned_date_sk#78] +(66) Scan parquet spark_catalog.default.web_returns +Output [4]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wr_returned_date_sk#74] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#78), dynamicpruningexpression(wr_returned_date_sk#78 IN dynamicpruning#79)] +PartitionFilters: [isnotnull(wr_returned_date_sk#74), dynamicpruningexpression(wr_returned_date_sk#74 IN dynamicpruning#75)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct -(81) CometFilter -Input [4]: [wr_web_page_sk#75, wr_return_amt#76, wr_net_loss#77, wr_returned_date_sk#78] -Condition : isnotnull(wr_web_page_sk#75) - -(82) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#80, d_date#81] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-03), LessThanOrEqual(d_date,2000-09-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(83) CometFilter -Input [2]: [d_date_sk#80, d_date#81] -Condition : (((isnotnull(d_date#81) AND (d_date#81 >= 2000-08-03)) AND (d_date#81 <= 2000-09-02)) AND isnotnull(d_date_sk#80)) +(67) CometFilter +Input [4]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wr_returned_date_sk#74] +Condition : isnotnull(wr_web_page_sk#71) -(84) CometProject -Input [2]: [d_date_sk#80, d_date#81] -Arguments: [d_date_sk#80], [d_date_sk#80] +(68) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#76] -(85) CometBroadcastExchange -Input [1]: [d_date_sk#80] -Arguments: [d_date_sk#80] - -(86) CometBroadcastHashJoin -Left output [4]: [wr_web_page_sk#75, wr_return_amt#76, wr_net_loss#77, wr_returned_date_sk#78] -Right output [1]: [d_date_sk#80] -Arguments: [wr_returned_date_sk#78], [d_date_sk#80], Inner - -(87) CometProject -Input [5]: [wr_web_page_sk#75, wr_return_amt#76, wr_net_loss#77, wr_returned_date_sk#78, d_date_sk#80] -Arguments: [wr_web_page_sk#75, wr_return_amt#76, wr_net_loss#77], [wr_web_page_sk#75, wr_return_amt#76, wr_net_loss#77] - -(88) Scan parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#82] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(89) CometFilter -Input [1]: [wp_web_page_sk#82] -Condition : isnotnull(wp_web_page_sk#82) - -(90) CometBroadcastExchange -Input [1]: [wp_web_page_sk#82] -Arguments: [wp_web_page_sk#82] - -(91) CometBroadcastHashJoin -Left output [3]: [wr_web_page_sk#75, wr_return_amt#76, wr_net_loss#77] -Right output [1]: [wp_web_page_sk#82] -Arguments: [wr_web_page_sk#75], [wp_web_page_sk#82], Inner - -(92) CometProject -Input [4]: [wr_web_page_sk#75, wr_return_amt#76, wr_net_loss#77, wp_web_page_sk#82] -Arguments: [wr_return_amt#76, wr_net_loss#77, wp_web_page_sk#82], [wr_return_amt#76, wr_net_loss#77, wp_web_page_sk#82] - -(93) CometHashAggregate -Input [3]: [wr_return_amt#76, wr_net_loss#77, wp_web_page_sk#82] -Keys [1]: [wp_web_page_sk#82] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#76)), partial_sum(UnscaledValue(wr_net_loss#77))] - -(94) ColumnarToRow [codegen id : 10] -Input [3]: [wp_web_page_sk#82, sum#83, sum#84] - -(95) Exchange -Input [3]: [wp_web_page_sk#82, sum#83, sum#84] -Arguments: hashpartitioning(wp_web_page_sk#82, 5), ENSURE_REQUIREMENTS, [plan_id=8] - -(96) HashAggregate [codegen id : 11] -Input [3]: [wp_web_page_sk#82, sum#83, sum#84] -Keys [1]: [wp_web_page_sk#82] -Functions [2]: [sum(UnscaledValue(wr_return_amt#76)), sum(UnscaledValue(wr_net_loss#77))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#76))#85, sum(UnscaledValue(wr_net_loss#77))#86] -Results [3]: [wp_web_page_sk#82, MakeDecimal(sum(UnscaledValue(wr_return_amt#76))#85,17,2) AS returns#87, MakeDecimal(sum(UnscaledValue(wr_net_loss#77))#86,17,2) AS profit_loss#88] +(69) CometBroadcastHashJoin +Left output [4]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wr_returned_date_sk#74] +Right output [1]: [d_date_sk#76] +Arguments: [wr_returned_date_sk#74], [d_date_sk#76], Inner -(97) BroadcastExchange -Input [3]: [wp_web_page_sk#82, returns#87, profit_loss#88] +(70) CometProject +Input [5]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wr_returned_date_sk#74, d_date_sk#76] +Arguments: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73], [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73] + +(71) ReusedExchange [Reuses operator id: 59] +Output [1]: [wp_web_page_sk#77] + +(72) CometBroadcastHashJoin +Left output [3]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73] +Right output [1]: [wp_web_page_sk#77] +Arguments: [wr_web_page_sk#71], [wp_web_page_sk#77], Inner + +(73) CometProject +Input [4]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wp_web_page_sk#77] +Arguments: [wr_return_amt#72, wr_net_loss#73, wp_web_page_sk#77], [wr_return_amt#72, wr_net_loss#73, wp_web_page_sk#77] + +(74) CometHashAggregate +Input [3]: [wr_return_amt#72, wr_net_loss#73, wp_web_page_sk#77] +Keys [1]: [wp_web_page_sk#77] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#72)), partial_sum(UnscaledValue(wr_net_loss#73))] + +(75) ColumnarToRow [codegen id : 10] +Input [3]: [wp_web_page_sk#77, sum#78, sum#79] + +(76) Exchange +Input [3]: [wp_web_page_sk#77, sum#78, sum#79] +Arguments: hashpartitioning(wp_web_page_sk#77, 5), ENSURE_REQUIREMENTS, [plan_id=8] + +(77) HashAggregate [codegen id : 11] +Input [3]: [wp_web_page_sk#77, sum#78, sum#79] +Keys [1]: [wp_web_page_sk#77] +Functions [2]: [sum(UnscaledValue(wr_return_amt#72)), sum(UnscaledValue(wr_net_loss#73))] +Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#72))#80, sum(UnscaledValue(wr_net_loss#73))#81] +Results [3]: [wp_web_page_sk#77, MakeDecimal(sum(UnscaledValue(wr_return_amt#72))#80,17,2) AS returns#82, MakeDecimal(sum(UnscaledValue(wr_net_loss#73))#81,17,2) AS profit_loss#83] + +(78) BroadcastExchange +Input [3]: [wp_web_page_sk#77, returns#82, profit_loss#83] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -(98) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [wp_web_page_sk#68] -Right keys [1]: [wp_web_page_sk#82] +(79) BroadcastHashJoin [codegen id : 12] +Left keys [1]: [wp_web_page_sk#64] +Right keys [1]: [wp_web_page_sk#77] Join type: LeftOuter Join condition: None -(99) Project [codegen id : 12] -Output [5]: [sales#73, coalesce(returns#87, 0.00) AS returns#89, (profit#74 - coalesce(profit_loss#88, 0.00)) AS profit#90, web channel AS channel#91, wp_web_page_sk#68 AS id#92] -Input [6]: [wp_web_page_sk#68, sales#73, profit#74, wp_web_page_sk#82, returns#87, profit_loss#88] +(80) Project [codegen id : 12] +Output [5]: [sales#69, coalesce(returns#82, 0.00) AS returns#84, (profit#70 - coalesce(profit_loss#83, 0.00)) AS profit#85, web channel AS channel#86, wp_web_page_sk#64 AS id#87] +Input [6]: [wp_web_page_sk#64, sales#69, profit#70, wp_web_page_sk#77, returns#82, profit_loss#83] -(100) Union +(81) Union -(101) Expand [codegen id : 13] -Input [5]: [sales#13, returns#29, profit#30, channel#31, id#32] -Arguments: [[sales#13, returns#29, profit#30, channel#31, id#32, 0], [sales#13, returns#29, profit#30, channel#31, null, 1], [sales#13, returns#29, profit#30, null, null, 3]], [sales#13, returns#29, profit#30, channel#93, id#94, spark_grouping_id#95] +(82) Expand [codegen id : 13] +Input [5]: [sales#13, returns#28, profit#29, channel#30, id#31] +Arguments: [[sales#13, returns#28, profit#29, channel#30, id#31, 0], [sales#13, returns#28, profit#29, channel#30, null, 1], [sales#13, returns#28, profit#29, null, null, 3]], [sales#13, returns#28, profit#29, channel#88, id#89, spark_grouping_id#90] -(102) HashAggregate [codegen id : 13] -Input [6]: [sales#13, returns#29, profit#30, channel#93, id#94, spark_grouping_id#95] -Keys [3]: [channel#93, id#94, spark_grouping_id#95] -Functions [3]: [partial_sum(sales#13), partial_sum(returns#29), partial_sum(profit#30)] -Aggregate Attributes [6]: [sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101] -Results [9]: [channel#93, id#94, spark_grouping_id#95, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107] +(83) HashAggregate [codegen id : 13] +Input [6]: [sales#13, returns#28, profit#29, channel#88, id#89, spark_grouping_id#90] +Keys [3]: [channel#88, id#89, spark_grouping_id#90] +Functions [3]: [partial_sum(sales#13), partial_sum(returns#28), partial_sum(profit#29)] +Aggregate Attributes [6]: [sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] +Results [9]: [channel#88, id#89, spark_grouping_id#90, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -(103) Exchange -Input [9]: [channel#93, id#94, spark_grouping_id#95, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107] -Arguments: hashpartitioning(channel#93, id#94, spark_grouping_id#95, 5), ENSURE_REQUIREMENTS, [plan_id=10] +(84) Exchange +Input [9]: [channel#88, id#89, spark_grouping_id#90, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102] +Arguments: hashpartitioning(channel#88, id#89, spark_grouping_id#90, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(104) HashAggregate [codegen id : 14] -Input [9]: [channel#93, id#94, spark_grouping_id#95, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107] -Keys [3]: [channel#93, id#94, spark_grouping_id#95] -Functions [3]: [sum(sales#13), sum(returns#29), sum(profit#30)] -Aggregate Attributes [3]: [sum(sales#13)#108, sum(returns#29)#109, sum(profit#30)#110] -Results [5]: [channel#93, id#94, sum(sales#13)#108 AS sales#111, sum(returns#29)#109 AS returns#112, sum(profit#30)#110 AS profit#113] +(85) HashAggregate [codegen id : 14] +Input [9]: [channel#88, id#89, spark_grouping_id#90, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102] +Keys [3]: [channel#88, id#89, spark_grouping_id#90] +Functions [3]: [sum(sales#13), sum(returns#28), sum(profit#29)] +Aggregate Attributes [3]: [sum(sales#13)#103, sum(returns#28)#104, sum(profit#29)#105] +Results [5]: [channel#88, id#89, sum(sales#13)#103 AS sales#106, sum(returns#28)#104 AS returns#107, sum(profit#29)#105 AS profit#108] -(105) TakeOrderedAndProject -Input [5]: [channel#93, id#94, sales#111, returns#112, profit#113] -Arguments: 100, [channel#93 ASC NULLS FIRST, id#94 ASC NULLS FIRST], [channel#93, id#94, sales#111, returns#112, profit#113] +(86) TakeOrderedAndProject +Input [5]: [channel#88, id#89, sales#106, returns#107, profit#108] +Arguments: 100, [channel#88 ASC NULLS FIRST, id#89 ASC NULLS FIRST], [channel#88, id#89, sales#106, returns#107, profit#108] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (110) -+- * ColumnarToRow (109) - +- CometProject (108) - +- CometFilter (107) - +- CometScan parquet spark_catalog.default.date_dim (106) +BroadcastExchange (91) ++- * ColumnarToRow (90) + +- CometProject (89) + +- CometFilter (88) + +- CometScan parquet spark_catalog.default.date_dim (87) -(106) Scan parquet spark_catalog.default.date_dim +(87) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#6, d_date#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-03), LessThanOrEqual(d_date,2000-09-02), IsNotNull(d_date_sk)] ReadSchema: struct -(107) CometFilter +(88) CometFilter Input [2]: [d_date_sk#6, d_date#7] Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 2000-08-03)) AND (d_date#7 <= 2000-09-02)) AND isnotnull(d_date_sk#6)) -(108) CometProject +(89) CometProject Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(109) ColumnarToRow [codegen id : 1] +(90) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(110) BroadcastExchange +(91) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] Subquery:2 Hosting operator id = 18 Hosting Expression = sr_returned_date_sk#18 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 38 Hosting Expression = cs_sold_date_sk#36 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#35 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 50 Hosting Expression = cr_returned_date_sk#48 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 42 Hosting Expression = cr_returned_date_sk#46 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#64 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 52 Hosting Expression = ws_sold_date_sk#61 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 80 Hosting Expression = wr_returned_date_sk#78 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 66 Hosting Expression = wr_returned_date_sk#74 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt index 07accf8b5..590c59fdc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt @@ -56,22 +56,17 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometFilter [sr_store_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #8 - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #9 - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] + ReusedExchange [d_date_sk] #4 + ReusedExchange [s_store_sk] #5 WholeStageCodegen (8) Project [sales,returns,profit,profit_loss,cs_call_center_sk] BroadcastNestedLoopJoin InputAdapter - BroadcastExchange #10 + BroadcastExchange #8 WholeStageCodegen (6) HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] InputAdapter - Exchange [cs_call_center_sk] #11 + Exchange [cs_call_center_sk] #9 WholeStageCodegen (5) ColumnarToRow InputAdapter @@ -80,13 +75,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #12 - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + ReusedExchange [d_date_sk] #4 HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] InputAdapter - Exchange #13 + Exchange #10 WholeStageCodegen (7) ColumnarToRow InputAdapter @@ -95,16 +87,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #14 - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + ReusedExchange [d_date_sk] #4 WholeStageCodegen (12) Project [sales,returns,profit,profit_loss,wp_web_page_sk] BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] InputAdapter - Exchange [wp_web_page_sk] #15 + Exchange [wp_web_page_sk] #11 WholeStageCodegen (9) ColumnarToRow InputAdapter @@ -116,19 +105,16 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometFilter [ws_web_page_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #16 - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #17 + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange #12 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter - BroadcastExchange #18 + BroadcastExchange #13 WholeStageCodegen (11) HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] InputAdapter - Exchange [wp_web_page_sk] #19 + Exchange [wp_web_page_sk] #14 WholeStageCodegen (10) ColumnarToRow InputAdapter @@ -140,10 +126,5 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometFilter [wr_web_page_sk] CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #20 - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #21 - CometFilter [wp_web_page_sk] - CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] + ReusedExchange [d_date_sk] #4 + ReusedExchange [wp_web_page_sk] #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt index 2ba867f94..e69a8d82f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (64) -+- * Project (63) - +- * BroadcastHashJoin Inner BuildRight (62) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) +TakeOrderedAndProject (50) ++- * Project (49) + +- * BroadcastHashJoin Inner BuildRight (48) + :- * Project (34) + : +- * BroadcastHashJoin Inner BuildRight (33) : :- * HashAggregate (19) : : +- Exchange (18) : : +- * ColumnarToRow (17) @@ -23,46 +23,32 @@ TakeOrderedAndProject (64) : : :- CometFilter (9) : : : +- CometScan parquet spark_catalog.default.date_dim (8) : : +- ReusedExchange (10) - : +- BroadcastExchange (39) - : +- * HashAggregate (38) - : +- Exchange (37) - : +- * ColumnarToRow (36) - : +- CometHashAggregate (35) - : +- CometProject (34) - : +- CometBroadcastHashJoin (33) - : :- CometProject (26) - : : +- CometBroadcastHashJoin (25) + : +- BroadcastExchange (32) + : +- * HashAggregate (31) + : +- Exchange (30) + : +- * ColumnarToRow (29) + : +- CometHashAggregate (28) + : +- CometProject (27) + : +- CometBroadcastHashJoin (26) + : :- CometProject (24) + : : +- CometBroadcastHashJoin (23) : : :- CometFilter (21) : : : +- CometScan parquet spark_catalog.default.catalog_returns (20) - : : +- CometBroadcastExchange (24) - : : +- CometFilter (23) - : : +- CometScan parquet spark_catalog.default.item (22) - : +- CometBroadcastExchange (32) - : +- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometFilter (28) - : : +- CometScan parquet spark_catalog.default.date_dim (27) - : +- ReusedExchange (29) - +- BroadcastExchange (61) - +- * HashAggregate (60) - +- Exchange (59) - +- * ColumnarToRow (58) - +- CometHashAggregate (57) - +- CometProject (56) - +- CometBroadcastHashJoin (55) - :- CometProject (48) - : +- CometBroadcastHashJoin (47) - : :- CometFilter (43) - : : +- CometScan parquet spark_catalog.default.web_returns (42) - : +- CometBroadcastExchange (46) - : +- CometFilter (45) - : +- CometScan parquet spark_catalog.default.item (44) - +- CometBroadcastExchange (54) - +- CometProject (53) - +- CometBroadcastHashJoin (52) - :- CometFilter (50) - : +- CometScan parquet spark_catalog.default.date_dim (49) - +- ReusedExchange (51) + : : +- ReusedExchange (22) + : +- ReusedExchange (25) + +- BroadcastExchange (47) + +- * HashAggregate (46) + +- Exchange (45) + +- * ColumnarToRow (44) + +- CometHashAggregate (43) + +- CometProject (42) + +- CometBroadcastHashJoin (41) + :- CometProject (39) + : +- CometBroadcastHashJoin (38) + : :- CometFilter (36) + : : +- CometScan parquet spark_catalog.default.web_returns (35) + : +- ReusedExchange (37) + +- ReusedExchange (40) (1) Scan parquet spark_catalog.default.store_returns @@ -112,7 +98,7 @@ ReadSchema: struct Input [2]: [d_date_sk#7, d_date#8] Condition : isnotnull(d_date_sk#7) -(10) ReusedExchange [Reuses operator id: 74] +(10) ReusedExchange [Reuses operator id: 60] Output [1]: [d_date#9] (11) CometBroadcastHashJoin @@ -168,451 +154,221 @@ ReadSchema: struct Input [3]: [cr_item_sk#14, cr_return_quantity#15, cr_returned_date_sk#16] Condition : isnotnull(cr_item_sk#14) -(22) Scan parquet spark_catalog.default.item +(22) ReusedExchange [Reuses operator id: 5] Output [2]: [i_item_sk#18, i_item_id#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] -ReadSchema: struct -(23) CometFilter -Input [2]: [i_item_sk#18, i_item_id#19] -Condition : (isnotnull(i_item_sk#18) AND isnotnull(i_item_id#19)) - -(24) CometBroadcastExchange -Input [2]: [i_item_sk#18, i_item_id#19] -Arguments: [i_item_sk#18, i_item_id#19] - -(25) CometBroadcastHashJoin +(23) CometBroadcastHashJoin Left output [3]: [cr_item_sk#14, cr_return_quantity#15, cr_returned_date_sk#16] Right output [2]: [i_item_sk#18, i_item_id#19] Arguments: [cr_item_sk#14], [i_item_sk#18], Inner -(26) CometProject +(24) CometProject Input [5]: [cr_item_sk#14, cr_return_quantity#15, cr_returned_date_sk#16, i_item_sk#18, i_item_id#19] Arguments: [cr_return_quantity#15, cr_returned_date_sk#16, i_item_id#19], [cr_return_quantity#15, cr_returned_date_sk#16, i_item_id#19] -(27) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_date#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [d_date_sk#20, d_date#21] -Condition : isnotnull(d_date_sk#20) - -(29) ReusedExchange [Reuses operator id: 88] -Output [1]: [d_date#22] +(25) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_date_sk#20] -(30) CometBroadcastHashJoin -Left output [2]: [d_date_sk#20, d_date#21] -Right output [1]: [d_date#22] -Arguments: [d_date#21], [d_date#22], LeftSemi - -(31) CometProject -Input [2]: [d_date_sk#20, d_date#21] -Arguments: [d_date_sk#20], [d_date_sk#20] - -(32) CometBroadcastExchange -Input [1]: [d_date_sk#20] -Arguments: [d_date_sk#20] - -(33) CometBroadcastHashJoin +(26) CometBroadcastHashJoin Left output [3]: [cr_return_quantity#15, cr_returned_date_sk#16, i_item_id#19] Right output [1]: [d_date_sk#20] Arguments: [cr_returned_date_sk#16], [d_date_sk#20], Inner -(34) CometProject +(27) CometProject Input [4]: [cr_return_quantity#15, cr_returned_date_sk#16, i_item_id#19, d_date_sk#20] Arguments: [cr_return_quantity#15, i_item_id#19], [cr_return_quantity#15, i_item_id#19] -(35) CometHashAggregate +(28) CometHashAggregate Input [2]: [cr_return_quantity#15, i_item_id#19] Keys [1]: [i_item_id#19] Functions [1]: [partial_sum(cr_return_quantity#15)] -(36) ColumnarToRow [codegen id : 2] -Input [2]: [i_item_id#19, sum#23] +(29) ColumnarToRow [codegen id : 2] +Input [2]: [i_item_id#19, sum#21] -(37) Exchange -Input [2]: [i_item_id#19, sum#23] +(30) Exchange +Input [2]: [i_item_id#19, sum#21] Arguments: hashpartitioning(i_item_id#19, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(38) HashAggregate [codegen id : 3] -Input [2]: [i_item_id#19, sum#23] +(31) HashAggregate [codegen id : 3] +Input [2]: [i_item_id#19, sum#21] Keys [1]: [i_item_id#19] Functions [1]: [sum(cr_return_quantity#15)] -Aggregate Attributes [1]: [sum(cr_return_quantity#15)#24] -Results [2]: [i_item_id#19 AS item_id#25, sum(cr_return_quantity#15)#24 AS cr_item_qty#26] +Aggregate Attributes [1]: [sum(cr_return_quantity#15)#22] +Results [2]: [i_item_id#19 AS item_id#23, sum(cr_return_quantity#15)#22 AS cr_item_qty#24] -(39) BroadcastExchange -Input [2]: [item_id#25, cr_item_qty#26] +(32) BroadcastExchange +Input [2]: [item_id#23, cr_item_qty#24] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] -(40) BroadcastHashJoin [codegen id : 6] +(33) BroadcastHashJoin [codegen id : 6] Left keys [1]: [item_id#12] -Right keys [1]: [item_id#25] +Right keys [1]: [item_id#23] Join type: Inner Join condition: None -(41) Project [codegen id : 6] -Output [3]: [item_id#12, sr_item_qty#13, cr_item_qty#26] -Input [4]: [item_id#12, sr_item_qty#13, item_id#25, cr_item_qty#26] +(34) Project [codegen id : 6] +Output [3]: [item_id#12, sr_item_qty#13, cr_item_qty#24] +Input [4]: [item_id#12, sr_item_qty#13, item_id#23, cr_item_qty#24] -(42) Scan parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#27, wr_return_quantity#28, wr_returned_date_sk#29] +(35) Scan parquet spark_catalog.default.web_returns +Output [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#29), dynamicpruningexpression(wr_returned_date_sk#29 IN dynamicpruning#30)] +PartitionFilters: [isnotnull(wr_returned_date_sk#27), dynamicpruningexpression(wr_returned_date_sk#27 IN dynamicpruning#28)] PushedFilters: [IsNotNull(wr_item_sk)] ReadSchema: struct -(43) CometFilter -Input [3]: [wr_item_sk#27, wr_return_quantity#28, wr_returned_date_sk#29] -Condition : isnotnull(wr_item_sk#27) - -(44) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#31, i_item_id#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] -ReadSchema: struct - -(45) CometFilter -Input [2]: [i_item_sk#31, i_item_id#32] -Condition : (isnotnull(i_item_sk#31) AND isnotnull(i_item_id#32)) +(36) CometFilter +Input [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] +Condition : isnotnull(wr_item_sk#25) -(46) CometBroadcastExchange -Input [2]: [i_item_sk#31, i_item_id#32] -Arguments: [i_item_sk#31, i_item_id#32] +(37) ReusedExchange [Reuses operator id: 5] +Output [2]: [i_item_sk#29, i_item_id#30] -(47) CometBroadcastHashJoin -Left output [3]: [wr_item_sk#27, wr_return_quantity#28, wr_returned_date_sk#29] -Right output [2]: [i_item_sk#31, i_item_id#32] -Arguments: [wr_item_sk#27], [i_item_sk#31], Inner +(38) CometBroadcastHashJoin +Left output [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] +Right output [2]: [i_item_sk#29, i_item_id#30] +Arguments: [wr_item_sk#25], [i_item_sk#29], Inner -(48) CometProject -Input [5]: [wr_item_sk#27, wr_return_quantity#28, wr_returned_date_sk#29, i_item_sk#31, i_item_id#32] -Arguments: [wr_return_quantity#28, wr_returned_date_sk#29, i_item_id#32], [wr_return_quantity#28, wr_returned_date_sk#29, i_item_id#32] +(39) CometProject +Input [5]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27, i_item_sk#29, i_item_id#30] +Arguments: [wr_return_quantity#26, wr_returned_date_sk#27, i_item_id#30], [wr_return_quantity#26, wr_returned_date_sk#27, i_item_id#30] -(49) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_date#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct +(40) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_date_sk#31] -(50) CometFilter -Input [2]: [d_date_sk#33, d_date#34] -Condition : isnotnull(d_date_sk#33) +(41) CometBroadcastHashJoin +Left output [3]: [wr_return_quantity#26, wr_returned_date_sk#27, i_item_id#30] +Right output [1]: [d_date_sk#31] +Arguments: [wr_returned_date_sk#27], [d_date_sk#31], Inner -(51) ReusedExchange [Reuses operator id: 102] -Output [1]: [d_date#35] +(42) CometProject +Input [4]: [wr_return_quantity#26, wr_returned_date_sk#27, i_item_id#30, d_date_sk#31] +Arguments: [wr_return_quantity#26, i_item_id#30], [wr_return_quantity#26, i_item_id#30] -(52) CometBroadcastHashJoin -Left output [2]: [d_date_sk#33, d_date#34] -Right output [1]: [d_date#35] -Arguments: [d_date#34], [d_date#35], LeftSemi +(43) CometHashAggregate +Input [2]: [wr_return_quantity#26, i_item_id#30] +Keys [1]: [i_item_id#30] +Functions [1]: [partial_sum(wr_return_quantity#26)] -(53) CometProject -Input [2]: [d_date_sk#33, d_date#34] -Arguments: [d_date_sk#33], [d_date_sk#33] +(44) ColumnarToRow [codegen id : 4] +Input [2]: [i_item_id#30, sum#32] -(54) CometBroadcastExchange -Input [1]: [d_date_sk#33] -Arguments: [d_date_sk#33] +(45) Exchange +Input [2]: [i_item_id#30, sum#32] +Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(55) CometBroadcastHashJoin -Left output [3]: [wr_return_quantity#28, wr_returned_date_sk#29, i_item_id#32] -Right output [1]: [d_date_sk#33] -Arguments: [wr_returned_date_sk#29], [d_date_sk#33], Inner +(46) HashAggregate [codegen id : 5] +Input [2]: [i_item_id#30, sum#32] +Keys [1]: [i_item_id#30] +Functions [1]: [sum(wr_return_quantity#26)] +Aggregate Attributes [1]: [sum(wr_return_quantity#26)#33] +Results [2]: [i_item_id#30 AS item_id#34, sum(wr_return_quantity#26)#33 AS wr_item_qty#35] -(56) CometProject -Input [4]: [wr_return_quantity#28, wr_returned_date_sk#29, i_item_id#32, d_date_sk#33] -Arguments: [wr_return_quantity#28, i_item_id#32], [wr_return_quantity#28, i_item_id#32] - -(57) CometHashAggregate -Input [2]: [wr_return_quantity#28, i_item_id#32] -Keys [1]: [i_item_id#32] -Functions [1]: [partial_sum(wr_return_quantity#28)] - -(58) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_id#32, sum#36] - -(59) Exchange -Input [2]: [i_item_id#32, sum#36] -Arguments: hashpartitioning(i_item_id#32, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(60) HashAggregate [codegen id : 5] -Input [2]: [i_item_id#32, sum#36] -Keys [1]: [i_item_id#32] -Functions [1]: [sum(wr_return_quantity#28)] -Aggregate Attributes [1]: [sum(wr_return_quantity#28)#37] -Results [2]: [i_item_id#32 AS item_id#38, sum(wr_return_quantity#28)#37 AS wr_item_qty#39] - -(61) BroadcastExchange -Input [2]: [item_id#38, wr_item_qty#39] +(47) BroadcastExchange +Input [2]: [item_id#34, wr_item_qty#35] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(62) BroadcastHashJoin [codegen id : 6] +(48) BroadcastHashJoin [codegen id : 6] Left keys [1]: [item_id#12] -Right keys [1]: [item_id#38] +Right keys [1]: [item_id#34] Join type: Inner Join condition: None -(63) Project [codegen id : 6] -Output [8]: [item_id#12, sr_item_qty#13, (((cast(sr_item_qty#13 as double) / cast(((sr_item_qty#13 + cr_item_qty#26) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS sr_dev#40, cr_item_qty#26, (((cast(cr_item_qty#26 as double) / cast(((sr_item_qty#13 + cr_item_qty#26) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS cr_dev#41, wr_item_qty#39, (((cast(wr_item_qty#39 as double) / cast(((sr_item_qty#13 + cr_item_qty#26) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS wr_dev#42, (cast(((sr_item_qty#13 + cr_item_qty#26) + wr_item_qty#39) as decimal(20,0)) / 3.0) AS average#43] -Input [5]: [item_id#12, sr_item_qty#13, cr_item_qty#26, item_id#38, wr_item_qty#39] +(49) Project [codegen id : 6] +Output [8]: [item_id#12, sr_item_qty#13, (((cast(sr_item_qty#13 as double) / cast(((sr_item_qty#13 + cr_item_qty#24) + wr_item_qty#35) as double)) / 3.0) * 100.0) AS sr_dev#36, cr_item_qty#24, (((cast(cr_item_qty#24 as double) / cast(((sr_item_qty#13 + cr_item_qty#24) + wr_item_qty#35) as double)) / 3.0) * 100.0) AS cr_dev#37, wr_item_qty#35, (((cast(wr_item_qty#35 as double) / cast(((sr_item_qty#13 + cr_item_qty#24) + wr_item_qty#35) as double)) / 3.0) * 100.0) AS wr_dev#38, (cast(((sr_item_qty#13 + cr_item_qty#24) + wr_item_qty#35) as decimal(20,0)) / 3.0) AS average#39] +Input [5]: [item_id#12, sr_item_qty#13, cr_item_qty#24, item_id#34, wr_item_qty#35] -(64) TakeOrderedAndProject -Input [8]: [item_id#12, sr_item_qty#13, sr_dev#40, cr_item_qty#26, cr_dev#41, wr_item_qty#39, wr_dev#42, average#43] -Arguments: 100, [item_id#12 ASC NULLS FIRST, sr_item_qty#13 ASC NULLS FIRST], [item_id#12, sr_item_qty#13, sr_dev#40, cr_item_qty#26, cr_dev#41, wr_item_qty#39, wr_dev#42, average#43] +(50) TakeOrderedAndProject +Input [8]: [item_id#12, sr_item_qty#13, sr_dev#36, cr_item_qty#24, cr_dev#37, wr_item_qty#35, wr_dev#38, average#39] +Arguments: 100, [item_id#12 ASC NULLS FIRST, sr_item_qty#13 ASC NULLS FIRST], [item_id#12, sr_item_qty#13, sr_dev#36, cr_item_qty#24, cr_dev#37, wr_item_qty#35, wr_dev#38, average#39] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (78) -+- * ColumnarToRow (77) - +- CometProject (76) - +- CometBroadcastHashJoin (75) - :- CometFilter (66) - : +- CometScan parquet spark_catalog.default.date_dim (65) - +- CometBroadcastExchange (74) - +- CometProject (73) - +- CometBroadcastHashJoin (72) - :- CometScan parquet spark_catalog.default.date_dim (67) - +- CometBroadcastExchange (71) - +- CometProject (70) - +- CometFilter (69) - +- CometScan parquet spark_catalog.default.date_dim (68) - - -(65) Scan parquet spark_catalog.default.date_dim +BroadcastExchange (64) ++- * ColumnarToRow (63) + +- CometProject (62) + +- CometBroadcastHashJoin (61) + :- CometFilter (52) + : +- CometScan parquet spark_catalog.default.date_dim (51) + +- CometBroadcastExchange (60) + +- CometProject (59) + +- CometBroadcastHashJoin (58) + :- CometScan parquet spark_catalog.default.date_dim (53) + +- CometBroadcastExchange (57) + +- CometProject (56) + +- CometFilter (55) + +- CometScan parquet spark_catalog.default.date_dim (54) + + +(51) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#7, d_date#8] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct -(66) CometFilter +(52) CometFilter Input [2]: [d_date_sk#7, d_date#8] Condition : isnotnull(d_date_sk#7) -(67) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date#9, d_week_seq#44] +(53) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date#9, d_week_seq#40] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] ReadSchema: struct -(68) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date#45, d_week_seq#46] +(54) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date#41, d_week_seq#42] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] ReadSchema: struct -(69) CometFilter -Input [2]: [d_date#45, d_week_seq#46] -Condition : cast(d_date#45 as string) IN (2000-06-30,2000-09-27,2000-11-17) +(55) CometFilter +Input [2]: [d_date#41, d_week_seq#42] +Condition : cast(d_date#41 as string) IN (2000-06-30,2000-09-27,2000-11-17) -(70) CometProject -Input [2]: [d_date#45, d_week_seq#46] -Arguments: [d_week_seq#46], [d_week_seq#46] +(56) CometProject +Input [2]: [d_date#41, d_week_seq#42] +Arguments: [d_week_seq#42], [d_week_seq#42] -(71) CometBroadcastExchange -Input [1]: [d_week_seq#46] -Arguments: [d_week_seq#46] +(57) CometBroadcastExchange +Input [1]: [d_week_seq#42] +Arguments: [d_week_seq#42] -(72) CometBroadcastHashJoin -Left output [2]: [d_date#9, d_week_seq#44] -Right output [1]: [d_week_seq#46] -Arguments: [d_week_seq#44], [d_week_seq#46], LeftSemi +(58) CometBroadcastHashJoin +Left output [2]: [d_date#9, d_week_seq#40] +Right output [1]: [d_week_seq#42] +Arguments: [d_week_seq#40], [d_week_seq#42], LeftSemi -(73) CometProject -Input [2]: [d_date#9, d_week_seq#44] +(59) CometProject +Input [2]: [d_date#9, d_week_seq#40] Arguments: [d_date#9], [d_date#9] -(74) CometBroadcastExchange +(60) CometBroadcastExchange Input [1]: [d_date#9] Arguments: [d_date#9] -(75) CometBroadcastHashJoin +(61) CometBroadcastHashJoin Left output [2]: [d_date_sk#7, d_date#8] Right output [1]: [d_date#9] Arguments: [d_date#8], [d_date#9], LeftSemi -(76) CometProject +(62) CometProject Input [2]: [d_date_sk#7, d_date#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(77) ColumnarToRow [codegen id : 1] +(63) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(78) BroadcastExchange +(64) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 20 Hosting Expression = cr_returned_date_sk#16 IN dynamicpruning#17 -BroadcastExchange (92) -+- * ColumnarToRow (91) - +- CometProject (90) - +- CometBroadcastHashJoin (89) - :- CometFilter (80) - : +- CometScan parquet spark_catalog.default.date_dim (79) - +- CometBroadcastExchange (88) - +- CometProject (87) - +- CometBroadcastHashJoin (86) - :- CometScan parquet spark_catalog.default.date_dim (81) - +- CometBroadcastExchange (85) - +- CometProject (84) - +- CometFilter (83) - +- CometScan parquet spark_catalog.default.date_dim (82) - - -(79) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_date#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(80) CometFilter -Input [2]: [d_date_sk#20, d_date#21] -Condition : isnotnull(d_date_sk#20) - -(81) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date#22, d_week_seq#47] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -ReadSchema: struct - -(82) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date#48, d_week_seq#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -ReadSchema: struct - -(83) CometFilter -Input [2]: [d_date#48, d_week_seq#49] -Condition : cast(d_date#48 as string) IN (2000-06-30,2000-09-27,2000-11-17) - -(84) CometProject -Input [2]: [d_date#48, d_week_seq#49] -Arguments: [d_week_seq#49], [d_week_seq#49] - -(85) CometBroadcastExchange -Input [1]: [d_week_seq#49] -Arguments: [d_week_seq#49] - -(86) CometBroadcastHashJoin -Left output [2]: [d_date#22, d_week_seq#47] -Right output [1]: [d_week_seq#49] -Arguments: [d_week_seq#47], [d_week_seq#49], LeftSemi - -(87) CometProject -Input [2]: [d_date#22, d_week_seq#47] -Arguments: [d_date#22], [d_date#22] - -(88) CometBroadcastExchange -Input [1]: [d_date#22] -Arguments: [d_date#22] - -(89) CometBroadcastHashJoin -Left output [2]: [d_date_sk#20, d_date#21] -Right output [1]: [d_date#22] -Arguments: [d_date#21], [d_date#22], LeftSemi - -(90) CometProject -Input [2]: [d_date_sk#20, d_date#21] -Arguments: [d_date_sk#20], [d_date_sk#20] - -(91) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#20] - -(92) BroadcastExchange -Input [1]: [d_date_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:3 Hosting operator id = 42 Hosting Expression = wr_returned_date_sk#29 IN dynamicpruning#30 -BroadcastExchange (106) -+- * ColumnarToRow (105) - +- CometProject (104) - +- CometBroadcastHashJoin (103) - :- CometFilter (94) - : +- CometScan parquet spark_catalog.default.date_dim (93) - +- CometBroadcastExchange (102) - +- CometProject (101) - +- CometBroadcastHashJoin (100) - :- CometScan parquet spark_catalog.default.date_dim (95) - +- CometBroadcastExchange (99) - +- CometProject (98) - +- CometFilter (97) - +- CometScan parquet spark_catalog.default.date_dim (96) - - -(93) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_date#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(94) CometFilter -Input [2]: [d_date_sk#33, d_date#34] -Condition : isnotnull(d_date_sk#33) - -(95) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date#35, d_week_seq#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -ReadSchema: struct - -(96) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date#51, d_week_seq#52] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -ReadSchema: struct - -(97) CometFilter -Input [2]: [d_date#51, d_week_seq#52] -Condition : cast(d_date#51 as string) IN (2000-06-30,2000-09-27,2000-11-17) - -(98) CometProject -Input [2]: [d_date#51, d_week_seq#52] -Arguments: [d_week_seq#52], [d_week_seq#52] - -(99) CometBroadcastExchange -Input [1]: [d_week_seq#52] -Arguments: [d_week_seq#52] - -(100) CometBroadcastHashJoin -Left output [2]: [d_date#35, d_week_seq#50] -Right output [1]: [d_week_seq#52] -Arguments: [d_week_seq#50], [d_week_seq#52], LeftSemi - -(101) CometProject -Input [2]: [d_date#35, d_week_seq#50] -Arguments: [d_date#35], [d_date#35] - -(102) CometBroadcastExchange -Input [1]: [d_date#35] -Arguments: [d_date#35] - -(103) CometBroadcastHashJoin -Left output [2]: [d_date_sk#33, d_date#34] -Right output [1]: [d_date#35] -Arguments: [d_date#34], [d_date#35], LeftSemi - -(104) CometProject -Input [2]: [d_date_sk#33, d_date#34] -Arguments: [d_date_sk#33], [d_date_sk#33] - -(105) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#33] +Subquery:2 Hosting operator id = 20 Hosting Expression = cr_returned_date_sk#16 IN dynamicpruning#4 -(106) BroadcastExchange -Input [1]: [d_date_sk#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Subquery:3 Hosting operator id = 35 Hosting Expression = wr_returned_date_sk#27 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt index 7ea0d5009..ef7d35e21 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt @@ -59,38 +59,15 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty CometBroadcastHashJoin [cr_item_sk,i_item_sk] CometFilter [cr_item_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #9 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #10 - CometProject [d_date] - CometBroadcastHashJoin [d_week_seq,d_week_seq] - CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #11 - CometProject [d_week_seq] - CometFilter [d_date] - CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #12 - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #13 - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date] #10 + ReusedSubquery [d_date_sk] #1 + ReusedExchange [i_item_sk,i_item_id] #5 + ReusedExchange [d_date_sk] #6 InputAdapter - BroadcastExchange #14 + BroadcastExchange #9 WholeStageCodegen (5) HashAggregate [i_item_id,sum] [sum(wr_return_quantity),item_id,wr_item_qty,sum] InputAdapter - Exchange [i_item_id] #15 + Exchange [i_item_id] #10 WholeStageCodegen (4) ColumnarToRow InputAdapter @@ -101,29 +78,6 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty CometBroadcastHashJoin [wr_item_sk,i_item_sk] CometFilter [wr_item_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #16 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #17 - CometProject [d_date] - CometBroadcastHashJoin [d_week_seq,d_week_seq] - CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #18 - CometProject [d_week_seq] - CometFilter [d_date] - CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #19 - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #20 - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date] #17 + ReusedSubquery [d_date_sk] #1 + ReusedExchange [i_item_sk,i_item_id] #5 + ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt index 4e2c52b5a..0e3bee47d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -* HashAggregate (59) -+- Exchange (58) - +- * HashAggregate (57) - +- * Project (56) - +- * BroadcastHashJoin LeftAnti BuildRight (55) - :- * BroadcastHashJoin LeftAnti BuildRight (36) +* HashAggregate (49) ++- Exchange (48) + +- * HashAggregate (47) + +- * Project (46) + +- * BroadcastHashJoin LeftAnti BuildRight (45) + :- * BroadcastHashJoin LeftAnti BuildRight (31) : :- * HashAggregate (17) : : +- Exchange (16) : : +- * ColumnarToRow (15) @@ -22,42 +22,32 @@ : : +- CometBroadcastExchange (11) : : +- CometFilter (10) : : +- CometScan parquet spark_catalog.default.customer (9) - : +- BroadcastExchange (35) - : +- * HashAggregate (34) - : +- Exchange (33) - : +- * ColumnarToRow (32) - : +- CometHashAggregate (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometProject (25) - : : +- CometBroadcastHashJoin (24) + : +- BroadcastExchange (30) + : +- * HashAggregate (29) + : +- Exchange (28) + : +- * ColumnarToRow (27) + : +- CometHashAggregate (26) + : +- CometProject (25) + : +- CometBroadcastHashJoin (24) + : :- CometProject (22) + : : +- CometBroadcastHashJoin (21) : : :- CometFilter (19) : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) - : : +- CometBroadcastExchange (23) - : : +- CometProject (22) - : : +- CometFilter (21) - : : +- CometScan parquet spark_catalog.default.date_dim (20) - : +- CometBroadcastExchange (28) - : +- CometFilter (27) - : +- CometScan parquet spark_catalog.default.customer (26) - +- BroadcastExchange (54) - +- * HashAggregate (53) - +- Exchange (52) - +- * ColumnarToRow (51) - +- CometHashAggregate (50) - +- CometProject (49) - +- CometBroadcastHashJoin (48) - :- CometProject (44) - : +- CometBroadcastHashJoin (43) - : :- CometFilter (38) - : : +- CometScan parquet spark_catalog.default.web_sales (37) - : +- CometBroadcastExchange (42) - : +- CometProject (41) - : +- CometFilter (40) - : +- CometScan parquet spark_catalog.default.date_dim (39) - +- CometBroadcastExchange (47) - +- CometFilter (46) - +- CometScan parquet spark_catalog.default.customer (45) + : : +- ReusedExchange (20) + : +- ReusedExchange (23) + +- BroadcastExchange (44) + +- * HashAggregate (43) + +- Exchange (42) + +- * ColumnarToRow (41) + +- CometHashAggregate (40) + +- CometProject (39) + +- CometBroadcastHashJoin (38) + :- CometProject (36) + : +- CometBroadcastHashJoin (35) + : :- CometFilter (33) + : : +- CometScan parquet spark_catalog.default.web_sales (32) + : +- ReusedExchange (34) + +- ReusedExchange (37) (1) Scan parquet spark_catalog.default.store_sales @@ -155,236 +145,180 @@ ReadSchema: struct Input [2]: [cs_bill_customer_sk#10, cs_sold_date_sk#11] Condition : isnotnull(cs_bill_customer_sk#10) -(20) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_date#14, d_month_seq#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [d_date_sk#13, d_date#14, d_month_seq#15] -Condition : (((isnotnull(d_month_seq#15) AND (d_month_seq#15 >= 1200)) AND (d_month_seq#15 <= 1211)) AND isnotnull(d_date_sk#13)) +(20) ReusedExchange [Reuses operator id: 6] +Output [2]: [d_date_sk#13, d_date#14] -(22) CometProject -Input [3]: [d_date_sk#13, d_date#14, d_month_seq#15] -Arguments: [d_date_sk#13, d_date#14], [d_date_sk#13, d_date#14] - -(23) CometBroadcastExchange -Input [2]: [d_date_sk#13, d_date#14] -Arguments: [d_date_sk#13, d_date#14] - -(24) CometBroadcastHashJoin +(21) CometBroadcastHashJoin Left output [2]: [cs_bill_customer_sk#10, cs_sold_date_sk#11] Right output [2]: [d_date_sk#13, d_date#14] Arguments: [cs_sold_date_sk#11], [d_date_sk#13], Inner -(25) CometProject +(22) CometProject Input [4]: [cs_bill_customer_sk#10, cs_sold_date_sk#11, d_date_sk#13, d_date#14] Arguments: [cs_bill_customer_sk#10, d_date#14], [cs_bill_customer_sk#10, d_date#14] -(26) Scan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#16, c_first_name#17, c_last_name#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(27) CometFilter -Input [3]: [c_customer_sk#16, c_first_name#17, c_last_name#18] -Condition : isnotnull(c_customer_sk#16) - -(28) CometBroadcastExchange -Input [3]: [c_customer_sk#16, c_first_name#17, c_last_name#18] -Arguments: [c_customer_sk#16, c_first_name#17, c_last_name#18] +(23) ReusedExchange [Reuses operator id: 11] +Output [3]: [c_customer_sk#15, c_first_name#16, c_last_name#17] -(29) CometBroadcastHashJoin +(24) CometBroadcastHashJoin Left output [2]: [cs_bill_customer_sk#10, d_date#14] -Right output [3]: [c_customer_sk#16, c_first_name#17, c_last_name#18] -Arguments: [cs_bill_customer_sk#10], [c_customer_sk#16], Inner +Right output [3]: [c_customer_sk#15, c_first_name#16, c_last_name#17] +Arguments: [cs_bill_customer_sk#10], [c_customer_sk#15], Inner -(30) CometProject -Input [5]: [cs_bill_customer_sk#10, d_date#14, c_customer_sk#16, c_first_name#17, c_last_name#18] -Arguments: [c_last_name#18, c_first_name#17, d_date#14], [c_last_name#18, c_first_name#17, d_date#14] +(25) CometProject +Input [5]: [cs_bill_customer_sk#10, d_date#14, c_customer_sk#15, c_first_name#16, c_last_name#17] +Arguments: [c_last_name#17, c_first_name#16, d_date#14], [c_last_name#17, c_first_name#16, d_date#14] -(31) CometHashAggregate -Input [3]: [c_last_name#18, c_first_name#17, d_date#14] -Keys [3]: [c_last_name#18, c_first_name#17, d_date#14] +(26) CometHashAggregate +Input [3]: [c_last_name#17, c_first_name#16, d_date#14] +Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] Functions: [] -(32) ColumnarToRow [codegen id : 2] -Input [3]: [c_last_name#18, c_first_name#17, d_date#14] +(27) ColumnarToRow [codegen id : 2] +Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -(33) Exchange -Input [3]: [c_last_name#18, c_first_name#17, d_date#14] -Arguments: hashpartitioning(c_last_name#18, c_first_name#17, d_date#14, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(28) Exchange +Input [3]: [c_last_name#17, c_first_name#16, d_date#14] +Arguments: hashpartitioning(c_last_name#17, c_first_name#16, d_date#14, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(34) HashAggregate [codegen id : 3] -Input [3]: [c_last_name#18, c_first_name#17, d_date#14] -Keys [3]: [c_last_name#18, c_first_name#17, d_date#14] +(29) HashAggregate [codegen id : 3] +Input [3]: [c_last_name#17, c_first_name#16, d_date#14] +Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#18, c_first_name#17, d_date#14] +Results [3]: [c_last_name#17, c_first_name#16, d_date#14] -(35) BroadcastExchange -Input [3]: [c_last_name#18, c_first_name#17, d_date#14] +(30) BroadcastExchange +Input [3]: [c_last_name#17, c_first_name#16, d_date#14] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=3] -(36) BroadcastHashJoin [codegen id : 6] +(31) BroadcastHashJoin [codegen id : 6] Left keys [6]: [coalesce(c_last_name#9, ), isnull(c_last_name#9), coalesce(c_first_name#8, ), isnull(c_first_name#8), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#18, ), isnull(c_last_name#18), coalesce(c_first_name#17, ), isnull(c_first_name#17), coalesce(d_date#14, 1970-01-01), isnull(d_date#14)] +Right keys [6]: [coalesce(c_last_name#17, ), isnull(c_last_name#17), coalesce(c_first_name#16, ), isnull(c_first_name#16), coalesce(d_date#14, 1970-01-01), isnull(d_date#14)] Join type: LeftAnti Join condition: None -(37) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#19, ws_sold_date_sk#20] +(32) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#20), dynamicpruningexpression(ws_sold_date_sk#20 IN dynamicpruning#21)] +PartitionFilters: [isnotnull(ws_sold_date_sk#19), dynamicpruningexpression(ws_sold_date_sk#19 IN dynamicpruning#20)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(38) CometFilter -Input [2]: [ws_bill_customer_sk#19, ws_sold_date_sk#20] -Condition : isnotnull(ws_bill_customer_sk#19) +(33) CometFilter +Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] +Condition : isnotnull(ws_bill_customer_sk#18) -(39) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#22, d_date#23, d_month_seq#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct +(34) ReusedExchange [Reuses operator id: 6] +Output [2]: [d_date_sk#21, d_date#22] -(40) CometFilter -Input [3]: [d_date_sk#22, d_date#23, d_month_seq#24] -Condition : (((isnotnull(d_month_seq#24) AND (d_month_seq#24 >= 1200)) AND (d_month_seq#24 <= 1211)) AND isnotnull(d_date_sk#22)) +(35) CometBroadcastHashJoin +Left output [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] +Right output [2]: [d_date_sk#21, d_date#22] +Arguments: [ws_sold_date_sk#19], [d_date_sk#21], Inner -(41) CometProject -Input [3]: [d_date_sk#22, d_date#23, d_month_seq#24] -Arguments: [d_date_sk#22, d_date#23], [d_date_sk#22, d_date#23] +(36) CometProject +Input [4]: [ws_bill_customer_sk#18, ws_sold_date_sk#19, d_date_sk#21, d_date#22] +Arguments: [ws_bill_customer_sk#18, d_date#22], [ws_bill_customer_sk#18, d_date#22] -(42) CometBroadcastExchange -Input [2]: [d_date_sk#22, d_date#23] -Arguments: [d_date_sk#22, d_date#23] +(37) ReusedExchange [Reuses operator id: 11] +Output [3]: [c_customer_sk#23, c_first_name#24, c_last_name#25] -(43) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#19, ws_sold_date_sk#20] -Right output [2]: [d_date_sk#22, d_date#23] -Arguments: [ws_sold_date_sk#20], [d_date_sk#22], Inner +(38) CometBroadcastHashJoin +Left output [2]: [ws_bill_customer_sk#18, d_date#22] +Right output [3]: [c_customer_sk#23, c_first_name#24, c_last_name#25] +Arguments: [ws_bill_customer_sk#18], [c_customer_sk#23], Inner -(44) CometProject -Input [4]: [ws_bill_customer_sk#19, ws_sold_date_sk#20, d_date_sk#22, d_date#23] -Arguments: [ws_bill_customer_sk#19, d_date#23], [ws_bill_customer_sk#19, d_date#23] +(39) CometProject +Input [5]: [ws_bill_customer_sk#18, d_date#22, c_customer_sk#23, c_first_name#24, c_last_name#25] +Arguments: [c_last_name#25, c_first_name#24, d_date#22], [c_last_name#25, c_first_name#24, d_date#22] -(45) Scan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(46) CometFilter -Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Condition : isnotnull(c_customer_sk#25) - -(47) CometBroadcastExchange -Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: [c_customer_sk#25, c_first_name#26, c_last_name#27] - -(48) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#19, d_date#23] -Right output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: [ws_bill_customer_sk#19], [c_customer_sk#25], Inner - -(49) CometProject -Input [5]: [ws_bill_customer_sk#19, d_date#23, c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: [c_last_name#27, c_first_name#26, d_date#23], [c_last_name#27, c_first_name#26, d_date#23] - -(50) CometHashAggregate -Input [3]: [c_last_name#27, c_first_name#26, d_date#23] -Keys [3]: [c_last_name#27, c_first_name#26, d_date#23] +(40) CometHashAggregate +Input [3]: [c_last_name#25, c_first_name#24, d_date#22] +Keys [3]: [c_last_name#25, c_first_name#24, d_date#22] Functions: [] -(51) ColumnarToRow [codegen id : 4] -Input [3]: [c_last_name#27, c_first_name#26, d_date#23] +(41) ColumnarToRow [codegen id : 4] +Input [3]: [c_last_name#25, c_first_name#24, d_date#22] -(52) Exchange -Input [3]: [c_last_name#27, c_first_name#26, d_date#23] -Arguments: hashpartitioning(c_last_name#27, c_first_name#26, d_date#23, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(42) Exchange +Input [3]: [c_last_name#25, c_first_name#24, d_date#22] +Arguments: hashpartitioning(c_last_name#25, c_first_name#24, d_date#22, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(53) HashAggregate [codegen id : 5] -Input [3]: [c_last_name#27, c_first_name#26, d_date#23] -Keys [3]: [c_last_name#27, c_first_name#26, d_date#23] +(43) HashAggregate [codegen id : 5] +Input [3]: [c_last_name#25, c_first_name#24, d_date#22] +Keys [3]: [c_last_name#25, c_first_name#24, d_date#22] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#27, c_first_name#26, d_date#23] +Results [3]: [c_last_name#25, c_first_name#24, d_date#22] -(54) BroadcastExchange -Input [3]: [c_last_name#27, c_first_name#26, d_date#23] +(44) BroadcastExchange +Input [3]: [c_last_name#25, c_first_name#24, d_date#22] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=5] -(55) BroadcastHashJoin [codegen id : 6] +(45) BroadcastHashJoin [codegen id : 6] Left keys [6]: [coalesce(c_last_name#9, ), isnull(c_last_name#9), coalesce(c_first_name#8, ), isnull(c_first_name#8), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#27, ), isnull(c_last_name#27), coalesce(c_first_name#26, ), isnull(c_first_name#26), coalesce(d_date#23, 1970-01-01), isnull(d_date#23)] +Right keys [6]: [coalesce(c_last_name#25, ), isnull(c_last_name#25), coalesce(c_first_name#24, ), isnull(c_first_name#24), coalesce(d_date#22, 1970-01-01), isnull(d_date#22)] Join type: LeftAnti Join condition: None -(56) Project [codegen id : 6] +(46) Project [codegen id : 6] Output: [] Input [3]: [c_last_name#9, c_first_name#8, d_date#5] -(57) HashAggregate [codegen id : 6] +(47) HashAggregate [codegen id : 6] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#28] -Results [1]: [count#29] +Aggregate Attributes [1]: [count#26] +Results [1]: [count#27] -(58) Exchange -Input [1]: [count#29] +(48) Exchange +Input [1]: [count#27] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(59) HashAggregate [codegen id : 7] -Input [1]: [count#29] +(49) HashAggregate [codegen id : 7] +Input [1]: [count#27] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#30] -Results [1]: [count(1)#30 AS count(1)#31] +Aggregate Attributes [1]: [count(1)#28] +Results [1]: [count(1)#28 AS count(1)#29] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 -BroadcastExchange (64) -+- * ColumnarToRow (63) - +- CometProject (62) - +- CometFilter (61) - +- CometScan parquet spark_catalog.default.date_dim (60) +BroadcastExchange (54) ++- * ColumnarToRow (53) + +- CometProject (52) + +- CometFilter (51) + +- CometScan parquet spark_catalog.default.date_dim (50) -(60) Scan parquet spark_catalog.default.date_dim +(50) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#4, d_date#5, d_month_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(61) CometFilter +(51) CometFilter Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#4)) -(62) CometProject +(52) CometProject Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(63) ColumnarToRow [codegen id : 1] +(53) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#4, d_date#5] -(64) BroadcastExchange +(54) BroadcastExchange Input [2]: [d_date_sk#4, d_date#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] Subquery:2 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#11 IN dynamicpruning#3 -Subquery:3 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#20 IN dynamicpruning#3 +Subquery:3 Hosting operator id = 32 Hosting Expression = ws_sold_date_sk#19 IN dynamicpruning#3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt index 2dae3c911..cfac83844 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt @@ -52,19 +52,14 @@ WholeStageCodegen (7) CometFilter [cs_bill_customer_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #8 - CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #9 - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] + ReusedExchange [d_date_sk,d_date] #4 + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 InputAdapter - BroadcastExchange #10 + BroadcastExchange #8 WholeStageCodegen (5) HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_last_name,c_first_name,d_date] #11 + Exchange [c_last_name,c_first_name,d_date] #9 WholeStageCodegen (4) ColumnarToRow InputAdapter @@ -76,10 +71,5 @@ WholeStageCodegen (7) CometFilter [ws_bill_customer_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #12 - CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #13 - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] + ReusedExchange [d_date_sk,d_date] #4 + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt index 4a47daa7b..00c882540 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt @@ -1,11 +1,11 @@ == Physical Plan == -* BroadcastNestedLoopJoin Inner BuildRight (214) -:- * BroadcastNestedLoopJoin Inner BuildRight (187) -: :- * BroadcastNestedLoopJoin Inner BuildRight (160) -: : :- * BroadcastNestedLoopJoin Inner BuildRight (133) -: : : :- * BroadcastNestedLoopJoin Inner BuildRight (106) -: : : : :- * BroadcastNestedLoopJoin Inner BuildRight (79) -: : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (52) +* BroadcastNestedLoopJoin Inner BuildRight (172) +:- * BroadcastNestedLoopJoin Inner BuildRight (151) +: :- * BroadcastNestedLoopJoin Inner BuildRight (130) +: : :- * BroadcastNestedLoopJoin Inner BuildRight (109) +: : : :- * BroadcastNestedLoopJoin Inner BuildRight (88) +: : : : :- * BroadcastNestedLoopJoin Inner BuildRight (67) +: : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (46) : : : : : : :- * HashAggregate (25) : : : : : : : +- Exchange (24) : : : : : : : +- * ColumnarToRow (23) @@ -31,188 +31,146 @@ : : : : : : : +- CometProject (18) : : : : : : : +- CometFilter (17) : : : : : : : +- CometScan parquet spark_catalog.default.store (16) -: : : : : : +- BroadcastExchange (51) -: : : : : : +- * HashAggregate (50) -: : : : : : +- Exchange (49) -: : : : : : +- * ColumnarToRow (48) -: : : : : : +- CometHashAggregate (47) -: : : : : : +- CometProject (46) -: : : : : : +- CometBroadcastHashJoin (45) -: : : : : : :- CometProject (40) -: : : : : : : +- CometBroadcastHashJoin (39) -: : : : : : : :- CometProject (34) -: : : : : : : : +- CometBroadcastHashJoin (33) +: : : : : : +- BroadcastExchange (45) +: : : : : : +- * HashAggregate (44) +: : : : : : +- Exchange (43) +: : : : : : +- * ColumnarToRow (42) +: : : : : : +- CometHashAggregate (41) +: : : : : : +- CometProject (40) +: : : : : : +- CometBroadcastHashJoin (39) +: : : : : : :- CometProject (37) +: : : : : : : +- CometBroadcastHashJoin (36) +: : : : : : : :- CometProject (31) +: : : : : : : : +- CometBroadcastHashJoin (30) : : : : : : : : :- CometProject (28) : : : : : : : : : +- CometFilter (27) : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (26) -: : : : : : : : +- CometBroadcastExchange (32) -: : : : : : : : +- CometProject (31) -: : : : : : : : +- CometFilter (30) -: : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (29) -: : : : : : : +- CometBroadcastExchange (38) -: : : : : : : +- CometProject (37) -: : : : : : : +- CometFilter (36) -: : : : : : : +- CometScan parquet spark_catalog.default.time_dim (35) -: : : : : : +- CometBroadcastExchange (44) -: : : : : : +- CometProject (43) -: : : : : : +- CometFilter (42) -: : : : : : +- CometScan parquet spark_catalog.default.store (41) -: : : : : +- BroadcastExchange (78) -: : : : : +- * HashAggregate (77) -: : : : : +- Exchange (76) -: : : : : +- * ColumnarToRow (75) -: : : : : +- CometHashAggregate (74) -: : : : : +- CometProject (73) -: : : : : +- CometBroadcastHashJoin (72) -: : : : : :- CometProject (67) -: : : : : : +- CometBroadcastHashJoin (66) -: : : : : : :- CometProject (61) -: : : : : : : +- CometBroadcastHashJoin (60) -: : : : : : : :- CometProject (55) -: : : : : : : : +- CometFilter (54) -: : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (53) -: : : : : : : +- CometBroadcastExchange (59) -: : : : : : : +- CometProject (58) -: : : : : : : +- CometFilter (57) -: : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (56) -: : : : : : +- CometBroadcastExchange (65) -: : : : : : +- CometProject (64) -: : : : : : +- CometFilter (63) -: : : : : : +- CometScan parquet spark_catalog.default.time_dim (62) -: : : : : +- CometBroadcastExchange (71) -: : : : : +- CometProject (70) -: : : : : +- CometFilter (69) -: : : : : +- CometScan parquet spark_catalog.default.store (68) -: : : : +- BroadcastExchange (105) -: : : : +- * HashAggregate (104) -: : : : +- Exchange (103) -: : : : +- * ColumnarToRow (102) -: : : : +- CometHashAggregate (101) -: : : : +- CometProject (100) -: : : : +- CometBroadcastHashJoin (99) -: : : : :- CometProject (94) -: : : : : +- CometBroadcastHashJoin (93) -: : : : : :- CometProject (88) -: : : : : : +- CometBroadcastHashJoin (87) -: : : : : : :- CometProject (82) -: : : : : : : +- CometFilter (81) -: : : : : : : +- CometScan parquet spark_catalog.default.store_sales (80) -: : : : : : +- CometBroadcastExchange (86) -: : : : : : +- CometProject (85) -: : : : : : +- CometFilter (84) -: : : : : : +- CometScan parquet spark_catalog.default.household_demographics (83) -: : : : : +- CometBroadcastExchange (92) -: : : : : +- CometProject (91) -: : : : : +- CometFilter (90) -: : : : : +- CometScan parquet spark_catalog.default.time_dim (89) -: : : : +- CometBroadcastExchange (98) -: : : : +- CometProject (97) -: : : : +- CometFilter (96) -: : : : +- CometScan parquet spark_catalog.default.store (95) -: : : +- BroadcastExchange (132) -: : : +- * HashAggregate (131) -: : : +- Exchange (130) -: : : +- * ColumnarToRow (129) -: : : +- CometHashAggregate (128) -: : : +- CometProject (127) -: : : +- CometBroadcastHashJoin (126) -: : : :- CometProject (121) -: : : : +- CometBroadcastHashJoin (120) -: : : : :- CometProject (115) -: : : : : +- CometBroadcastHashJoin (114) -: : : : : :- CometProject (109) -: : : : : : +- CometFilter (108) -: : : : : : +- CometScan parquet spark_catalog.default.store_sales (107) -: : : : : +- CometBroadcastExchange (113) -: : : : : +- CometProject (112) -: : : : : +- CometFilter (111) -: : : : : +- CometScan parquet spark_catalog.default.household_demographics (110) -: : : : +- CometBroadcastExchange (119) -: : : : +- CometProject (118) -: : : : +- CometFilter (117) -: : : : +- CometScan parquet spark_catalog.default.time_dim (116) -: : : +- CometBroadcastExchange (125) -: : : +- CometProject (124) -: : : +- CometFilter (123) -: : : +- CometScan parquet spark_catalog.default.store (122) -: : +- BroadcastExchange (159) -: : +- * HashAggregate (158) -: : +- Exchange (157) -: : +- * ColumnarToRow (156) -: : +- CometHashAggregate (155) -: : +- CometProject (154) -: : +- CometBroadcastHashJoin (153) -: : :- CometProject (148) -: : : +- CometBroadcastHashJoin (147) -: : : :- CometProject (142) -: : : : +- CometBroadcastHashJoin (141) -: : : : :- CometProject (136) -: : : : : +- CometFilter (135) -: : : : : +- CometScan parquet spark_catalog.default.store_sales (134) -: : : : +- CometBroadcastExchange (140) -: : : : +- CometProject (139) -: : : : +- CometFilter (138) -: : : : +- CometScan parquet spark_catalog.default.household_demographics (137) -: : : +- CometBroadcastExchange (146) -: : : +- CometProject (145) -: : : +- CometFilter (144) -: : : +- CometScan parquet spark_catalog.default.time_dim (143) -: : +- CometBroadcastExchange (152) -: : +- CometProject (151) -: : +- CometFilter (150) -: : +- CometScan parquet spark_catalog.default.store (149) -: +- BroadcastExchange (186) -: +- * HashAggregate (185) -: +- Exchange (184) -: +- * ColumnarToRow (183) -: +- CometHashAggregate (182) -: +- CometProject (181) -: +- CometBroadcastHashJoin (180) -: :- CometProject (175) -: : +- CometBroadcastHashJoin (174) -: : :- CometProject (169) -: : : +- CometBroadcastHashJoin (168) -: : : :- CometProject (163) -: : : : +- CometFilter (162) -: : : : +- CometScan parquet spark_catalog.default.store_sales (161) -: : : +- CometBroadcastExchange (167) -: : : +- CometProject (166) -: : : +- CometFilter (165) -: : : +- CometScan parquet spark_catalog.default.household_demographics (164) -: : +- CometBroadcastExchange (173) -: : +- CometProject (172) -: : +- CometFilter (171) -: : +- CometScan parquet spark_catalog.default.time_dim (170) -: +- CometBroadcastExchange (179) -: +- CometProject (178) -: +- CometFilter (177) -: +- CometScan parquet spark_catalog.default.store (176) -+- BroadcastExchange (213) - +- * HashAggregate (212) - +- Exchange (211) - +- * ColumnarToRow (210) - +- CometHashAggregate (209) - +- CometProject (208) - +- CometBroadcastHashJoin (207) - :- CometProject (202) - : +- CometBroadcastHashJoin (201) - : :- CometProject (196) - : : +- CometBroadcastHashJoin (195) - : : :- CometProject (190) - : : : +- CometFilter (189) - : : : +- CometScan parquet spark_catalog.default.store_sales (188) - : : +- CometBroadcastExchange (194) - : : +- CometProject (193) - : : +- CometFilter (192) - : : +- CometScan parquet spark_catalog.default.household_demographics (191) - : +- CometBroadcastExchange (200) - : +- CometProject (199) - : +- CometFilter (198) - : +- CometScan parquet spark_catalog.default.time_dim (197) - +- CometBroadcastExchange (206) - +- CometProject (205) - +- CometFilter (204) - +- CometScan parquet spark_catalog.default.store (203) +: : : : : : : : +- ReusedExchange (29) +: : : : : : : +- CometBroadcastExchange (35) +: : : : : : : +- CometProject (34) +: : : : : : : +- CometFilter (33) +: : : : : : : +- CometScan parquet spark_catalog.default.time_dim (32) +: : : : : : +- ReusedExchange (38) +: : : : : +- BroadcastExchange (66) +: : : : : +- * HashAggregate (65) +: : : : : +- Exchange (64) +: : : : : +- * ColumnarToRow (63) +: : : : : +- CometHashAggregate (62) +: : : : : +- CometProject (61) +: : : : : +- CometBroadcastHashJoin (60) +: : : : : :- CometProject (58) +: : : : : : +- CometBroadcastHashJoin (57) +: : : : : : :- CometProject (52) +: : : : : : : +- CometBroadcastHashJoin (51) +: : : : : : : :- CometProject (49) +: : : : : : : : +- CometFilter (48) +: : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (47) +: : : : : : : +- ReusedExchange (50) +: : : : : : +- CometBroadcastExchange (56) +: : : : : : +- CometProject (55) +: : : : : : +- CometFilter (54) +: : : : : : +- CometScan parquet spark_catalog.default.time_dim (53) +: : : : : +- ReusedExchange (59) +: : : : +- BroadcastExchange (87) +: : : : +- * HashAggregate (86) +: : : : +- Exchange (85) +: : : : +- * ColumnarToRow (84) +: : : : +- CometHashAggregate (83) +: : : : +- CometProject (82) +: : : : +- CometBroadcastHashJoin (81) +: : : : :- CometProject (79) +: : : : : +- CometBroadcastHashJoin (78) +: : : : : :- CometProject (73) +: : : : : : +- CometBroadcastHashJoin (72) +: : : : : : :- CometProject (70) +: : : : : : : +- CometFilter (69) +: : : : : : : +- CometScan parquet spark_catalog.default.store_sales (68) +: : : : : : +- ReusedExchange (71) +: : : : : +- CometBroadcastExchange (77) +: : : : : +- CometProject (76) +: : : : : +- CometFilter (75) +: : : : : +- CometScan parquet spark_catalog.default.time_dim (74) +: : : : +- ReusedExchange (80) +: : : +- BroadcastExchange (108) +: : : +- * HashAggregate (107) +: : : +- Exchange (106) +: : : +- * ColumnarToRow (105) +: : : +- CometHashAggregate (104) +: : : +- CometProject (103) +: : : +- CometBroadcastHashJoin (102) +: : : :- CometProject (100) +: : : : +- CometBroadcastHashJoin (99) +: : : : :- CometProject (94) +: : : : : +- CometBroadcastHashJoin (93) +: : : : : :- CometProject (91) +: : : : : : +- CometFilter (90) +: : : : : : +- CometScan parquet spark_catalog.default.store_sales (89) +: : : : : +- ReusedExchange (92) +: : : : +- CometBroadcastExchange (98) +: : : : +- CometProject (97) +: : : : +- CometFilter (96) +: : : : +- CometScan parquet spark_catalog.default.time_dim (95) +: : : +- ReusedExchange (101) +: : +- BroadcastExchange (129) +: : +- * HashAggregate (128) +: : +- Exchange (127) +: : +- * ColumnarToRow (126) +: : +- CometHashAggregate (125) +: : +- CometProject (124) +: : +- CometBroadcastHashJoin (123) +: : :- CometProject (121) +: : : +- CometBroadcastHashJoin (120) +: : : :- CometProject (115) +: : : : +- CometBroadcastHashJoin (114) +: : : : :- CometProject (112) +: : : : : +- CometFilter (111) +: : : : : +- CometScan parquet spark_catalog.default.store_sales (110) +: : : : +- ReusedExchange (113) +: : : +- CometBroadcastExchange (119) +: : : +- CometProject (118) +: : : +- CometFilter (117) +: : : +- CometScan parquet spark_catalog.default.time_dim (116) +: : +- ReusedExchange (122) +: +- BroadcastExchange (150) +: +- * HashAggregate (149) +: +- Exchange (148) +: +- * ColumnarToRow (147) +: +- CometHashAggregate (146) +: +- CometProject (145) +: +- CometBroadcastHashJoin (144) +: :- CometProject (142) +: : +- CometBroadcastHashJoin (141) +: : :- CometProject (136) +: : : +- CometBroadcastHashJoin (135) +: : : :- CometProject (133) +: : : : +- CometFilter (132) +: : : : +- CometScan parquet spark_catalog.default.store_sales (131) +: : : +- ReusedExchange (134) +: : +- CometBroadcastExchange (140) +: : +- CometProject (139) +: : +- CometFilter (138) +: : +- CometScan parquet spark_catalog.default.time_dim (137) +: +- ReusedExchange (143) ++- BroadcastExchange (171) + +- * HashAggregate (170) + +- Exchange (169) + +- * ColumnarToRow (168) + +- CometHashAggregate (167) + +- CometProject (166) + +- CometBroadcastHashJoin (165) + :- CometProject (163) + : +- CometBroadcastHashJoin (162) + : :- CometProject (157) + : : +- CometBroadcastHashJoin (156) + : : :- CometProject (154) + : : : +- CometFilter (153) + : : : +- CometScan parquet spark_catalog.default.store_sales (152) + : : +- ReusedExchange (155) + : +- CometBroadcastExchange (161) + : +- CometProject (160) + : +- CometFilter (159) + : +- CometScan parquet spark_catalog.default.time_dim (158) + +- ReusedExchange (164) (1) Scan parquet spark_catalog.default.store_sales @@ -347,863 +305,639 @@ Condition : ((isnotnull(ss_hdemo_sk#17) AND isnotnull(ss_sold_time_sk#16)) AND i Input [4]: [ss_sold_time_sk#16, ss_hdemo_sk#17, ss_store_sk#18, ss_sold_date_sk#19] Arguments: [ss_sold_time_sk#16, ss_hdemo_sk#17, ss_store_sk#18], [ss_sold_time_sk#16, ss_hdemo_sk#17, ss_store_sk#18] -(29) Scan parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#20, hd_dep_count#21, hd_vehicle_count#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(Or(And(EqualTo(hd_dep_count,4),LessThanOrEqual(hd_vehicle_count,6)),And(EqualTo(hd_dep_count,2),LessThanOrEqual(hd_vehicle_count,4))),And(EqualTo(hd_dep_count,0),LessThanOrEqual(hd_vehicle_count,2))), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(30) CometFilter -Input [3]: [hd_demo_sk#20, hd_dep_count#21, hd_vehicle_count#22] -Condition : (((((hd_dep_count#21 = 4) AND (hd_vehicle_count#22 <= 6)) OR ((hd_dep_count#21 = 2) AND (hd_vehicle_count#22 <= 4))) OR ((hd_dep_count#21 = 0) AND (hd_vehicle_count#22 <= 2))) AND isnotnull(hd_demo_sk#20)) - -(31) CometProject -Input [3]: [hd_demo_sk#20, hd_dep_count#21, hd_vehicle_count#22] -Arguments: [hd_demo_sk#20], [hd_demo_sk#20] +(29) ReusedExchange [Reuses operator id: 7] +Output [1]: [hd_demo_sk#20] -(32) CometBroadcastExchange -Input [1]: [hd_demo_sk#20] -Arguments: [hd_demo_sk#20] - -(33) CometBroadcastHashJoin +(30) CometBroadcastHashJoin Left output [3]: [ss_sold_time_sk#16, ss_hdemo_sk#17, ss_store_sk#18] Right output [1]: [hd_demo_sk#20] Arguments: [ss_hdemo_sk#17], [hd_demo_sk#20], Inner -(34) CometProject +(31) CometProject Input [4]: [ss_sold_time_sk#16, ss_hdemo_sk#17, ss_store_sk#18, hd_demo_sk#20] Arguments: [ss_sold_time_sk#16, ss_store_sk#18], [ss_sold_time_sk#16, ss_store_sk#18] -(35) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#23, t_hour#24, t_minute#25] +(32) Scan parquet spark_catalog.default.time_dim +Output [3]: [t_time_sk#21, t_hour#22, t_minute#23] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(36) CometFilter -Input [3]: [t_time_sk#23, t_hour#24, t_minute#25] -Condition : ((((isnotnull(t_hour#24) AND isnotnull(t_minute#25)) AND (t_hour#24 = 9)) AND (t_minute#25 < 30)) AND isnotnull(t_time_sk#23)) +(33) CometFilter +Input [3]: [t_time_sk#21, t_hour#22, t_minute#23] +Condition : ((((isnotnull(t_hour#22) AND isnotnull(t_minute#23)) AND (t_hour#22 = 9)) AND (t_minute#23 < 30)) AND isnotnull(t_time_sk#21)) -(37) CometProject -Input [3]: [t_time_sk#23, t_hour#24, t_minute#25] -Arguments: [t_time_sk#23], [t_time_sk#23] +(34) CometProject +Input [3]: [t_time_sk#21, t_hour#22, t_minute#23] +Arguments: [t_time_sk#21], [t_time_sk#21] -(38) CometBroadcastExchange -Input [1]: [t_time_sk#23] -Arguments: [t_time_sk#23] +(35) CometBroadcastExchange +Input [1]: [t_time_sk#21] +Arguments: [t_time_sk#21] -(39) CometBroadcastHashJoin +(36) CometBroadcastHashJoin Left output [2]: [ss_sold_time_sk#16, ss_store_sk#18] -Right output [1]: [t_time_sk#23] -Arguments: [ss_sold_time_sk#16], [t_time_sk#23], Inner +Right output [1]: [t_time_sk#21] +Arguments: [ss_sold_time_sk#16], [t_time_sk#21], Inner -(40) CometProject -Input [3]: [ss_sold_time_sk#16, ss_store_sk#18, t_time_sk#23] +(37) CometProject +Input [3]: [ss_sold_time_sk#16, ss_store_sk#18, t_time_sk#21] Arguments: [ss_store_sk#18], [ss_store_sk#18] -(41) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#26, s_store_name#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_name), EqualTo(s_store_name,ese), IsNotNull(s_store_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [s_store_sk#26, s_store_name#27] -Condition : ((isnotnull(s_store_name#27) AND (s_store_name#27 = ese)) AND isnotnull(s_store_sk#26)) - -(43) CometProject -Input [2]: [s_store_sk#26, s_store_name#27] -Arguments: [s_store_sk#26], [s_store_sk#26] +(38) ReusedExchange [Reuses operator id: 19] +Output [1]: [s_store_sk#24] -(44) CometBroadcastExchange -Input [1]: [s_store_sk#26] -Arguments: [s_store_sk#26] - -(45) CometBroadcastHashJoin +(39) CometBroadcastHashJoin Left output [1]: [ss_store_sk#18] -Right output [1]: [s_store_sk#26] -Arguments: [ss_store_sk#18], [s_store_sk#26], Inner +Right output [1]: [s_store_sk#24] +Arguments: [ss_store_sk#18], [s_store_sk#24], Inner -(46) CometProject -Input [2]: [ss_store_sk#18, s_store_sk#26] +(40) CometProject +Input [2]: [ss_store_sk#18, s_store_sk#24] -(47) CometHashAggregate +(41) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -(48) ColumnarToRow [codegen id : 2] -Input [1]: [count#28] +(42) ColumnarToRow [codegen id : 2] +Input [1]: [count#25] -(49) Exchange -Input [1]: [count#28] +(43) Exchange +Input [1]: [count#25] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] -(50) HashAggregate [codegen id : 3] -Input [1]: [count#28] +(44) HashAggregate [codegen id : 3] +Input [1]: [count#25] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#29] -Results [1]: [count(1)#29 AS h9_to_9_30#30] +Aggregate Attributes [1]: [count(1)#26] +Results [1]: [count(1)#26 AS h9_to_9_30#27] -(51) BroadcastExchange -Input [1]: [h9_to_9_30#30] +(45) BroadcastExchange +Input [1]: [h9_to_9_30#27] Arguments: IdentityBroadcastMode, [plan_id=3] -(52) BroadcastNestedLoopJoin [codegen id : 16] +(46) BroadcastNestedLoopJoin [codegen id : 16] Join type: Inner Join condition: None -(53) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#31, ss_hdemo_sk#32, ss_store_sk#33, ss_sold_date_sk#34] +(47) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_sold_time_sk#28, ss_hdemo_sk#29, ss_store_sk#30, ss_sold_date_sk#31] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(54) CometFilter -Input [4]: [ss_sold_time_sk#31, ss_hdemo_sk#32, ss_store_sk#33, ss_sold_date_sk#34] -Condition : ((isnotnull(ss_hdemo_sk#32) AND isnotnull(ss_sold_time_sk#31)) AND isnotnull(ss_store_sk#33)) - -(55) CometProject -Input [4]: [ss_sold_time_sk#31, ss_hdemo_sk#32, ss_store_sk#33, ss_sold_date_sk#34] -Arguments: [ss_sold_time_sk#31, ss_hdemo_sk#32, ss_store_sk#33], [ss_sold_time_sk#31, ss_hdemo_sk#32, ss_store_sk#33] - -(56) Scan parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#35, hd_dep_count#36, hd_vehicle_count#37] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(Or(And(EqualTo(hd_dep_count,4),LessThanOrEqual(hd_vehicle_count,6)),And(EqualTo(hd_dep_count,2),LessThanOrEqual(hd_vehicle_count,4))),And(EqualTo(hd_dep_count,0),LessThanOrEqual(hd_vehicle_count,2))), IsNotNull(hd_demo_sk)] -ReadSchema: struct +(48) CometFilter +Input [4]: [ss_sold_time_sk#28, ss_hdemo_sk#29, ss_store_sk#30, ss_sold_date_sk#31] +Condition : ((isnotnull(ss_hdemo_sk#29) AND isnotnull(ss_sold_time_sk#28)) AND isnotnull(ss_store_sk#30)) -(57) CometFilter -Input [3]: [hd_demo_sk#35, hd_dep_count#36, hd_vehicle_count#37] -Condition : (((((hd_dep_count#36 = 4) AND (hd_vehicle_count#37 <= 6)) OR ((hd_dep_count#36 = 2) AND (hd_vehicle_count#37 <= 4))) OR ((hd_dep_count#36 = 0) AND (hd_vehicle_count#37 <= 2))) AND isnotnull(hd_demo_sk#35)) +(49) CometProject +Input [4]: [ss_sold_time_sk#28, ss_hdemo_sk#29, ss_store_sk#30, ss_sold_date_sk#31] +Arguments: [ss_sold_time_sk#28, ss_hdemo_sk#29, ss_store_sk#30], [ss_sold_time_sk#28, ss_hdemo_sk#29, ss_store_sk#30] -(58) CometProject -Input [3]: [hd_demo_sk#35, hd_dep_count#36, hd_vehicle_count#37] -Arguments: [hd_demo_sk#35], [hd_demo_sk#35] - -(59) CometBroadcastExchange -Input [1]: [hd_demo_sk#35] -Arguments: [hd_demo_sk#35] +(50) ReusedExchange [Reuses operator id: 7] +Output [1]: [hd_demo_sk#32] -(60) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#31, ss_hdemo_sk#32, ss_store_sk#33] -Right output [1]: [hd_demo_sk#35] -Arguments: [ss_hdemo_sk#32], [hd_demo_sk#35], Inner +(51) CometBroadcastHashJoin +Left output [3]: [ss_sold_time_sk#28, ss_hdemo_sk#29, ss_store_sk#30] +Right output [1]: [hd_demo_sk#32] +Arguments: [ss_hdemo_sk#29], [hd_demo_sk#32], Inner -(61) CometProject -Input [4]: [ss_sold_time_sk#31, ss_hdemo_sk#32, ss_store_sk#33, hd_demo_sk#35] -Arguments: [ss_sold_time_sk#31, ss_store_sk#33], [ss_sold_time_sk#31, ss_store_sk#33] +(52) CometProject +Input [4]: [ss_sold_time_sk#28, ss_hdemo_sk#29, ss_store_sk#30, hd_demo_sk#32] +Arguments: [ss_sold_time_sk#28, ss_store_sk#30], [ss_sold_time_sk#28, ss_store_sk#30] -(62) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#38, t_hour#39, t_minute#40] +(53) Scan parquet spark_catalog.default.time_dim +Output [3]: [t_time_sk#33, t_hour#34, t_minute#35] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(63) CometFilter -Input [3]: [t_time_sk#38, t_hour#39, t_minute#40] -Condition : ((((isnotnull(t_hour#39) AND isnotnull(t_minute#40)) AND (t_hour#39 = 9)) AND (t_minute#40 >= 30)) AND isnotnull(t_time_sk#38)) - -(64) CometProject -Input [3]: [t_time_sk#38, t_hour#39, t_minute#40] -Arguments: [t_time_sk#38], [t_time_sk#38] - -(65) CometBroadcastExchange -Input [1]: [t_time_sk#38] -Arguments: [t_time_sk#38] - -(66) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#31, ss_store_sk#33] -Right output [1]: [t_time_sk#38] -Arguments: [ss_sold_time_sk#31], [t_time_sk#38], Inner +(54) CometFilter +Input [3]: [t_time_sk#33, t_hour#34, t_minute#35] +Condition : ((((isnotnull(t_hour#34) AND isnotnull(t_minute#35)) AND (t_hour#34 = 9)) AND (t_minute#35 >= 30)) AND isnotnull(t_time_sk#33)) -(67) CometProject -Input [3]: [ss_sold_time_sk#31, ss_store_sk#33, t_time_sk#38] -Arguments: [ss_store_sk#33], [ss_store_sk#33] +(55) CometProject +Input [3]: [t_time_sk#33, t_hour#34, t_minute#35] +Arguments: [t_time_sk#33], [t_time_sk#33] -(68) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#41, s_store_name#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_name), EqualTo(s_store_name,ese), IsNotNull(s_store_sk)] -ReadSchema: struct +(56) CometBroadcastExchange +Input [1]: [t_time_sk#33] +Arguments: [t_time_sk#33] -(69) CometFilter -Input [2]: [s_store_sk#41, s_store_name#42] -Condition : ((isnotnull(s_store_name#42) AND (s_store_name#42 = ese)) AND isnotnull(s_store_sk#41)) +(57) CometBroadcastHashJoin +Left output [2]: [ss_sold_time_sk#28, ss_store_sk#30] +Right output [1]: [t_time_sk#33] +Arguments: [ss_sold_time_sk#28], [t_time_sk#33], Inner -(70) CometProject -Input [2]: [s_store_sk#41, s_store_name#42] -Arguments: [s_store_sk#41], [s_store_sk#41] +(58) CometProject +Input [3]: [ss_sold_time_sk#28, ss_store_sk#30, t_time_sk#33] +Arguments: [ss_store_sk#30], [ss_store_sk#30] -(71) CometBroadcastExchange -Input [1]: [s_store_sk#41] -Arguments: [s_store_sk#41] +(59) ReusedExchange [Reuses operator id: 19] +Output [1]: [s_store_sk#36] -(72) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#33] -Right output [1]: [s_store_sk#41] -Arguments: [ss_store_sk#33], [s_store_sk#41], Inner +(60) CometBroadcastHashJoin +Left output [1]: [ss_store_sk#30] +Right output [1]: [s_store_sk#36] +Arguments: [ss_store_sk#30], [s_store_sk#36], Inner -(73) CometProject -Input [2]: [ss_store_sk#33, s_store_sk#41] +(61) CometProject +Input [2]: [ss_store_sk#30, s_store_sk#36] -(74) CometHashAggregate +(62) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -(75) ColumnarToRow [codegen id : 4] -Input [1]: [count#43] +(63) ColumnarToRow [codegen id : 4] +Input [1]: [count#37] -(76) Exchange -Input [1]: [count#43] +(64) Exchange +Input [1]: [count#37] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] -(77) HashAggregate [codegen id : 5] -Input [1]: [count#43] +(65) HashAggregate [codegen id : 5] +Input [1]: [count#37] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#44] -Results [1]: [count(1)#44 AS h9_30_to_10#45] +Aggregate Attributes [1]: [count(1)#38] +Results [1]: [count(1)#38 AS h9_30_to_10#39] -(78) BroadcastExchange -Input [1]: [h9_30_to_10#45] +(66) BroadcastExchange +Input [1]: [h9_30_to_10#39] Arguments: IdentityBroadcastMode, [plan_id=5] -(79) BroadcastNestedLoopJoin [codegen id : 16] +(67) BroadcastNestedLoopJoin [codegen id : 16] Join type: Inner Join condition: None -(80) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#46, ss_hdemo_sk#47, ss_store_sk#48, ss_sold_date_sk#49] +(68) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_sold_time_sk#40, ss_hdemo_sk#41, ss_store_sk#42, ss_sold_date_sk#43] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(81) CometFilter -Input [4]: [ss_sold_time_sk#46, ss_hdemo_sk#47, ss_store_sk#48, ss_sold_date_sk#49] -Condition : ((isnotnull(ss_hdemo_sk#47) AND isnotnull(ss_sold_time_sk#46)) AND isnotnull(ss_store_sk#48)) - -(82) CometProject -Input [4]: [ss_sold_time_sk#46, ss_hdemo_sk#47, ss_store_sk#48, ss_sold_date_sk#49] -Arguments: [ss_sold_time_sk#46, ss_hdemo_sk#47, ss_store_sk#48], [ss_sold_time_sk#46, ss_hdemo_sk#47, ss_store_sk#48] - -(83) Scan parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#50, hd_dep_count#51, hd_vehicle_count#52] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(Or(And(EqualTo(hd_dep_count,4),LessThanOrEqual(hd_vehicle_count,6)),And(EqualTo(hd_dep_count,2),LessThanOrEqual(hd_vehicle_count,4))),And(EqualTo(hd_dep_count,0),LessThanOrEqual(hd_vehicle_count,2))), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(84) CometFilter -Input [3]: [hd_demo_sk#50, hd_dep_count#51, hd_vehicle_count#52] -Condition : (((((hd_dep_count#51 = 4) AND (hd_vehicle_count#52 <= 6)) OR ((hd_dep_count#51 = 2) AND (hd_vehicle_count#52 <= 4))) OR ((hd_dep_count#51 = 0) AND (hd_vehicle_count#52 <= 2))) AND isnotnull(hd_demo_sk#50)) +(69) CometFilter +Input [4]: [ss_sold_time_sk#40, ss_hdemo_sk#41, ss_store_sk#42, ss_sold_date_sk#43] +Condition : ((isnotnull(ss_hdemo_sk#41) AND isnotnull(ss_sold_time_sk#40)) AND isnotnull(ss_store_sk#42)) -(85) CometProject -Input [3]: [hd_demo_sk#50, hd_dep_count#51, hd_vehicle_count#52] -Arguments: [hd_demo_sk#50], [hd_demo_sk#50] +(70) CometProject +Input [4]: [ss_sold_time_sk#40, ss_hdemo_sk#41, ss_store_sk#42, ss_sold_date_sk#43] +Arguments: [ss_sold_time_sk#40, ss_hdemo_sk#41, ss_store_sk#42], [ss_sold_time_sk#40, ss_hdemo_sk#41, ss_store_sk#42] -(86) CometBroadcastExchange -Input [1]: [hd_demo_sk#50] -Arguments: [hd_demo_sk#50] +(71) ReusedExchange [Reuses operator id: 7] +Output [1]: [hd_demo_sk#44] -(87) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#46, ss_hdemo_sk#47, ss_store_sk#48] -Right output [1]: [hd_demo_sk#50] -Arguments: [ss_hdemo_sk#47], [hd_demo_sk#50], Inner +(72) CometBroadcastHashJoin +Left output [3]: [ss_sold_time_sk#40, ss_hdemo_sk#41, ss_store_sk#42] +Right output [1]: [hd_demo_sk#44] +Arguments: [ss_hdemo_sk#41], [hd_demo_sk#44], Inner -(88) CometProject -Input [4]: [ss_sold_time_sk#46, ss_hdemo_sk#47, ss_store_sk#48, hd_demo_sk#50] -Arguments: [ss_sold_time_sk#46, ss_store_sk#48], [ss_sold_time_sk#46, ss_store_sk#48] +(73) CometProject +Input [4]: [ss_sold_time_sk#40, ss_hdemo_sk#41, ss_store_sk#42, hd_demo_sk#44] +Arguments: [ss_sold_time_sk#40, ss_store_sk#42], [ss_sold_time_sk#40, ss_store_sk#42] -(89) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#53, t_hour#54, t_minute#55] +(74) Scan parquet spark_catalog.default.time_dim +Output [3]: [t_time_sk#45, t_hour#46, t_minute#47] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(90) CometFilter -Input [3]: [t_time_sk#53, t_hour#54, t_minute#55] -Condition : ((((isnotnull(t_hour#54) AND isnotnull(t_minute#55)) AND (t_hour#54 = 10)) AND (t_minute#55 < 30)) AND isnotnull(t_time_sk#53)) - -(91) CometProject -Input [3]: [t_time_sk#53, t_hour#54, t_minute#55] -Arguments: [t_time_sk#53], [t_time_sk#53] - -(92) CometBroadcastExchange -Input [1]: [t_time_sk#53] -Arguments: [t_time_sk#53] - -(93) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#46, ss_store_sk#48] -Right output [1]: [t_time_sk#53] -Arguments: [ss_sold_time_sk#46], [t_time_sk#53], Inner +(75) CometFilter +Input [3]: [t_time_sk#45, t_hour#46, t_minute#47] +Condition : ((((isnotnull(t_hour#46) AND isnotnull(t_minute#47)) AND (t_hour#46 = 10)) AND (t_minute#47 < 30)) AND isnotnull(t_time_sk#45)) -(94) CometProject -Input [3]: [ss_sold_time_sk#46, ss_store_sk#48, t_time_sk#53] -Arguments: [ss_store_sk#48], [ss_store_sk#48] +(76) CometProject +Input [3]: [t_time_sk#45, t_hour#46, t_minute#47] +Arguments: [t_time_sk#45], [t_time_sk#45] -(95) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#56, s_store_name#57] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_name), EqualTo(s_store_name,ese), IsNotNull(s_store_sk)] -ReadSchema: struct +(77) CometBroadcastExchange +Input [1]: [t_time_sk#45] +Arguments: [t_time_sk#45] -(96) CometFilter -Input [2]: [s_store_sk#56, s_store_name#57] -Condition : ((isnotnull(s_store_name#57) AND (s_store_name#57 = ese)) AND isnotnull(s_store_sk#56)) +(78) CometBroadcastHashJoin +Left output [2]: [ss_sold_time_sk#40, ss_store_sk#42] +Right output [1]: [t_time_sk#45] +Arguments: [ss_sold_time_sk#40], [t_time_sk#45], Inner -(97) CometProject -Input [2]: [s_store_sk#56, s_store_name#57] -Arguments: [s_store_sk#56], [s_store_sk#56] +(79) CometProject +Input [3]: [ss_sold_time_sk#40, ss_store_sk#42, t_time_sk#45] +Arguments: [ss_store_sk#42], [ss_store_sk#42] -(98) CometBroadcastExchange -Input [1]: [s_store_sk#56] -Arguments: [s_store_sk#56] +(80) ReusedExchange [Reuses operator id: 19] +Output [1]: [s_store_sk#48] -(99) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#48] -Right output [1]: [s_store_sk#56] -Arguments: [ss_store_sk#48], [s_store_sk#56], Inner +(81) CometBroadcastHashJoin +Left output [1]: [ss_store_sk#42] +Right output [1]: [s_store_sk#48] +Arguments: [ss_store_sk#42], [s_store_sk#48], Inner -(100) CometProject -Input [2]: [ss_store_sk#48, s_store_sk#56] +(82) CometProject +Input [2]: [ss_store_sk#42, s_store_sk#48] -(101) CometHashAggregate +(83) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -(102) ColumnarToRow [codegen id : 6] -Input [1]: [count#58] +(84) ColumnarToRow [codegen id : 6] +Input [1]: [count#49] -(103) Exchange -Input [1]: [count#58] +(85) Exchange +Input [1]: [count#49] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(104) HashAggregate [codegen id : 7] -Input [1]: [count#58] +(86) HashAggregate [codegen id : 7] +Input [1]: [count#49] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#59] -Results [1]: [count(1)#59 AS h10_to_10_30#60] +Aggregate Attributes [1]: [count(1)#50] +Results [1]: [count(1)#50 AS h10_to_10_30#51] -(105) BroadcastExchange -Input [1]: [h10_to_10_30#60] +(87) BroadcastExchange +Input [1]: [h10_to_10_30#51] Arguments: IdentityBroadcastMode, [plan_id=7] -(106) BroadcastNestedLoopJoin [codegen id : 16] +(88) BroadcastNestedLoopJoin [codegen id : 16] Join type: Inner Join condition: None -(107) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#61, ss_hdemo_sk#62, ss_store_sk#63, ss_sold_date_sk#64] +(89) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_sold_time_sk#52, ss_hdemo_sk#53, ss_store_sk#54, ss_sold_date_sk#55] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(108) CometFilter -Input [4]: [ss_sold_time_sk#61, ss_hdemo_sk#62, ss_store_sk#63, ss_sold_date_sk#64] -Condition : ((isnotnull(ss_hdemo_sk#62) AND isnotnull(ss_sold_time_sk#61)) AND isnotnull(ss_store_sk#63)) - -(109) CometProject -Input [4]: [ss_sold_time_sk#61, ss_hdemo_sk#62, ss_store_sk#63, ss_sold_date_sk#64] -Arguments: [ss_sold_time_sk#61, ss_hdemo_sk#62, ss_store_sk#63], [ss_sold_time_sk#61, ss_hdemo_sk#62, ss_store_sk#63] - -(110) Scan parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#65, hd_dep_count#66, hd_vehicle_count#67] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(Or(And(EqualTo(hd_dep_count,4),LessThanOrEqual(hd_vehicle_count,6)),And(EqualTo(hd_dep_count,2),LessThanOrEqual(hd_vehicle_count,4))),And(EqualTo(hd_dep_count,0),LessThanOrEqual(hd_vehicle_count,2))), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(111) CometFilter -Input [3]: [hd_demo_sk#65, hd_dep_count#66, hd_vehicle_count#67] -Condition : (((((hd_dep_count#66 = 4) AND (hd_vehicle_count#67 <= 6)) OR ((hd_dep_count#66 = 2) AND (hd_vehicle_count#67 <= 4))) OR ((hd_dep_count#66 = 0) AND (hd_vehicle_count#67 <= 2))) AND isnotnull(hd_demo_sk#65)) +(90) CometFilter +Input [4]: [ss_sold_time_sk#52, ss_hdemo_sk#53, ss_store_sk#54, ss_sold_date_sk#55] +Condition : ((isnotnull(ss_hdemo_sk#53) AND isnotnull(ss_sold_time_sk#52)) AND isnotnull(ss_store_sk#54)) -(112) CometProject -Input [3]: [hd_demo_sk#65, hd_dep_count#66, hd_vehicle_count#67] -Arguments: [hd_demo_sk#65], [hd_demo_sk#65] +(91) CometProject +Input [4]: [ss_sold_time_sk#52, ss_hdemo_sk#53, ss_store_sk#54, ss_sold_date_sk#55] +Arguments: [ss_sold_time_sk#52, ss_hdemo_sk#53, ss_store_sk#54], [ss_sold_time_sk#52, ss_hdemo_sk#53, ss_store_sk#54] -(113) CometBroadcastExchange -Input [1]: [hd_demo_sk#65] -Arguments: [hd_demo_sk#65] +(92) ReusedExchange [Reuses operator id: 7] +Output [1]: [hd_demo_sk#56] -(114) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#61, ss_hdemo_sk#62, ss_store_sk#63] -Right output [1]: [hd_demo_sk#65] -Arguments: [ss_hdemo_sk#62], [hd_demo_sk#65], Inner +(93) CometBroadcastHashJoin +Left output [3]: [ss_sold_time_sk#52, ss_hdemo_sk#53, ss_store_sk#54] +Right output [1]: [hd_demo_sk#56] +Arguments: [ss_hdemo_sk#53], [hd_demo_sk#56], Inner -(115) CometProject -Input [4]: [ss_sold_time_sk#61, ss_hdemo_sk#62, ss_store_sk#63, hd_demo_sk#65] -Arguments: [ss_sold_time_sk#61, ss_store_sk#63], [ss_sold_time_sk#61, ss_store_sk#63] +(94) CometProject +Input [4]: [ss_sold_time_sk#52, ss_hdemo_sk#53, ss_store_sk#54, hd_demo_sk#56] +Arguments: [ss_sold_time_sk#52, ss_store_sk#54], [ss_sold_time_sk#52, ss_store_sk#54] -(116) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#68, t_hour#69, t_minute#70] +(95) Scan parquet spark_catalog.default.time_dim +Output [3]: [t_time_sk#57, t_hour#58, t_minute#59] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(117) CometFilter -Input [3]: [t_time_sk#68, t_hour#69, t_minute#70] -Condition : ((((isnotnull(t_hour#69) AND isnotnull(t_minute#70)) AND (t_hour#69 = 10)) AND (t_minute#70 >= 30)) AND isnotnull(t_time_sk#68)) - -(118) CometProject -Input [3]: [t_time_sk#68, t_hour#69, t_minute#70] -Arguments: [t_time_sk#68], [t_time_sk#68] - -(119) CometBroadcastExchange -Input [1]: [t_time_sk#68] -Arguments: [t_time_sk#68] - -(120) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#61, ss_store_sk#63] -Right output [1]: [t_time_sk#68] -Arguments: [ss_sold_time_sk#61], [t_time_sk#68], Inner +(96) CometFilter +Input [3]: [t_time_sk#57, t_hour#58, t_minute#59] +Condition : ((((isnotnull(t_hour#58) AND isnotnull(t_minute#59)) AND (t_hour#58 = 10)) AND (t_minute#59 >= 30)) AND isnotnull(t_time_sk#57)) -(121) CometProject -Input [3]: [ss_sold_time_sk#61, ss_store_sk#63, t_time_sk#68] -Arguments: [ss_store_sk#63], [ss_store_sk#63] +(97) CometProject +Input [3]: [t_time_sk#57, t_hour#58, t_minute#59] +Arguments: [t_time_sk#57], [t_time_sk#57] -(122) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#71, s_store_name#72] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_name), EqualTo(s_store_name,ese), IsNotNull(s_store_sk)] -ReadSchema: struct +(98) CometBroadcastExchange +Input [1]: [t_time_sk#57] +Arguments: [t_time_sk#57] -(123) CometFilter -Input [2]: [s_store_sk#71, s_store_name#72] -Condition : ((isnotnull(s_store_name#72) AND (s_store_name#72 = ese)) AND isnotnull(s_store_sk#71)) +(99) CometBroadcastHashJoin +Left output [2]: [ss_sold_time_sk#52, ss_store_sk#54] +Right output [1]: [t_time_sk#57] +Arguments: [ss_sold_time_sk#52], [t_time_sk#57], Inner -(124) CometProject -Input [2]: [s_store_sk#71, s_store_name#72] -Arguments: [s_store_sk#71], [s_store_sk#71] +(100) CometProject +Input [3]: [ss_sold_time_sk#52, ss_store_sk#54, t_time_sk#57] +Arguments: [ss_store_sk#54], [ss_store_sk#54] -(125) CometBroadcastExchange -Input [1]: [s_store_sk#71] -Arguments: [s_store_sk#71] +(101) ReusedExchange [Reuses operator id: 19] +Output [1]: [s_store_sk#60] -(126) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#63] -Right output [1]: [s_store_sk#71] -Arguments: [ss_store_sk#63], [s_store_sk#71], Inner +(102) CometBroadcastHashJoin +Left output [1]: [ss_store_sk#54] +Right output [1]: [s_store_sk#60] +Arguments: [ss_store_sk#54], [s_store_sk#60], Inner -(127) CometProject -Input [2]: [ss_store_sk#63, s_store_sk#71] +(103) CometProject +Input [2]: [ss_store_sk#54, s_store_sk#60] -(128) CometHashAggregate +(104) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -(129) ColumnarToRow [codegen id : 8] -Input [1]: [count#73] +(105) ColumnarToRow [codegen id : 8] +Input [1]: [count#61] -(130) Exchange -Input [1]: [count#73] +(106) Exchange +Input [1]: [count#61] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=8] -(131) HashAggregate [codegen id : 9] -Input [1]: [count#73] +(107) HashAggregate [codegen id : 9] +Input [1]: [count#61] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#74] -Results [1]: [count(1)#74 AS h10_30_to_11#75] +Aggregate Attributes [1]: [count(1)#62] +Results [1]: [count(1)#62 AS h10_30_to_11#63] -(132) BroadcastExchange -Input [1]: [h10_30_to_11#75] +(108) BroadcastExchange +Input [1]: [h10_30_to_11#63] Arguments: IdentityBroadcastMode, [plan_id=9] -(133) BroadcastNestedLoopJoin [codegen id : 16] +(109) BroadcastNestedLoopJoin [codegen id : 16] Join type: Inner Join condition: None -(134) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78, ss_sold_date_sk#79] +(110) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_sold_time_sk#64, ss_hdemo_sk#65, ss_store_sk#66, ss_sold_date_sk#67] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(135) CometFilter -Input [4]: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78, ss_sold_date_sk#79] -Condition : ((isnotnull(ss_hdemo_sk#77) AND isnotnull(ss_sold_time_sk#76)) AND isnotnull(ss_store_sk#78)) - -(136) CometProject -Input [4]: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78, ss_sold_date_sk#79] -Arguments: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78], [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78] - -(137) Scan parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#80, hd_dep_count#81, hd_vehicle_count#82] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(Or(And(EqualTo(hd_dep_count,4),LessThanOrEqual(hd_vehicle_count,6)),And(EqualTo(hd_dep_count,2),LessThanOrEqual(hd_vehicle_count,4))),And(EqualTo(hd_dep_count,0),LessThanOrEqual(hd_vehicle_count,2))), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(138) CometFilter -Input [3]: [hd_demo_sk#80, hd_dep_count#81, hd_vehicle_count#82] -Condition : (((((hd_dep_count#81 = 4) AND (hd_vehicle_count#82 <= 6)) OR ((hd_dep_count#81 = 2) AND (hd_vehicle_count#82 <= 4))) OR ((hd_dep_count#81 = 0) AND (hd_vehicle_count#82 <= 2))) AND isnotnull(hd_demo_sk#80)) +(111) CometFilter +Input [4]: [ss_sold_time_sk#64, ss_hdemo_sk#65, ss_store_sk#66, ss_sold_date_sk#67] +Condition : ((isnotnull(ss_hdemo_sk#65) AND isnotnull(ss_sold_time_sk#64)) AND isnotnull(ss_store_sk#66)) -(139) CometProject -Input [3]: [hd_demo_sk#80, hd_dep_count#81, hd_vehicle_count#82] -Arguments: [hd_demo_sk#80], [hd_demo_sk#80] +(112) CometProject +Input [4]: [ss_sold_time_sk#64, ss_hdemo_sk#65, ss_store_sk#66, ss_sold_date_sk#67] +Arguments: [ss_sold_time_sk#64, ss_hdemo_sk#65, ss_store_sk#66], [ss_sold_time_sk#64, ss_hdemo_sk#65, ss_store_sk#66] -(140) CometBroadcastExchange -Input [1]: [hd_demo_sk#80] -Arguments: [hd_demo_sk#80] +(113) ReusedExchange [Reuses operator id: 7] +Output [1]: [hd_demo_sk#68] -(141) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78] -Right output [1]: [hd_demo_sk#80] -Arguments: [ss_hdemo_sk#77], [hd_demo_sk#80], Inner +(114) CometBroadcastHashJoin +Left output [3]: [ss_sold_time_sk#64, ss_hdemo_sk#65, ss_store_sk#66] +Right output [1]: [hd_demo_sk#68] +Arguments: [ss_hdemo_sk#65], [hd_demo_sk#68], Inner -(142) CometProject -Input [4]: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78, hd_demo_sk#80] -Arguments: [ss_sold_time_sk#76, ss_store_sk#78], [ss_sold_time_sk#76, ss_store_sk#78] +(115) CometProject +Input [4]: [ss_sold_time_sk#64, ss_hdemo_sk#65, ss_store_sk#66, hd_demo_sk#68] +Arguments: [ss_sold_time_sk#64, ss_store_sk#66], [ss_sold_time_sk#64, ss_store_sk#66] -(143) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#83, t_hour#84, t_minute#85] +(116) Scan parquet spark_catalog.default.time_dim +Output [3]: [t_time_sk#69, t_hour#70, t_minute#71] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(144) CometFilter -Input [3]: [t_time_sk#83, t_hour#84, t_minute#85] -Condition : ((((isnotnull(t_hour#84) AND isnotnull(t_minute#85)) AND (t_hour#84 = 11)) AND (t_minute#85 < 30)) AND isnotnull(t_time_sk#83)) - -(145) CometProject -Input [3]: [t_time_sk#83, t_hour#84, t_minute#85] -Arguments: [t_time_sk#83], [t_time_sk#83] - -(146) CometBroadcastExchange -Input [1]: [t_time_sk#83] -Arguments: [t_time_sk#83] - -(147) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#76, ss_store_sk#78] -Right output [1]: [t_time_sk#83] -Arguments: [ss_sold_time_sk#76], [t_time_sk#83], Inner +(117) CometFilter +Input [3]: [t_time_sk#69, t_hour#70, t_minute#71] +Condition : ((((isnotnull(t_hour#70) AND isnotnull(t_minute#71)) AND (t_hour#70 = 11)) AND (t_minute#71 < 30)) AND isnotnull(t_time_sk#69)) -(148) CometProject -Input [3]: [ss_sold_time_sk#76, ss_store_sk#78, t_time_sk#83] -Arguments: [ss_store_sk#78], [ss_store_sk#78] +(118) CometProject +Input [3]: [t_time_sk#69, t_hour#70, t_minute#71] +Arguments: [t_time_sk#69], [t_time_sk#69] -(149) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#86, s_store_name#87] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_name), EqualTo(s_store_name,ese), IsNotNull(s_store_sk)] -ReadSchema: struct +(119) CometBroadcastExchange +Input [1]: [t_time_sk#69] +Arguments: [t_time_sk#69] -(150) CometFilter -Input [2]: [s_store_sk#86, s_store_name#87] -Condition : ((isnotnull(s_store_name#87) AND (s_store_name#87 = ese)) AND isnotnull(s_store_sk#86)) +(120) CometBroadcastHashJoin +Left output [2]: [ss_sold_time_sk#64, ss_store_sk#66] +Right output [1]: [t_time_sk#69] +Arguments: [ss_sold_time_sk#64], [t_time_sk#69], Inner -(151) CometProject -Input [2]: [s_store_sk#86, s_store_name#87] -Arguments: [s_store_sk#86], [s_store_sk#86] +(121) CometProject +Input [3]: [ss_sold_time_sk#64, ss_store_sk#66, t_time_sk#69] +Arguments: [ss_store_sk#66], [ss_store_sk#66] -(152) CometBroadcastExchange -Input [1]: [s_store_sk#86] -Arguments: [s_store_sk#86] +(122) ReusedExchange [Reuses operator id: 19] +Output [1]: [s_store_sk#72] -(153) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#78] -Right output [1]: [s_store_sk#86] -Arguments: [ss_store_sk#78], [s_store_sk#86], Inner +(123) CometBroadcastHashJoin +Left output [1]: [ss_store_sk#66] +Right output [1]: [s_store_sk#72] +Arguments: [ss_store_sk#66], [s_store_sk#72], Inner -(154) CometProject -Input [2]: [ss_store_sk#78, s_store_sk#86] +(124) CometProject +Input [2]: [ss_store_sk#66, s_store_sk#72] -(155) CometHashAggregate +(125) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -(156) ColumnarToRow [codegen id : 10] -Input [1]: [count#88] +(126) ColumnarToRow [codegen id : 10] +Input [1]: [count#73] -(157) Exchange -Input [1]: [count#88] +(127) Exchange +Input [1]: [count#73] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] -(158) HashAggregate [codegen id : 11] -Input [1]: [count#88] +(128) HashAggregate [codegen id : 11] +Input [1]: [count#73] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#89] -Results [1]: [count(1)#89 AS h11_to_11_30#90] +Aggregate Attributes [1]: [count(1)#74] +Results [1]: [count(1)#74 AS h11_to_11_30#75] -(159) BroadcastExchange -Input [1]: [h11_to_11_30#90] +(129) BroadcastExchange +Input [1]: [h11_to_11_30#75] Arguments: IdentityBroadcastMode, [plan_id=11] -(160) BroadcastNestedLoopJoin [codegen id : 16] +(130) BroadcastNestedLoopJoin [codegen id : 16] Join type: Inner Join condition: None -(161) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#91, ss_hdemo_sk#92, ss_store_sk#93, ss_sold_date_sk#94] +(131) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78, ss_sold_date_sk#79] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(162) CometFilter -Input [4]: [ss_sold_time_sk#91, ss_hdemo_sk#92, ss_store_sk#93, ss_sold_date_sk#94] -Condition : ((isnotnull(ss_hdemo_sk#92) AND isnotnull(ss_sold_time_sk#91)) AND isnotnull(ss_store_sk#93)) - -(163) CometProject -Input [4]: [ss_sold_time_sk#91, ss_hdemo_sk#92, ss_store_sk#93, ss_sold_date_sk#94] -Arguments: [ss_sold_time_sk#91, ss_hdemo_sk#92, ss_store_sk#93], [ss_sold_time_sk#91, ss_hdemo_sk#92, ss_store_sk#93] - -(164) Scan parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#95, hd_dep_count#96, hd_vehicle_count#97] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(Or(And(EqualTo(hd_dep_count,4),LessThanOrEqual(hd_vehicle_count,6)),And(EqualTo(hd_dep_count,2),LessThanOrEqual(hd_vehicle_count,4))),And(EqualTo(hd_dep_count,0),LessThanOrEqual(hd_vehicle_count,2))), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(165) CometFilter -Input [3]: [hd_demo_sk#95, hd_dep_count#96, hd_vehicle_count#97] -Condition : (((((hd_dep_count#96 = 4) AND (hd_vehicle_count#97 <= 6)) OR ((hd_dep_count#96 = 2) AND (hd_vehicle_count#97 <= 4))) OR ((hd_dep_count#96 = 0) AND (hd_vehicle_count#97 <= 2))) AND isnotnull(hd_demo_sk#95)) +(132) CometFilter +Input [4]: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78, ss_sold_date_sk#79] +Condition : ((isnotnull(ss_hdemo_sk#77) AND isnotnull(ss_sold_time_sk#76)) AND isnotnull(ss_store_sk#78)) -(166) CometProject -Input [3]: [hd_demo_sk#95, hd_dep_count#96, hd_vehicle_count#97] -Arguments: [hd_demo_sk#95], [hd_demo_sk#95] +(133) CometProject +Input [4]: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78, ss_sold_date_sk#79] +Arguments: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78], [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78] -(167) CometBroadcastExchange -Input [1]: [hd_demo_sk#95] -Arguments: [hd_demo_sk#95] +(134) ReusedExchange [Reuses operator id: 7] +Output [1]: [hd_demo_sk#80] -(168) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#91, ss_hdemo_sk#92, ss_store_sk#93] -Right output [1]: [hd_demo_sk#95] -Arguments: [ss_hdemo_sk#92], [hd_demo_sk#95], Inner +(135) CometBroadcastHashJoin +Left output [3]: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78] +Right output [1]: [hd_demo_sk#80] +Arguments: [ss_hdemo_sk#77], [hd_demo_sk#80], Inner -(169) CometProject -Input [4]: [ss_sold_time_sk#91, ss_hdemo_sk#92, ss_store_sk#93, hd_demo_sk#95] -Arguments: [ss_sold_time_sk#91, ss_store_sk#93], [ss_sold_time_sk#91, ss_store_sk#93] +(136) CometProject +Input [4]: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78, hd_demo_sk#80] +Arguments: [ss_sold_time_sk#76, ss_store_sk#78], [ss_sold_time_sk#76, ss_store_sk#78] -(170) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#98, t_hour#99, t_minute#100] +(137) Scan parquet spark_catalog.default.time_dim +Output [3]: [t_time_sk#81, t_hour#82, t_minute#83] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(171) CometFilter -Input [3]: [t_time_sk#98, t_hour#99, t_minute#100] -Condition : ((((isnotnull(t_hour#99) AND isnotnull(t_minute#100)) AND (t_hour#99 = 11)) AND (t_minute#100 >= 30)) AND isnotnull(t_time_sk#98)) - -(172) CometProject -Input [3]: [t_time_sk#98, t_hour#99, t_minute#100] -Arguments: [t_time_sk#98], [t_time_sk#98] - -(173) CometBroadcastExchange -Input [1]: [t_time_sk#98] -Arguments: [t_time_sk#98] - -(174) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#91, ss_store_sk#93] -Right output [1]: [t_time_sk#98] -Arguments: [ss_sold_time_sk#91], [t_time_sk#98], Inner +(138) CometFilter +Input [3]: [t_time_sk#81, t_hour#82, t_minute#83] +Condition : ((((isnotnull(t_hour#82) AND isnotnull(t_minute#83)) AND (t_hour#82 = 11)) AND (t_minute#83 >= 30)) AND isnotnull(t_time_sk#81)) -(175) CometProject -Input [3]: [ss_sold_time_sk#91, ss_store_sk#93, t_time_sk#98] -Arguments: [ss_store_sk#93], [ss_store_sk#93] +(139) CometProject +Input [3]: [t_time_sk#81, t_hour#82, t_minute#83] +Arguments: [t_time_sk#81], [t_time_sk#81] -(176) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#101, s_store_name#102] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_name), EqualTo(s_store_name,ese), IsNotNull(s_store_sk)] -ReadSchema: struct +(140) CometBroadcastExchange +Input [1]: [t_time_sk#81] +Arguments: [t_time_sk#81] -(177) CometFilter -Input [2]: [s_store_sk#101, s_store_name#102] -Condition : ((isnotnull(s_store_name#102) AND (s_store_name#102 = ese)) AND isnotnull(s_store_sk#101)) +(141) CometBroadcastHashJoin +Left output [2]: [ss_sold_time_sk#76, ss_store_sk#78] +Right output [1]: [t_time_sk#81] +Arguments: [ss_sold_time_sk#76], [t_time_sk#81], Inner -(178) CometProject -Input [2]: [s_store_sk#101, s_store_name#102] -Arguments: [s_store_sk#101], [s_store_sk#101] +(142) CometProject +Input [3]: [ss_sold_time_sk#76, ss_store_sk#78, t_time_sk#81] +Arguments: [ss_store_sk#78], [ss_store_sk#78] -(179) CometBroadcastExchange -Input [1]: [s_store_sk#101] -Arguments: [s_store_sk#101] +(143) ReusedExchange [Reuses operator id: 19] +Output [1]: [s_store_sk#84] -(180) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#93] -Right output [1]: [s_store_sk#101] -Arguments: [ss_store_sk#93], [s_store_sk#101], Inner +(144) CometBroadcastHashJoin +Left output [1]: [ss_store_sk#78] +Right output [1]: [s_store_sk#84] +Arguments: [ss_store_sk#78], [s_store_sk#84], Inner -(181) CometProject -Input [2]: [ss_store_sk#93, s_store_sk#101] +(145) CometProject +Input [2]: [ss_store_sk#78, s_store_sk#84] -(182) CometHashAggregate +(146) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -(183) ColumnarToRow [codegen id : 12] -Input [1]: [count#103] +(147) ColumnarToRow [codegen id : 12] +Input [1]: [count#85] -(184) Exchange -Input [1]: [count#103] +(148) Exchange +Input [1]: [count#85] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] -(185) HashAggregate [codegen id : 13] -Input [1]: [count#103] +(149) HashAggregate [codegen id : 13] +Input [1]: [count#85] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#104] -Results [1]: [count(1)#104 AS h11_30_to_12#105] +Aggregate Attributes [1]: [count(1)#86] +Results [1]: [count(1)#86 AS h11_30_to_12#87] -(186) BroadcastExchange -Input [1]: [h11_30_to_12#105] +(150) BroadcastExchange +Input [1]: [h11_30_to_12#87] Arguments: IdentityBroadcastMode, [plan_id=13] -(187) BroadcastNestedLoopJoin [codegen id : 16] +(151) BroadcastNestedLoopJoin [codegen id : 16] Join type: Inner Join condition: None -(188) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#106, ss_hdemo_sk#107, ss_store_sk#108, ss_sold_date_sk#109] +(152) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_sold_time_sk#88, ss_hdemo_sk#89, ss_store_sk#90, ss_sold_date_sk#91] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(189) CometFilter -Input [4]: [ss_sold_time_sk#106, ss_hdemo_sk#107, ss_store_sk#108, ss_sold_date_sk#109] -Condition : ((isnotnull(ss_hdemo_sk#107) AND isnotnull(ss_sold_time_sk#106)) AND isnotnull(ss_store_sk#108)) - -(190) CometProject -Input [4]: [ss_sold_time_sk#106, ss_hdemo_sk#107, ss_store_sk#108, ss_sold_date_sk#109] -Arguments: [ss_sold_time_sk#106, ss_hdemo_sk#107, ss_store_sk#108], [ss_sold_time_sk#106, ss_hdemo_sk#107, ss_store_sk#108] +(153) CometFilter +Input [4]: [ss_sold_time_sk#88, ss_hdemo_sk#89, ss_store_sk#90, ss_sold_date_sk#91] +Condition : ((isnotnull(ss_hdemo_sk#89) AND isnotnull(ss_sold_time_sk#88)) AND isnotnull(ss_store_sk#90)) -(191) Scan parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#110, hd_dep_count#111, hd_vehicle_count#112] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(Or(And(EqualTo(hd_dep_count,4),LessThanOrEqual(hd_vehicle_count,6)),And(EqualTo(hd_dep_count,2),LessThanOrEqual(hd_vehicle_count,4))),And(EqualTo(hd_dep_count,0),LessThanOrEqual(hd_vehicle_count,2))), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(192) CometFilter -Input [3]: [hd_demo_sk#110, hd_dep_count#111, hd_vehicle_count#112] -Condition : (((((hd_dep_count#111 = 4) AND (hd_vehicle_count#112 <= 6)) OR ((hd_dep_count#111 = 2) AND (hd_vehicle_count#112 <= 4))) OR ((hd_dep_count#111 = 0) AND (hd_vehicle_count#112 <= 2))) AND isnotnull(hd_demo_sk#110)) - -(193) CometProject -Input [3]: [hd_demo_sk#110, hd_dep_count#111, hd_vehicle_count#112] -Arguments: [hd_demo_sk#110], [hd_demo_sk#110] +(154) CometProject +Input [4]: [ss_sold_time_sk#88, ss_hdemo_sk#89, ss_store_sk#90, ss_sold_date_sk#91] +Arguments: [ss_sold_time_sk#88, ss_hdemo_sk#89, ss_store_sk#90], [ss_sold_time_sk#88, ss_hdemo_sk#89, ss_store_sk#90] -(194) CometBroadcastExchange -Input [1]: [hd_demo_sk#110] -Arguments: [hd_demo_sk#110] +(155) ReusedExchange [Reuses operator id: 7] +Output [1]: [hd_demo_sk#92] -(195) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#106, ss_hdemo_sk#107, ss_store_sk#108] -Right output [1]: [hd_demo_sk#110] -Arguments: [ss_hdemo_sk#107], [hd_demo_sk#110], Inner +(156) CometBroadcastHashJoin +Left output [3]: [ss_sold_time_sk#88, ss_hdemo_sk#89, ss_store_sk#90] +Right output [1]: [hd_demo_sk#92] +Arguments: [ss_hdemo_sk#89], [hd_demo_sk#92], Inner -(196) CometProject -Input [4]: [ss_sold_time_sk#106, ss_hdemo_sk#107, ss_store_sk#108, hd_demo_sk#110] -Arguments: [ss_sold_time_sk#106, ss_store_sk#108], [ss_sold_time_sk#106, ss_store_sk#108] +(157) CometProject +Input [4]: [ss_sold_time_sk#88, ss_hdemo_sk#89, ss_store_sk#90, hd_demo_sk#92] +Arguments: [ss_sold_time_sk#88, ss_store_sk#90], [ss_sold_time_sk#88, ss_store_sk#90] -(197) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#113, t_hour#114, t_minute#115] +(158) Scan parquet spark_catalog.default.time_dim +Output [3]: [t_time_sk#93, t_hour#94, t_minute#95] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,12), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(198) CometFilter -Input [3]: [t_time_sk#113, t_hour#114, t_minute#115] -Condition : ((((isnotnull(t_hour#114) AND isnotnull(t_minute#115)) AND (t_hour#114 = 12)) AND (t_minute#115 < 30)) AND isnotnull(t_time_sk#113)) +(159) CometFilter +Input [3]: [t_time_sk#93, t_hour#94, t_minute#95] +Condition : ((((isnotnull(t_hour#94) AND isnotnull(t_minute#95)) AND (t_hour#94 = 12)) AND (t_minute#95 < 30)) AND isnotnull(t_time_sk#93)) -(199) CometProject -Input [3]: [t_time_sk#113, t_hour#114, t_minute#115] -Arguments: [t_time_sk#113], [t_time_sk#113] +(160) CometProject +Input [3]: [t_time_sk#93, t_hour#94, t_minute#95] +Arguments: [t_time_sk#93], [t_time_sk#93] -(200) CometBroadcastExchange -Input [1]: [t_time_sk#113] -Arguments: [t_time_sk#113] +(161) CometBroadcastExchange +Input [1]: [t_time_sk#93] +Arguments: [t_time_sk#93] -(201) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#106, ss_store_sk#108] -Right output [1]: [t_time_sk#113] -Arguments: [ss_sold_time_sk#106], [t_time_sk#113], Inner +(162) CometBroadcastHashJoin +Left output [2]: [ss_sold_time_sk#88, ss_store_sk#90] +Right output [1]: [t_time_sk#93] +Arguments: [ss_sold_time_sk#88], [t_time_sk#93], Inner -(202) CometProject -Input [3]: [ss_sold_time_sk#106, ss_store_sk#108, t_time_sk#113] -Arguments: [ss_store_sk#108], [ss_store_sk#108] - -(203) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#116, s_store_name#117] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_name), EqualTo(s_store_name,ese), IsNotNull(s_store_sk)] -ReadSchema: struct - -(204) CometFilter -Input [2]: [s_store_sk#116, s_store_name#117] -Condition : ((isnotnull(s_store_name#117) AND (s_store_name#117 = ese)) AND isnotnull(s_store_sk#116)) - -(205) CometProject -Input [2]: [s_store_sk#116, s_store_name#117] -Arguments: [s_store_sk#116], [s_store_sk#116] +(163) CometProject +Input [3]: [ss_sold_time_sk#88, ss_store_sk#90, t_time_sk#93] +Arguments: [ss_store_sk#90], [ss_store_sk#90] -(206) CometBroadcastExchange -Input [1]: [s_store_sk#116] -Arguments: [s_store_sk#116] +(164) ReusedExchange [Reuses operator id: 19] +Output [1]: [s_store_sk#96] -(207) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#108] -Right output [1]: [s_store_sk#116] -Arguments: [ss_store_sk#108], [s_store_sk#116], Inner +(165) CometBroadcastHashJoin +Left output [1]: [ss_store_sk#90] +Right output [1]: [s_store_sk#96] +Arguments: [ss_store_sk#90], [s_store_sk#96], Inner -(208) CometProject -Input [2]: [ss_store_sk#108, s_store_sk#116] +(166) CometProject +Input [2]: [ss_store_sk#90, s_store_sk#96] -(209) CometHashAggregate +(167) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -(210) ColumnarToRow [codegen id : 14] -Input [1]: [count#118] +(168) ColumnarToRow [codegen id : 14] +Input [1]: [count#97] -(211) Exchange -Input [1]: [count#118] +(169) Exchange +Input [1]: [count#97] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=14] -(212) HashAggregate [codegen id : 15] -Input [1]: [count#118] +(170) HashAggregate [codegen id : 15] +Input [1]: [count#97] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#119] -Results [1]: [count(1)#119 AS h12_to_12_30#120] +Aggregate Attributes [1]: [count(1)#98] +Results [1]: [count(1)#98 AS h12_to_12_30#99] -(213) BroadcastExchange -Input [1]: [h12_to_12_30#120] +(171) BroadcastExchange +Input [1]: [h12_to_12_30#99] Arguments: IdentityBroadcastMode, [plan_id=15] -(214) BroadcastNestedLoopJoin [codegen id : 16] +(172) BroadcastNestedLoopJoin [codegen id : 16] Join type: Inner Join condition: None diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt index 998472a3c..b846d25d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt @@ -53,24 +53,18 @@ WholeStageCodegen (16) CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange #7 - CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #8 + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange #7 CometProject [t_time_sk] CometFilter [t_hour,t_minute,t_time_sk] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange #9 - CometProject [s_store_sk] - CometFilter [s_store_name,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] + ReusedExchange [s_store_sk] #4 InputAdapter - BroadcastExchange #10 + BroadcastExchange #8 WholeStageCodegen (5) HashAggregate [count] [count(1),h9_30_to_10,count] InputAdapter - Exchange #11 + Exchange #9 WholeStageCodegen (4) ColumnarToRow InputAdapter @@ -84,24 +78,18 @@ WholeStageCodegen (16) CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange #12 - CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #13 + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange #10 CometProject [t_time_sk] CometFilter [t_hour,t_minute,t_time_sk] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange #14 - CometProject [s_store_sk] - CometFilter [s_store_name,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] + ReusedExchange [s_store_sk] #4 InputAdapter - BroadcastExchange #15 + BroadcastExchange #11 WholeStageCodegen (7) HashAggregate [count] [count(1),h10_to_10_30,count] InputAdapter - Exchange #16 + Exchange #12 WholeStageCodegen (6) ColumnarToRow InputAdapter @@ -115,24 +103,18 @@ WholeStageCodegen (16) CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange #17 - CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #18 + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange #13 CometProject [t_time_sk] CometFilter [t_hour,t_minute,t_time_sk] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange #19 - CometProject [s_store_sk] - CometFilter [s_store_name,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] + ReusedExchange [s_store_sk] #4 InputAdapter - BroadcastExchange #20 + BroadcastExchange #14 WholeStageCodegen (9) HashAggregate [count] [count(1),h10_30_to_11,count] InputAdapter - Exchange #21 + Exchange #15 WholeStageCodegen (8) ColumnarToRow InputAdapter @@ -146,24 +128,18 @@ WholeStageCodegen (16) CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange #22 - CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #23 + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange #16 CometProject [t_time_sk] CometFilter [t_hour,t_minute,t_time_sk] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange #24 - CometProject [s_store_sk] - CometFilter [s_store_name,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] + ReusedExchange [s_store_sk] #4 InputAdapter - BroadcastExchange #25 + BroadcastExchange #17 WholeStageCodegen (11) HashAggregate [count] [count(1),h11_to_11_30,count] InputAdapter - Exchange #26 + Exchange #18 WholeStageCodegen (10) ColumnarToRow InputAdapter @@ -177,24 +153,18 @@ WholeStageCodegen (16) CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange #27 - CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #28 + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange #19 CometProject [t_time_sk] CometFilter [t_hour,t_minute,t_time_sk] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange #29 - CometProject [s_store_sk] - CometFilter [s_store_name,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] + ReusedExchange [s_store_sk] #4 InputAdapter - BroadcastExchange #30 + BroadcastExchange #20 WholeStageCodegen (13) HashAggregate [count] [count(1),h11_30_to_12,count] InputAdapter - Exchange #31 + Exchange #21 WholeStageCodegen (12) ColumnarToRow InputAdapter @@ -208,24 +178,18 @@ WholeStageCodegen (16) CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange #32 - CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #33 + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange #22 CometProject [t_time_sk] CometFilter [t_hour,t_minute,t_time_sk] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange #34 - CometProject [s_store_sk] - CometFilter [s_store_name,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] + ReusedExchange [s_store_sk] #4 InputAdapter - BroadcastExchange #35 + BroadcastExchange #23 WholeStageCodegen (15) HashAggregate [count] [count(1),h12_to_12_30,count] InputAdapter - Exchange #36 + Exchange #24 WholeStageCodegen (14) ColumnarToRow InputAdapter @@ -239,15 +203,9 @@ WholeStageCodegen (16) CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange #37 - CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #38 + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange #25 CometProject [t_time_sk] CometFilter [t_hour,t_minute,t_time_sk] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange #39 - CometProject [s_store_sk] - CometFilter [s_store_name,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] + ReusedExchange [s_store_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt index 65a9e9d91..0505401ea 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == -* Project (53) -+- * BroadcastNestedLoopJoin Inner BuildRight (52) +* Project (47) ++- * BroadcastNestedLoopJoin Inner BuildRight (46) :- * HashAggregate (25) : +- Exchange (24) : +- * ColumnarToRow (23) @@ -26,32 +26,26 @@ : +- CometProject (18) : +- CometFilter (17) : +- CometScan parquet spark_catalog.default.web_page (16) - +- BroadcastExchange (51) - +- * HashAggregate (50) - +- Exchange (49) - +- * ColumnarToRow (48) - +- CometHashAggregate (47) - +- CometProject (46) - +- CometBroadcastHashJoin (45) - :- CometProject (40) - : +- CometBroadcastHashJoin (39) - : :- CometProject (34) - : : +- CometBroadcastHashJoin (33) + +- BroadcastExchange (45) + +- * HashAggregate (44) + +- Exchange (43) + +- * ColumnarToRow (42) + +- CometHashAggregate (41) + +- CometProject (40) + +- CometBroadcastHashJoin (39) + :- CometProject (37) + : +- CometBroadcastHashJoin (36) + : :- CometProject (31) + : : +- CometBroadcastHashJoin (30) : : :- CometProject (28) : : : +- CometFilter (27) : : : +- CometScan parquet spark_catalog.default.web_sales (26) - : : +- CometBroadcastExchange (32) - : : +- CometProject (31) - : : +- CometFilter (30) - : : +- CometScan parquet spark_catalog.default.household_demographics (29) - : +- CometBroadcastExchange (38) - : +- CometProject (37) - : +- CometFilter (36) - : +- CometScan parquet spark_catalog.default.time_dim (35) - +- CometBroadcastExchange (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan parquet spark_catalog.default.web_page (41) + : : +- ReusedExchange (29) + : +- CometBroadcastExchange (35) + : +- CometProject (34) + : +- CometFilter (33) + : +- CometScan parquet spark_catalog.default.time_dim (32) + +- ReusedExchange (38) (1) Scan parquet spark_catalog.default.web_sales @@ -186,117 +180,85 @@ Condition : ((isnotnull(ws_ship_hdemo_sk#15) AND isnotnull(ws_sold_time_sk#14)) Input [4]: [ws_sold_time_sk#14, ws_ship_hdemo_sk#15, ws_web_page_sk#16, ws_sold_date_sk#17] Arguments: [ws_sold_time_sk#14, ws_ship_hdemo_sk#15, ws_web_page_sk#16], [ws_sold_time_sk#14, ws_ship_hdemo_sk#15, ws_web_page_sk#16] -(29) Scan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#18, hd_dep_count#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_dep_count), EqualTo(hd_dep_count,6), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [hd_demo_sk#18, hd_dep_count#19] -Condition : ((isnotnull(hd_dep_count#19) AND (hd_dep_count#19 = 6)) AND isnotnull(hd_demo_sk#18)) - -(31) CometProject -Input [2]: [hd_demo_sk#18, hd_dep_count#19] -Arguments: [hd_demo_sk#18], [hd_demo_sk#18] - -(32) CometBroadcastExchange -Input [1]: [hd_demo_sk#18] -Arguments: [hd_demo_sk#18] +(29) ReusedExchange [Reuses operator id: 7] +Output [1]: [hd_demo_sk#18] -(33) CometBroadcastHashJoin +(30) CometBroadcastHashJoin Left output [3]: [ws_sold_time_sk#14, ws_ship_hdemo_sk#15, ws_web_page_sk#16] Right output [1]: [hd_demo_sk#18] Arguments: [ws_ship_hdemo_sk#15], [hd_demo_sk#18], Inner -(34) CometProject +(31) CometProject Input [4]: [ws_sold_time_sk#14, ws_ship_hdemo_sk#15, ws_web_page_sk#16, hd_demo_sk#18] Arguments: [ws_sold_time_sk#14, ws_web_page_sk#16], [ws_sold_time_sk#14, ws_web_page_sk#16] -(35) Scan parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#20, t_hour#21] +(32) Scan parquet spark_catalog.default.time_dim +Output [2]: [t_time_sk#19, t_hour#20] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,19), LessThanOrEqual(t_hour,20), IsNotNull(t_time_sk)] ReadSchema: struct -(36) CometFilter -Input [2]: [t_time_sk#20, t_hour#21] -Condition : (((isnotnull(t_hour#21) AND (t_hour#21 >= 19)) AND (t_hour#21 <= 20)) AND isnotnull(t_time_sk#20)) +(33) CometFilter +Input [2]: [t_time_sk#19, t_hour#20] +Condition : (((isnotnull(t_hour#20) AND (t_hour#20 >= 19)) AND (t_hour#20 <= 20)) AND isnotnull(t_time_sk#19)) -(37) CometProject -Input [2]: [t_time_sk#20, t_hour#21] -Arguments: [t_time_sk#20], [t_time_sk#20] +(34) CometProject +Input [2]: [t_time_sk#19, t_hour#20] +Arguments: [t_time_sk#19], [t_time_sk#19] -(38) CometBroadcastExchange -Input [1]: [t_time_sk#20] -Arguments: [t_time_sk#20] +(35) CometBroadcastExchange +Input [1]: [t_time_sk#19] +Arguments: [t_time_sk#19] -(39) CometBroadcastHashJoin +(36) CometBroadcastHashJoin Left output [2]: [ws_sold_time_sk#14, ws_web_page_sk#16] -Right output [1]: [t_time_sk#20] -Arguments: [ws_sold_time_sk#14], [t_time_sk#20], Inner +Right output [1]: [t_time_sk#19] +Arguments: [ws_sold_time_sk#14], [t_time_sk#19], Inner -(40) CometProject -Input [3]: [ws_sold_time_sk#14, ws_web_page_sk#16, t_time_sk#20] +(37) CometProject +Input [3]: [ws_sold_time_sk#14, ws_web_page_sk#16, t_time_sk#19] Arguments: [ws_web_page_sk#16], [ws_web_page_sk#16] -(41) Scan parquet spark_catalog.default.web_page -Output [2]: [wp_web_page_sk#22, wp_char_count#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_char_count), GreaterThanOrEqual(wp_char_count,5000), LessThanOrEqual(wp_char_count,5200), IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [wp_web_page_sk#22, wp_char_count#23] -Condition : (((isnotnull(wp_char_count#23) AND (wp_char_count#23 >= 5000)) AND (wp_char_count#23 <= 5200)) AND isnotnull(wp_web_page_sk#22)) - -(43) CometProject -Input [2]: [wp_web_page_sk#22, wp_char_count#23] -Arguments: [wp_web_page_sk#22], [wp_web_page_sk#22] +(38) ReusedExchange [Reuses operator id: 19] +Output [1]: [wp_web_page_sk#21] -(44) CometBroadcastExchange -Input [1]: [wp_web_page_sk#22] -Arguments: [wp_web_page_sk#22] - -(45) CometBroadcastHashJoin +(39) CometBroadcastHashJoin Left output [1]: [ws_web_page_sk#16] -Right output [1]: [wp_web_page_sk#22] -Arguments: [ws_web_page_sk#16], [wp_web_page_sk#22], Inner +Right output [1]: [wp_web_page_sk#21] +Arguments: [ws_web_page_sk#16], [wp_web_page_sk#21], Inner -(46) CometProject -Input [2]: [ws_web_page_sk#16, wp_web_page_sk#22] +(40) CometProject +Input [2]: [ws_web_page_sk#16, wp_web_page_sk#21] -(47) CometHashAggregate +(41) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -(48) ColumnarToRow [codegen id : 2] -Input [1]: [count#24] +(42) ColumnarToRow [codegen id : 2] +Input [1]: [count#22] -(49) Exchange -Input [1]: [count#24] +(43) Exchange +Input [1]: [count#22] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] -(50) HashAggregate [codegen id : 3] -Input [1]: [count#24] +(44) HashAggregate [codegen id : 3] +Input [1]: [count#22] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#25] -Results [1]: [count(1)#25 AS pmc#26] +Aggregate Attributes [1]: [count(1)#23] +Results [1]: [count(1)#23 AS pmc#24] -(51) BroadcastExchange -Input [1]: [pmc#26] +(45) BroadcastExchange +Input [1]: [pmc#24] Arguments: IdentityBroadcastMode, [plan_id=3] -(52) BroadcastNestedLoopJoin [codegen id : 4] +(46) BroadcastNestedLoopJoin [codegen id : 4] Join type: Inner Join condition: None -(53) Project [codegen id : 4] -Output [1]: [(cast(amc#13 as decimal(15,4)) / cast(pmc#26 as decimal(15,4))) AS am_pm_ratio#27] -Input [2]: [amc#13, pmc#26] +(47) Project [codegen id : 4] +Output [1]: [(cast(amc#13 as decimal(15,4)) / cast(pmc#24 as decimal(15,4))) AS am_pm_ratio#25] +Input [2]: [amc#13, pmc#24] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt index 661802ba5..50c8494fb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt @@ -48,15 +48,9 @@ WholeStageCodegen (4) CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometBroadcastExchange #7 - CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange #8 + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange #7 CometProject [t_time_sk] CometFilter [t_hour,t_time_sk] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - CometBroadcastExchange #9 - CometProject [wp_web_page_sk] - CometFilter [wp_char_count,wp_web_page_sk] - CometScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] + ReusedExchange [wp_web_page_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt index 135eba3ce..aa1d42ddb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -* HashAggregate (29) -+- Exchange (28) - +- * HashAggregate (27) - +- * Project (26) - +- * SortMergeJoin FullOuter (25) +* HashAggregate (26) ++- Exchange (25) + +- * HashAggregate (24) + +- * Project (23) + +- * SortMergeJoin FullOuter (22) :- * Sort (12) : +- * HashAggregate (11) : +- Exchange (10) @@ -16,18 +16,15 @@ : +- CometProject (4) : +- CometFilter (3) : +- CometScan parquet spark_catalog.default.date_dim (2) - +- * Sort (24) - +- * HashAggregate (23) - +- Exchange (22) - +- * ColumnarToRow (21) - +- CometHashAggregate (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) + +- * Sort (21) + +- * HashAggregate (20) + +- Exchange (19) + +- * ColumnarToRow (18) + +- CometHashAggregate (17) + +- CometProject (16) + +- CometBroadcastHashJoin (15) :- CometScan parquet spark_catalog.default.catalog_sales (13) - +- CometBroadcastExchange (17) - +- CometProject (16) - +- CometFilter (15) - +- CometScan parquet spark_catalog.default.date_dim (14) + +- ReusedExchange (14) (1) Scan parquet spark_catalog.default.store_sales @@ -95,114 +92,98 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#11), dynamicpruningexpression(cs_sold_date_sk#11 IN dynamicpruning#12)] ReadSchema: struct -(14) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_month_seq#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [d_date_sk#13, d_month_seq#14] -Condition : (((isnotnull(d_month_seq#14) AND (d_month_seq#14 >= 1200)) AND (d_month_seq#14 <= 1211)) AND isnotnull(d_date_sk#13)) +(14) ReusedExchange [Reuses operator id: 5] +Output [1]: [d_date_sk#13] -(16) CometProject -Input [2]: [d_date_sk#13, d_month_seq#14] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(18) CometBroadcastHashJoin +(15) CometBroadcastHashJoin Left output [3]: [cs_bill_customer_sk#9, cs_item_sk#10, cs_sold_date_sk#11] Right output [1]: [d_date_sk#13] Arguments: [cs_sold_date_sk#11], [d_date_sk#13], Inner -(19) CometProject +(16) CometProject Input [4]: [cs_bill_customer_sk#9, cs_item_sk#10, cs_sold_date_sk#11, d_date_sk#13] Arguments: [cs_bill_customer_sk#9, cs_item_sk#10], [cs_bill_customer_sk#9, cs_item_sk#10] -(20) CometHashAggregate +(17) CometHashAggregate Input [2]: [cs_bill_customer_sk#9, cs_item_sk#10] Keys [2]: [cs_bill_customer_sk#9, cs_item_sk#10] Functions: [] -(21) ColumnarToRow [codegen id : 3] +(18) ColumnarToRow [codegen id : 3] Input [2]: [cs_bill_customer_sk#9, cs_item_sk#10] -(22) Exchange +(19) Exchange Input [2]: [cs_bill_customer_sk#9, cs_item_sk#10] Arguments: hashpartitioning(cs_bill_customer_sk#9, cs_item_sk#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(23) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 4] Input [2]: [cs_bill_customer_sk#9, cs_item_sk#10] Keys [2]: [cs_bill_customer_sk#9, cs_item_sk#10] Functions: [] Aggregate Attributes: [] -Results [2]: [cs_bill_customer_sk#9 AS customer_sk#15, cs_item_sk#10 AS item_sk#16] +Results [2]: [cs_bill_customer_sk#9 AS customer_sk#14, cs_item_sk#10 AS item_sk#15] -(24) Sort [codegen id : 4] -Input [2]: [customer_sk#15, item_sk#16] -Arguments: [customer_sk#15 ASC NULLS FIRST, item_sk#16 ASC NULLS FIRST], false, 0 +(21) Sort [codegen id : 4] +Input [2]: [customer_sk#14, item_sk#15] +Arguments: [customer_sk#14 ASC NULLS FIRST, item_sk#15 ASC NULLS FIRST], false, 0 -(25) SortMergeJoin [codegen id : 5] +(22) SortMergeJoin [codegen id : 5] Left keys [2]: [customer_sk#7, item_sk#8] -Right keys [2]: [customer_sk#15, item_sk#16] +Right keys [2]: [customer_sk#14, item_sk#15] Join type: FullOuter Join condition: None -(26) Project [codegen id : 5] -Output [2]: [customer_sk#7, customer_sk#15] -Input [4]: [customer_sk#7, item_sk#8, customer_sk#15, item_sk#16] +(23) Project [codegen id : 5] +Output [2]: [customer_sk#7, customer_sk#14] +Input [4]: [customer_sk#7, item_sk#8, customer_sk#14, item_sk#15] -(27) HashAggregate [codegen id : 5] -Input [2]: [customer_sk#7, customer_sk#15] +(24) HashAggregate [codegen id : 5] +Input [2]: [customer_sk#7, customer_sk#14] Keys: [] -Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#7) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#7) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#7) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)] -Aggregate Attributes [3]: [sum#17, sum#18, sum#19] -Results [3]: [sum#20, sum#21, sum#22] +Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#7) AND isnull(customer_sk#14)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END)] +Aggregate Attributes [3]: [sum#16, sum#17, sum#18] +Results [3]: [sum#19, sum#20, sum#21] -(28) Exchange -Input [3]: [sum#20, sum#21, sum#22] +(25) Exchange +Input [3]: [sum#19, sum#20, sum#21] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] -(29) HashAggregate [codegen id : 6] -Input [3]: [sum#20, sum#21, sum#22] +(26) HashAggregate [codegen id : 6] +Input [3]: [sum#19, sum#20, sum#21] Keys: [] -Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#7) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#7) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#7) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)] -Aggregate Attributes [3]: [sum(CASE WHEN (isnotnull(customer_sk#7) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END)#23, sum(CASE WHEN (isnull(customer_sk#7) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#24, sum(CASE WHEN (isnotnull(customer_sk#7) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#25] -Results [3]: [sum(CASE WHEN (isnotnull(customer_sk#7) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END)#23 AS store_only#26, sum(CASE WHEN (isnull(customer_sk#7) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#24 AS catalog_only#27, sum(CASE WHEN (isnotnull(customer_sk#7) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#25 AS store_and_catalog#28] +Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#7) AND isnull(customer_sk#14)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END)] +Aggregate Attributes [3]: [sum(CASE WHEN (isnotnull(customer_sk#7) AND isnull(customer_sk#14)) THEN 1 ELSE 0 END)#22, sum(CASE WHEN (isnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END)#23, sum(CASE WHEN (isnotnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END)#24] +Results [3]: [sum(CASE WHEN (isnotnull(customer_sk#7) AND isnull(customer_sk#14)) THEN 1 ELSE 0 END)#22 AS store_only#25, sum(CASE WHEN (isnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END)#23 AS catalog_only#26, sum(CASE WHEN (isnotnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END)#24 AS store_and_catalog#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (34) -+- * ColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan parquet spark_catalog.default.date_dim (30) +BroadcastExchange (31) ++- * ColumnarToRow (30) + +- CometProject (29) + +- CometFilter (28) + +- CometScan parquet spark_catalog.default.date_dim (27) -(30) Scan parquet spark_catalog.default.date_dim +(27) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#5, d_month_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(31) CometFilter +(28) CometFilter Input [2]: [d_date_sk#5, d_month_seq#6] Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) -(32) CometProject +(29) CometProject Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(33) ColumnarToRow [codegen id : 1] +(30) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(34) BroadcastExchange +(31) BroadcastExchange Input [1]: [d_date_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt index 7a3da59da..24e6dceef 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt @@ -45,7 +45,4 @@ WholeStageCodegen (6) CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #6 - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt index ac63431a0..a7d6a4a60 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt @@ -1,15 +1,15 @@ == Physical Plan == -TakeOrderedAndProject (45) -+- * HashAggregate (44) - +- Exchange (43) - +- * ColumnarToRow (42) - +- CometHashAggregate (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (35) - : +- CometBroadcastHashJoin (34) - : :- CometProject (29) - : : +- CometBroadcastHashJoin (28) +TakeOrderedAndProject (39) ++- * HashAggregate (38) + +- Exchange (37) + +- * ColumnarToRow (36) + +- CometHashAggregate (35) + +- CometProject (34) + +- CometBroadcastHashJoin (33) + :- CometProject (29) + : +- CometBroadcastHashJoin (28) + : :- CometProject (23) + : : +- CometBroadcastHashJoin (22) : : :- CometBroadcastHashJoin (11) : : : :- CometFilter (2) : : : : +- CometScan parquet spark_catalog.default.customer (1) @@ -21,29 +21,23 @@ TakeOrderedAndProject (45) : : : +- CometProject (6) : : : +- CometFilter (5) : : : +- CometScan parquet spark_catalog.default.date_dim (4) - : : +- CometBroadcastExchange (27) - : : +- CometUnion (26) - : : :- CometProject (18) - : : : +- CometBroadcastHashJoin (17) + : : +- CometBroadcastExchange (21) + : : +- CometUnion (20) + : : :- CometProject (15) + : : : +- CometBroadcastHashJoin (14) : : : :- CometScan parquet spark_catalog.default.web_sales (12) - : : : +- CometBroadcastExchange (16) - : : : +- CometProject (15) - : : : +- CometFilter (14) - : : : +- CometScan parquet spark_catalog.default.date_dim (13) - : : +- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- CometScan parquet spark_catalog.default.catalog_sales (19) - : : +- CometBroadcastExchange (23) - : : +- CometProject (22) - : : +- CometFilter (21) - : : +- CometScan parquet spark_catalog.default.date_dim (20) - : +- CometBroadcastExchange (33) - : +- CometProject (32) - : +- CometFilter (31) - : +- CometScan parquet spark_catalog.default.customer_address (30) - +- CometBroadcastExchange (38) - +- CometFilter (37) - +- CometScan parquet spark_catalog.default.customer_demographics (36) + : : : +- ReusedExchange (13) + : : +- CometProject (19) + : : +- CometBroadcastHashJoin (18) + : : :- CometScan parquet spark_catalog.default.catalog_sales (16) + : : +- ReusedExchange (17) + : +- CometBroadcastExchange (27) + : +- CometProject (26) + : +- CometFilter (25) + : +- CometScan parquet spark_catalog.default.customer_address (24) + +- CometBroadcastExchange (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.customer_demographics (30) (1) Scan parquet spark_catalog.default.customer @@ -108,195 +102,163 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] ReadSchema: struct -(13) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : (((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2002)) AND (d_moy#15 >= 4)) AND (d_moy#15 <= 7)) AND isnotnull(d_date_sk#13)) - -(15) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(16) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] +(13) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#13] -(17) CometBroadcastHashJoin +(14) CometBroadcastHashJoin Left output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] Right output [1]: [d_date_sk#13] Arguments: [ws_sold_date_sk#11], [d_date_sk#13], Inner -(18) CometProject +(15) CometProject Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] -Arguments: [customer_sk#16], [ws_bill_customer_sk#10 AS customer_sk#16] +Arguments: [customer_sk#14], [ws_bill_customer_sk#10 AS customer_sk#14] -(19) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] +(16) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] ReadSchema: struct -(20) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#20, d_year#21, d_moy#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [d_date_sk#20, d_year#21, d_moy#22] -Condition : (((((isnotnull(d_year#21) AND isnotnull(d_moy#22)) AND (d_year#21 = 2002)) AND (d_moy#22 >= 4)) AND (d_moy#22 <= 7)) AND isnotnull(d_date_sk#20)) - -(22) CometProject -Input [3]: [d_date_sk#20, d_year#21, d_moy#22] -Arguments: [d_date_sk#20], [d_date_sk#20] +(17) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#18] -(23) CometBroadcastExchange -Input [1]: [d_date_sk#20] -Arguments: [d_date_sk#20] +(18) CometBroadcastHashJoin +Left output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +Right output [1]: [d_date_sk#18] +Arguments: [cs_sold_date_sk#16], [d_date_sk#18], Inner -(24) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] -Right output [1]: [d_date_sk#20] -Arguments: [cs_sold_date_sk#18], [d_date_sk#20], Inner +(19) CometProject +Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#18] +Arguments: [customer_sk#19], [cs_ship_customer_sk#15 AS customer_sk#19] -(25) CometProject -Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#20] -Arguments: [customer_sk#23], [cs_ship_customer_sk#17 AS customer_sk#23] +(20) CometUnion +Child 0 Input [1]: [customer_sk#14] +Child 1 Input [1]: [customer_sk#19] -(26) CometUnion -Child 0 Input [1]: [customer_sk#16] -Child 1 Input [1]: [customer_sk#23] +(21) CometBroadcastExchange +Input [1]: [customer_sk#14] +Arguments: [customer_sk#14] -(27) CometBroadcastExchange -Input [1]: [customer_sk#16] -Arguments: [customer_sk#16] - -(28) CometBroadcastHashJoin +(22) CometBroadcastHashJoin Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [customer_sk#16] -Arguments: [c_customer_sk#1], [customer_sk#16], LeftSemi +Right output [1]: [customer_sk#14] +Arguments: [c_customer_sk#1], [customer_sk#14], LeftSemi -(29) CometProject +(23) CometProject Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Arguments: [c_current_cdemo_sk#2, c_current_addr_sk#3], [c_current_cdemo_sk#2, c_current_addr_sk#3] -(30) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#24, ca_county#25] +(24) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#20, ca_county#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Dona Ana County,Douglas County,Gaines County,Richland County,Walker County]), IsNotNull(ca_address_sk)] ReadSchema: struct -(31) CometFilter -Input [2]: [ca_address_sk#24, ca_county#25] -Condition : (ca_county#25 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#24)) +(25) CometFilter +Input [2]: [ca_address_sk#20, ca_county#21] +Condition : (ca_county#21 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#20)) -(32) CometProject -Input [2]: [ca_address_sk#24, ca_county#25] -Arguments: [ca_address_sk#24], [ca_address_sk#24] +(26) CometProject +Input [2]: [ca_address_sk#20, ca_county#21] +Arguments: [ca_address_sk#20], [ca_address_sk#20] -(33) CometBroadcastExchange -Input [1]: [ca_address_sk#24] -Arguments: [ca_address_sk#24] +(27) CometBroadcastExchange +Input [1]: [ca_address_sk#20] +Arguments: [ca_address_sk#20] -(34) CometBroadcastHashJoin +(28) CometBroadcastHashJoin Left output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [ca_address_sk#24] -Arguments: [c_current_addr_sk#3], [ca_address_sk#24], Inner +Right output [1]: [ca_address_sk#20] +Arguments: [c_current_addr_sk#3], [ca_address_sk#20], Inner -(35) CometProject -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#24] +(29) CometProject +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20] Arguments: [c_current_cdemo_sk#2], [c_current_cdemo_sk#2] -(36) Scan parquet spark_catalog.default.customer_demographics -Output [9]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] +(30) Scan parquet spark_catalog.default.customer_demographics +Output [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(37) CometFilter -Input [9]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] -Condition : isnotnull(cd_demo_sk#26) +(31) CometFilter +Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Condition : isnotnull(cd_demo_sk#22) -(38) CometBroadcastExchange -Input [9]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] -Arguments: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] +(32) CometBroadcastExchange +Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Arguments: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(39) CometBroadcastHashJoin +(33) CometBroadcastHashJoin Left output [1]: [c_current_cdemo_sk#2] -Right output [9]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] -Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#26], Inner +Right output [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#22], Inner -(40) CometProject -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] -Arguments: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34], [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] +(34) CometProject +Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Arguments: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30], [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(41) CometHashAggregate -Input [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] -Keys [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] +(35) CometHashAggregate +Input [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Keys [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [1]: [partial_count(1)] -(42) ColumnarToRow [codegen id : 1] -Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34, count#35] +(36) ColumnarToRow [codegen id : 1] +Input [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#31] -(43) Exchange -Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34, count#35] -Arguments: hashpartitioning(cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34, 5), ENSURE_REQUIREMENTS, [plan_id=1] +(37) Exchange +Input [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#31] +Arguments: hashpartitioning(cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(44) HashAggregate [codegen id : 2] -Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34, count#35] -Keys [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] +(38) HashAggregate [codegen id : 2] +Input [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#31] +Keys [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#36] -Results [14]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, count(1)#36 AS cnt1#37, cd_purchase_estimate#30, count(1)#36 AS cnt2#38, cd_credit_rating#31, count(1)#36 AS cnt3#39, cd_dep_count#32, count(1)#36 AS cnt4#40, cd_dep_employed_count#33, count(1)#36 AS cnt5#41, cd_dep_college_count#34, count(1)#36 AS cnt6#42] +Aggregate Attributes [1]: [count(1)#32] +Results [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, count(1)#32 AS cnt1#33, cd_purchase_estimate#26, count(1)#32 AS cnt2#34, cd_credit_rating#27, count(1)#32 AS cnt3#35, cd_dep_count#28, count(1)#32 AS cnt4#36, cd_dep_employed_count#29, count(1)#32 AS cnt5#37, cd_dep_college_count#30, count(1)#32 AS cnt6#38] -(45) TakeOrderedAndProject -Input [14]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cnt1#37, cd_purchase_estimate#30, cnt2#38, cd_credit_rating#31, cnt3#39, cd_dep_count#32, cnt4#40, cd_dep_employed_count#33, cnt5#41, cd_dep_college_count#34, cnt6#42] -Arguments: 100, [cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, cd_education_status#29 ASC NULLS FIRST, cd_purchase_estimate#30 ASC NULLS FIRST, cd_credit_rating#31 ASC NULLS FIRST, cd_dep_count#32 ASC NULLS FIRST, cd_dep_employed_count#33 ASC NULLS FIRST, cd_dep_college_count#34 ASC NULLS FIRST], [cd_gender#27, cd_marital_status#28, cd_education_status#29, cnt1#37, cd_purchase_estimate#30, cnt2#38, cd_credit_rating#31, cnt3#39, cd_dep_count#32, cnt4#40, cd_dep_employed_count#33, cnt5#41, cd_dep_college_count#34, cnt6#42] +(39) TakeOrderedAndProject +Input [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#33, cd_purchase_estimate#26, cnt2#34, cd_credit_rating#27, cnt3#35, cd_dep_count#28, cnt4#36, cd_dep_employed_count#29, cnt5#37, cd_dep_college_count#30, cnt6#38] +Arguments: 100, [cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_education_status#25 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#33, cd_purchase_estimate#26, cnt2#34, cd_credit_rating#27, cnt3#35, cd_dep_count#28, cnt4#36, cd_dep_employed_count#29, cnt5#37, cd_dep_college_count#30, cnt6#38] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (50) -+- * ColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan parquet spark_catalog.default.date_dim (46) +BroadcastExchange (44) ++- * ColumnarToRow (43) + +- CometProject (42) + +- CometFilter (41) + +- CometScan parquet spark_catalog.default.date_dim (40) -(46) Scan parquet spark_catalog.default.date_dim +(40) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#7, d_year#8, d_moy#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] ReadSchema: struct -(47) CometFilter +(41) CometFilter Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2002)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 7)) AND isnotnull(d_date_sk#7)) -(48) CometProject +(42) CometProject Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(49) ColumnarToRow [codegen id : 1] +(43) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(50) BroadcastExchange +(44) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 19 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt index 4f6b924a7..520edc88d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt @@ -38,22 +38,16 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #6 - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + ReusedExchange [d_date_sk] #4 CometProject [cs_ship_customer_sk] [customer_sk] CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #7 - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #8 + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange #6 CometProject [ca_address_sk] CometFilter [ca_county,ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - CometBroadcastExchange #9 + CometBroadcastExchange #7 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt index 5bc357e6a..66c307116 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (75) -+- * Project (74) - +- * BroadcastHashJoin Inner BuildRight (73) - :- * Project (55) - : +- * BroadcastHashJoin Inner BuildRight (54) +TakeOrderedAndProject (71) ++- * Project (70) + +- * BroadcastHashJoin Inner BuildRight (69) + :- * Project (53) + : +- * BroadcastHashJoin Inner BuildRight (52) : :- * BroadcastHashJoin Inner BuildRight (35) : : :- * Filter (17) : : : +- * HashAggregate (16) @@ -39,14 +39,14 @@ TakeOrderedAndProject (75) : : +- CometBroadcastExchange (27) : : +- CometFilter (26) : : +- CometScan parquet spark_catalog.default.date_dim (25) - : +- BroadcastExchange (53) - : +- * Filter (52) - : +- * HashAggregate (51) - : +- Exchange (50) - : +- * ColumnarToRow (49) - : +- CometHashAggregate (48) - : +- CometProject (47) - : +- CometBroadcastHashJoin (46) + : +- BroadcastExchange (51) + : +- * Filter (50) + : +- * HashAggregate (49) + : +- Exchange (48) + : +- * ColumnarToRow (47) + : +- CometHashAggregate (46) + : +- CometProject (45) + : +- CometBroadcastHashJoin (44) : :- CometProject (42) : : +- CometBroadcastHashJoin (41) : : :- CometFilter (37) @@ -54,26 +54,22 @@ TakeOrderedAndProject (75) : : +- CometBroadcastExchange (40) : : +- CometFilter (39) : : +- CometScan parquet spark_catalog.default.web_sales (38) - : +- CometBroadcastExchange (45) - : +- CometFilter (44) - : +- CometScan parquet spark_catalog.default.date_dim (43) - +- BroadcastExchange (72) - +- * HashAggregate (71) - +- Exchange (70) - +- * ColumnarToRow (69) - +- CometHashAggregate (68) - +- CometProject (67) - +- CometBroadcastHashJoin (66) - :- CometProject (62) - : +- CometBroadcastHashJoin (61) - : :- CometFilter (57) - : : +- CometScan parquet spark_catalog.default.customer (56) - : +- CometBroadcastExchange (60) - : +- CometFilter (59) - : +- CometScan parquet spark_catalog.default.web_sales (58) - +- CometBroadcastExchange (65) - +- CometFilter (64) - +- CometScan parquet spark_catalog.default.date_dim (63) + : +- ReusedExchange (43) + +- BroadcastExchange (68) + +- * HashAggregate (67) + +- Exchange (66) + +- * ColumnarToRow (65) + +- CometHashAggregate (64) + +- CometProject (63) + +- CometBroadcastHashJoin (62) + :- CometProject (60) + : +- CometBroadcastHashJoin (59) + : :- CometFilter (55) + : : +- CometScan parquet spark_catalog.default.customer (54) + : +- CometBroadcastExchange (58) + : +- CometFilter (57) + : +- CometScan parquet spark_catalog.default.web_sales (56) + +- ReusedExchange (61) (1) Scan parquet spark_catalog.default.customer @@ -284,79 +280,67 @@ Arguments: [c_customer_sk#41], [ws_bill_customer_sk#49], Inner Input [12]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] Arguments: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52], [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] -(43) Scan parquet spark_catalog.default.date_dim +(43) ReusedExchange [Reuses operator id: 10] Output [2]: [d_date_sk#54, d_year#55] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct -(44) CometFilter -Input [2]: [d_date_sk#54, d_year#55] -Condition : ((isnotnull(d_year#55) AND (d_year#55 = 2001)) AND isnotnull(d_date_sk#54)) - -(45) CometBroadcastExchange -Input [2]: [d_date_sk#54, d_year#55] -Arguments: [d_date_sk#54, d_year#55] - -(46) CometBroadcastHashJoin +(44) CometBroadcastHashJoin Left output [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] Right output [2]: [d_date_sk#54, d_year#55] Arguments: [ws_sold_date_sk#52], [d_date_sk#54], Inner -(47) CometProject +(45) CometProject Input [12]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52, d_date_sk#54, d_year#55] Arguments: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, d_year#55], [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, d_year#55] -(48) CometHashAggregate +(46) CometHashAggregate Input [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, d_year#55] Keys [8]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55] Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))] -(49) ColumnarToRow [codegen id : 4] +(47) ColumnarToRow [codegen id : 4] Input [9]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, sum#56] -(50) Exchange +(48) Exchange Input [9]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, sum#56] Arguments: hashpartitioning(c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(51) HashAggregate [codegen id : 5] +(49) HashAggregate [codegen id : 5] Input [9]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, sum#56] Keys [8]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55] Functions [1]: [sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))] Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))#57] Results [2]: [c_customer_id#42 AS customer_id#58, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))#57,18,2) AS year_total#59] -(52) Filter [codegen id : 5] +(50) Filter [codegen id : 5] Input [2]: [customer_id#58, year_total#59] Condition : (isnotnull(year_total#59) AND (year_total#59 > 0.00)) -(53) BroadcastExchange +(51) BroadcastExchange Input [2]: [customer_id#58, year_total#59] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(54) BroadcastHashJoin [codegen id : 8] +(52) BroadcastHashJoin [codegen id : 8] Left keys [1]: [customer_id#18] Right keys [1]: [customer_id#58] Join type: Inner Join condition: None -(55) Project [codegen id : 8] +(53) Project [codegen id : 8] Output [8]: [customer_id#18, year_total#19, customer_id#36, customer_first_name#37, customer_last_name#38, customer_email_address#39, year_total#40, year_total#59] Input [9]: [customer_id#18, year_total#19, customer_id#36, customer_first_name#37, customer_last_name#38, customer_email_address#39, year_total#40, customer_id#58, year_total#59] -(56) Scan parquet spark_catalog.default.customer +(54) Scan parquet spark_catalog.default.customer Output [8]: [c_customer_sk#60, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(57) CometFilter +(55) CometFilter Input [8]: [c_customer_sk#60, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67] Condition : (isnotnull(c_customer_sk#60) AND isnotnull(c_customer_id#61)) -(58) Scan parquet spark_catalog.default.web_sales +(56) Scan parquet spark_catalog.default.web_sales Output [4]: [ws_bill_customer_sk#68, ws_ext_discount_amt#69, ws_ext_list_price#70, ws_sold_date_sk#71] Batched: true Location: InMemoryFileIndex [] @@ -364,138 +348,126 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#71), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(59) CometFilter +(57) CometFilter Input [4]: [ws_bill_customer_sk#68, ws_ext_discount_amt#69, ws_ext_list_price#70, ws_sold_date_sk#71] Condition : isnotnull(ws_bill_customer_sk#68) -(60) CometBroadcastExchange +(58) CometBroadcastExchange Input [4]: [ws_bill_customer_sk#68, ws_ext_discount_amt#69, ws_ext_list_price#70, ws_sold_date_sk#71] Arguments: [ws_bill_customer_sk#68, ws_ext_discount_amt#69, ws_ext_list_price#70, ws_sold_date_sk#71] -(61) CometBroadcastHashJoin +(59) CometBroadcastHashJoin Left output [8]: [c_customer_sk#60, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67] Right output [4]: [ws_bill_customer_sk#68, ws_ext_discount_amt#69, ws_ext_list_price#70, ws_sold_date_sk#71] Arguments: [c_customer_sk#60], [ws_bill_customer_sk#68], Inner -(62) CometProject +(60) CometProject Input [12]: [c_customer_sk#60, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, ws_bill_customer_sk#68, ws_ext_discount_amt#69, ws_ext_list_price#70, ws_sold_date_sk#71] Arguments: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, ws_ext_discount_amt#69, ws_ext_list_price#70, ws_sold_date_sk#71], [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, ws_ext_discount_amt#69, ws_ext_list_price#70, ws_sold_date_sk#71] -(63) Scan parquet spark_catalog.default.date_dim +(61) ReusedExchange [Reuses operator id: 27] Output [2]: [d_date_sk#73, d_year#74] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct -(64) CometFilter -Input [2]: [d_date_sk#73, d_year#74] -Condition : ((isnotnull(d_year#74) AND (d_year#74 = 2002)) AND isnotnull(d_date_sk#73)) - -(65) CometBroadcastExchange -Input [2]: [d_date_sk#73, d_year#74] -Arguments: [d_date_sk#73, d_year#74] - -(66) CometBroadcastHashJoin +(62) CometBroadcastHashJoin Left output [10]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, ws_ext_discount_amt#69, ws_ext_list_price#70, ws_sold_date_sk#71] Right output [2]: [d_date_sk#73, d_year#74] Arguments: [ws_sold_date_sk#71], [d_date_sk#73], Inner -(67) CometProject +(63) CometProject Input [12]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, ws_ext_discount_amt#69, ws_ext_list_price#70, ws_sold_date_sk#71, d_date_sk#73, d_year#74] Arguments: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, ws_ext_discount_amt#69, ws_ext_list_price#70, d_year#74], [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, ws_ext_discount_amt#69, ws_ext_list_price#70, d_year#74] -(68) CometHashAggregate +(64) CometHashAggregate Input [10]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, ws_ext_discount_amt#69, ws_ext_list_price#70, d_year#74] Keys [8]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, d_year#74] Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#70 - ws_ext_discount_amt#69)))] -(69) ColumnarToRow [codegen id : 6] +(65) ColumnarToRow [codegen id : 6] Input [9]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, d_year#74, sum#75] -(70) Exchange +(66) Exchange Input [9]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, d_year#74, sum#75] Arguments: hashpartitioning(c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, d_year#74, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(71) HashAggregate [codegen id : 7] +(67) HashAggregate [codegen id : 7] Input [9]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, d_year#74, sum#75] Keys [8]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, d_year#74] Functions [1]: [sum(UnscaledValue((ws_ext_list_price#70 - ws_ext_discount_amt#69)))] Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#70 - ws_ext_discount_amt#69)))#57] Results [2]: [c_customer_id#61 AS customer_id#76, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#70 - ws_ext_discount_amt#69)))#57,18,2) AS year_total#77] -(72) BroadcastExchange +(68) BroadcastExchange Input [2]: [customer_id#76, year_total#77] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] -(73) BroadcastHashJoin [codegen id : 8] +(69) BroadcastHashJoin [codegen id : 8] Left keys [1]: [customer_id#18] Right keys [1]: [customer_id#76] Join type: Inner Join condition: (CASE WHEN (year_total#59 > 0.00) THEN (year_total#77 / year_total#59) ELSE 0E-20 END > CASE WHEN (year_total#19 > 0.00) THEN (year_total#40 / year_total#19) ELSE 0E-20 END) -(74) Project [codegen id : 8] +(70) Project [codegen id : 8] Output [4]: [customer_id#36, customer_first_name#37, customer_last_name#38, customer_email_address#39] Input [10]: [customer_id#18, year_total#19, customer_id#36, customer_first_name#37, customer_last_name#38, customer_email_address#39, year_total#40, year_total#59, customer_id#76, year_total#77] -(75) TakeOrderedAndProject +(71) TakeOrderedAndProject Input [4]: [customer_id#36, customer_first_name#37, customer_last_name#38, customer_email_address#39] Arguments: 100, [customer_id#36 ASC NULLS FIRST, customer_first_name#37 ASC NULLS FIRST, customer_last_name#38 ASC NULLS FIRST, customer_email_address#39 ASC NULLS FIRST], [customer_id#36, customer_first_name#37, customer_last_name#38, customer_email_address#39] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (79) -+- * ColumnarToRow (78) - +- CometFilter (77) - +- CometScan parquet spark_catalog.default.date_dim (76) +BroadcastExchange (75) ++- * ColumnarToRow (74) + +- CometFilter (73) + +- CometScan parquet spark_catalog.default.date_dim (72) -(76) Scan parquet spark_catalog.default.date_dim +(72) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#14, d_year#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(77) CometFilter +(73) CometFilter Input [2]: [d_date_sk#14, d_year#15] Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2001)) AND isnotnull(d_date_sk#14)) -(78) ColumnarToRow [codegen id : 1] +(74) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#14, d_year#15] -(79) BroadcastExchange +(75) BroadcastExchange Input [2]: [d_date_sk#14, d_year#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#31 IN dynamicpruning#32 -BroadcastExchange (83) -+- * ColumnarToRow (82) - +- CometFilter (81) - +- CometScan parquet spark_catalog.default.date_dim (80) +BroadcastExchange (79) ++- * ColumnarToRow (78) + +- CometFilter (77) + +- CometScan parquet spark_catalog.default.date_dim (76) -(80) Scan parquet spark_catalog.default.date_dim +(76) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#33, d_year#34] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(81) CometFilter +(77) CometFilter Input [2]: [d_date_sk#33, d_year#34] Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2002)) AND isnotnull(d_date_sk#33)) -(82) ColumnarToRow [codegen id : 1] +(78) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#33, d_year#34] -(83) BroadcastExchange +(79) BroadcastExchange Input [2]: [d_date_sk#33, d_year#34] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] Subquery:3 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#13 -Subquery:4 Hosting operator id = 58 Hosting Expression = ws_sold_date_sk#71 IN dynamicpruning#32 +Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#71 IN dynamicpruning#32 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt index b2070dbe4..ecc421bd5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt @@ -82,15 +82,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom CometFilter [ws_bill_customer_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #13 - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk,d_year] #4 InputAdapter - BroadcastExchange #14 + BroadcastExchange #13 WholeStageCodegen (7) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #15 + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 WholeStageCodegen (6) ColumnarToRow InputAdapter @@ -101,10 +99,8 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #16 + CometBroadcastExchange #15 CometFilter [ws_bill_customer_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #17 - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt index 30fcd9258..8df183576 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt @@ -1,32 +1,32 @@ == Physical Plan == -TakeOrderedAndProject (92) -+- * BroadcastHashJoin Inner BuildRight (91) - :- * Filter (74) - : +- * HashAggregate (73) - : +- Exchange (72) - : +- * HashAggregate (71) - : +- * Project (70) - : +- * BroadcastHashJoin Inner BuildRight (69) - : :- * Project (67) - : : +- * BroadcastHashJoin Inner BuildRight (66) - : : :- * BroadcastHashJoin LeftSemi BuildRight (59) +TakeOrderedAndProject (84) ++- * BroadcastHashJoin Inner BuildRight (83) + :- * Filter (66) + : +- * HashAggregate (65) + : +- Exchange (64) + : +- * HashAggregate (63) + : +- * Project (62) + : +- * BroadcastHashJoin Inner BuildRight (61) + : :- * Project (59) + : : +- * BroadcastHashJoin Inner BuildRight (58) + : : :- * BroadcastHashJoin LeftSemi BuildRight (51) : : : :- * ColumnarToRow (3) : : : : +- CometFilter (2) : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (58) - : : : +- * Project (57) - : : : +- * BroadcastHashJoin Inner BuildRight (56) + : : : +- BroadcastExchange (50) + : : : +- * Project (49) + : : : +- * BroadcastHashJoin Inner BuildRight (48) : : : :- * ColumnarToRow (6) : : : : +- CometFilter (5) : : : : +- CometScan parquet spark_catalog.default.item (4) - : : : +- BroadcastExchange (55) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (54) - : : : :- * HashAggregate (38) - : : : : +- Exchange (37) - : : : : +- * ColumnarToRow (36) - : : : : +- CometHashAggregate (35) - : : : : +- CometProject (34) - : : : : +- CometBroadcastHashJoin (33) + : : : +- BroadcastExchange (47) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) + : : : :- * HashAggregate (35) + : : : : +- Exchange (34) + : : : : +- * ColumnarToRow (33) + : : : : +- CometHashAggregate (32) + : : : : +- CometProject (31) + : : : : +- CometBroadcastHashJoin (30) : : : : :- CometProject (28) : : : : : +- CometBroadcastHashJoin (27) : : : : : :- CometFilter (8) @@ -49,48 +49,40 @@ TakeOrderedAndProject (92) : : : : : +- CometProject (20) : : : : : +- CometFilter (19) : : : : : +- CometScan parquet spark_catalog.default.date_dim (18) - : : : : +- CometBroadcastExchange (32) - : : : : +- CometProject (31) - : : : : +- CometFilter (30) - : : : : +- CometScan parquet spark_catalog.default.date_dim (29) - : : : +- BroadcastExchange (53) - : : : +- * ColumnarToRow (52) - : : : +- CometProject (51) - : : : +- CometBroadcastHashJoin (50) - : : : :- CometProject (45) - : : : : +- CometBroadcastHashJoin (44) - : : : : :- CometFilter (40) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (39) - : : : : +- CometBroadcastExchange (43) - : : : : +- CometFilter (42) - : : : : +- CometScan parquet spark_catalog.default.item (41) - : : : +- CometBroadcastExchange (49) - : : : +- CometProject (48) - : : : +- CometFilter (47) - : : : +- CometScan parquet spark_catalog.default.date_dim (46) - : : +- BroadcastExchange (65) - : : +- * BroadcastHashJoin LeftSemi BuildRight (64) - : : :- * ColumnarToRow (62) - : : : +- CometFilter (61) - : : : +- CometScan parquet spark_catalog.default.item (60) - : : +- ReusedExchange (63) - : +- ReusedExchange (68) - +- BroadcastExchange (90) - +- * Filter (89) - +- * HashAggregate (88) - +- Exchange (87) - +- * HashAggregate (86) - +- * Project (85) - +- * BroadcastHashJoin Inner BuildRight (84) - :- * Project (82) - : +- * BroadcastHashJoin Inner BuildRight (81) - : :- * BroadcastHashJoin LeftSemi BuildRight (79) - : : :- * ColumnarToRow (77) - : : : +- CometFilter (76) - : : : +- CometScan parquet spark_catalog.default.store_sales (75) - : : +- ReusedExchange (78) - : +- ReusedExchange (80) - +- ReusedExchange (83) + : : : : +- ReusedExchange (29) + : : : +- BroadcastExchange (45) + : : : +- * ColumnarToRow (44) + : : : +- CometProject (43) + : : : +- CometBroadcastHashJoin (42) + : : : :- CometProject (40) + : : : : +- CometBroadcastHashJoin (39) + : : : : :- CometFilter (37) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (36) + : : : : +- ReusedExchange (38) + : : : +- ReusedExchange (41) + : : +- BroadcastExchange (57) + : : +- * BroadcastHashJoin LeftSemi BuildRight (56) + : : :- * ColumnarToRow (54) + : : : +- CometFilter (53) + : : : +- CometScan parquet spark_catalog.default.item (52) + : : +- ReusedExchange (55) + : +- ReusedExchange (60) + +- BroadcastExchange (82) + +- * Filter (81) + +- * HashAggregate (80) + +- Exchange (79) + +- * HashAggregate (78) + +- * Project (77) + +- * BroadcastHashJoin Inner BuildRight (76) + :- * Project (74) + : +- * BroadcastHashJoin Inner BuildRight (73) + : :- * BroadcastHashJoin LeftSemi BuildRight (71) + : : :- * ColumnarToRow (69) + : : : +- CometFilter (68) + : : : +- CometScan parquet spark_catalog.default.store_sales (67) + : : +- ReusedExchange (70) + : +- ReusedExchange (72) + +- ReusedExchange (75) (1) Scan parquet spark_catalog.default.store_sales @@ -231,622 +223,521 @@ Arguments: [ss_item_sk#10], [i_item_sk#13], Inner Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -(29) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct +(29) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#26] -(30) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : (((isnotnull(d_year#27) AND (d_year#27 >= 1998)) AND (d_year#27 <= 2000)) AND isnotnull(d_date_sk#26)) - -(31) CometProject -Input [2]: [d_date_sk#26, d_year#27] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(32) CometBroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: [d_date_sk#26] - -(33) CometBroadcastHashJoin +(30) CometBroadcastHashJoin Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] Right output [1]: [d_date_sk#26] Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner -(34) CometProject +(31) CometProject Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] -Arguments: [brand_id#28, class_id#29, category_id#30], [i_brand_id#14 AS brand_id#28, i_class_id#15 AS class_id#29, i_category_id#16 AS category_id#30] +Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] -(35) CometHashAggregate -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +(32) CometHashAggregate +Input [3]: [brand_id#27, class_id#28, category_id#29] +Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] -(36) ColumnarToRow [codegen id : 1] -Input [3]: [brand_id#28, class_id#29, category_id#30] +(33) ColumnarToRow [codegen id : 1] +Input [3]: [brand_id#27, class_id#28, category_id#29] -(37) Exchange -Input [3]: [brand_id#28, class_id#29, category_id#30] -Arguments: hashpartitioning(brand_id#28, class_id#29, category_id#30, 5), ENSURE_REQUIREMENTS, [plan_id=1] +(34) Exchange +Input [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(38) HashAggregate [codegen id : 3] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +(35) HashAggregate [codegen id : 3] +Input [3]: [brand_id#27, class_id#28, category_id#29] +Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#27, class_id#28, category_id#29] -(39) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#31, ws_sold_date_sk#32] +(36) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#32), dynamicpruningexpression(ws_sold_date_sk#32 IN dynamicpruning#33)] +PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(40) CometFilter -Input [2]: [ws_item_sk#31, ws_sold_date_sk#32] -Condition : isnotnull(ws_item_sk#31) +(37) CometFilter +Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] +Condition : isnotnull(ws_item_sk#30) -(41) Scan parquet spark_catalog.default.item -Output [4]: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(42) CometFilter -Input [4]: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] -Condition : isnotnull(i_item_sk#34) - -(43) CometBroadcastExchange -Input [4]: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] -Arguments: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] - -(44) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#31, ws_sold_date_sk#32] -Right output [4]: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] -Arguments: [ws_item_sk#31], [i_item_sk#34], Inner - -(45) CometProject -Input [6]: [ws_item_sk#31, ws_sold_date_sk#32, i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] -Arguments: [ws_sold_date_sk#32, i_brand_id#35, i_class_id#36, i_category_id#37], [ws_sold_date_sk#32, i_brand_id#35, i_class_id#36, i_category_id#37] +(38) ReusedExchange [Reuses operator id: 15] +Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -(46) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#38, d_year#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(47) CometFilter -Input [2]: [d_date_sk#38, d_year#39] -Condition : (((isnotnull(d_year#39) AND (d_year#39 >= 1998)) AND (d_year#39 <= 2000)) AND isnotnull(d_date_sk#38)) +(39) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] +Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [ws_item_sk#30], [i_item_sk#33], Inner -(48) CometProject -Input [2]: [d_date_sk#38, d_year#39] -Arguments: [d_date_sk#38], [d_date_sk#38] +(40) CometProject +Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -(49) CometBroadcastExchange -Input [1]: [d_date_sk#38] -Arguments: [d_date_sk#38] +(41) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#37] -(50) CometBroadcastHashJoin -Left output [4]: [ws_sold_date_sk#32, i_brand_id#35, i_class_id#36, i_category_id#37] -Right output [1]: [d_date_sk#38] -Arguments: [ws_sold_date_sk#32], [d_date_sk#38], Inner +(42) CometBroadcastHashJoin +Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] +Right output [1]: [d_date_sk#37] +Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner -(51) CometProject -Input [5]: [ws_sold_date_sk#32, i_brand_id#35, i_class_id#36, i_category_id#37, d_date_sk#38] -Arguments: [i_brand_id#35, i_class_id#36, i_category_id#37], [i_brand_id#35, i_class_id#36, i_category_id#37] +(43) CometProject +Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] +Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] -(52) ColumnarToRow [codegen id : 2] -Input [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] +(44) ColumnarToRow [codegen id : 2] +Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -(53) BroadcastExchange -Input [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] +(45) BroadcastExchange +Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] -(54) BroadcastHashJoin [codegen id : 3] -Left keys [6]: [coalesce(brand_id#28, 0), isnull(brand_id#28), coalesce(class_id#29, 0), isnull(class_id#29), coalesce(category_id#30, 0), isnull(category_id#30)] -Right keys [6]: [coalesce(i_brand_id#35, 0), isnull(i_brand_id#35), coalesce(i_class_id#36, 0), isnull(i_class_id#36), coalesce(i_category_id#37, 0), isnull(i_category_id#37)] +(46) BroadcastHashJoin [codegen id : 3] +Left keys [6]: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)] +Right keys [6]: [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)] Join type: LeftSemi Join condition: None -(55) BroadcastExchange -Input [3]: [brand_id#28, class_id#29, category_id#30] +(47) BroadcastExchange +Input [3]: [brand_id#27, class_id#28, category_id#29] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=3] -(56) BroadcastHashJoin [codegen id : 4] +(48) BroadcastHashJoin [codegen id : 4] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#28, class_id#29, category_id#30] +Right keys [3]: [brand_id#27, class_id#28, category_id#29] Join type: Inner Join condition: None -(57) Project [codegen id : 4] -Output [1]: [i_item_sk#6 AS ss_item_sk#40] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#28, class_id#29, category_id#30] +(49) Project [codegen id : 4] +Output [1]: [i_item_sk#6 AS ss_item_sk#38] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] -(58) BroadcastExchange -Input [1]: [ss_item_sk#40] +(50) BroadcastExchange +Input [1]: [ss_item_sk#38] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(59) BroadcastHashJoin [codegen id : 11] +(51) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#40] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None -(60) Scan parquet spark_catalog.default.item -Output [4]: [i_item_sk#41, i_brand_id#42, i_class_id#43, i_category_id#44] +(52) Scan parquet spark_catalog.default.item +Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(61) CometFilter -Input [4]: [i_item_sk#41, i_brand_id#42, i_class_id#43, i_category_id#44] -Condition : (((isnotnull(i_item_sk#41) AND isnotnull(i_brand_id#42)) AND isnotnull(i_class_id#43)) AND isnotnull(i_category_id#44)) +(53) CometFilter +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Condition : (((isnotnull(i_item_sk#39) AND isnotnull(i_brand_id#40)) AND isnotnull(i_class_id#41)) AND isnotnull(i_category_id#42)) -(62) ColumnarToRow [codegen id : 9] -Input [4]: [i_item_sk#41, i_brand_id#42, i_class_id#43, i_category_id#44] +(54) ColumnarToRow [codegen id : 9] +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -(63) ReusedExchange [Reuses operator id: 58] -Output [1]: [ss_item_sk#40] +(55) ReusedExchange [Reuses operator id: 50] +Output [1]: [ss_item_sk#38] -(64) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [i_item_sk#41] -Right keys [1]: [ss_item_sk#40] +(56) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [i_item_sk#39] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None -(65) BroadcastExchange -Input [4]: [i_item_sk#41, i_brand_id#42, i_class_id#43, i_category_id#44] +(57) BroadcastExchange +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(66) BroadcastHashJoin [codegen id : 11] +(58) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#41] +Right keys [1]: [i_item_sk#39] Join type: Inner Join condition: None -(67) Project [codegen id : 11] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#42, i_class_id#43, i_category_id#44] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#41, i_brand_id#42, i_class_id#43, i_category_id#44] +(59) Project [codegen id : 11] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -(68) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#45] +(60) ReusedExchange [Reuses operator id: 106] +Output [1]: [d_date_sk#43] -(69) BroadcastHashJoin [codegen id : 11] +(61) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#45] +Right keys [1]: [d_date_sk#43] Join type: Inner Join condition: None -(70) Project [codegen id : 11] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#42, i_class_id#43, i_category_id#44] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#42, i_class_id#43, i_category_id#44, d_date_sk#45] +(62) Project [codegen id : 11] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -(71) HashAggregate [codegen id : 11] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#42, i_class_id#43, i_category_id#44] -Keys [3]: [i_brand_id#42, i_class_id#43, i_category_id#44] +(63) HashAggregate [codegen id : 11] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] +Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] -Aggregate Attributes [3]: [sum#46, isEmpty#47, count#48] -Results [6]: [i_brand_id#42, i_class_id#43, i_category_id#44, sum#49, isEmpty#50, count#51] +Aggregate Attributes [3]: [sum#44, isEmpty#45, count#46] +Results [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] -(72) Exchange -Input [6]: [i_brand_id#42, i_class_id#43, i_category_id#44, sum#49, isEmpty#50, count#51] -Arguments: hashpartitioning(i_brand_id#42, i_class_id#43, i_category_id#44, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(64) Exchange +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] +Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(73) HashAggregate [codegen id : 24] -Input [6]: [i_brand_id#42, i_class_id#43, i_category_id#44, sum#49, isEmpty#50, count#51] -Keys [3]: [i_brand_id#42, i_class_id#43, i_category_id#44] +(65) HashAggregate [codegen id : 24] +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] +Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#52, count(1)#53] -Results [6]: [store AS channel#54, i_brand_id#42, i_class_id#43, i_category_id#44, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#52 AS sales#55, count(1)#53 AS number_sales#56] +Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#50, count(1)#51] +Results [6]: [store AS channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#50 AS sales#53, count(1)#51 AS number_sales#54] -(74) Filter [codegen id : 24] -Input [6]: [channel#54, i_brand_id#42, i_class_id#43, i_category_id#44, sales#55, number_sales#56] -Condition : (isnotnull(sales#55) AND (cast(sales#55 as decimal(32,6)) > cast(Subquery scalar-subquery#57, [id=#58] as decimal(32,6)))) +(66) Filter [codegen id : 24] +Input [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sales#53, number_sales#54] +Condition : (isnotnull(sales#53) AND (cast(sales#53 as decimal(32,6)) > cast(Subquery scalar-subquery#55, [id=#56] as decimal(32,6)))) -(75) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#59, ss_quantity#60, ss_list_price#61, ss_sold_date_sk#62] +(67) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#62), dynamicpruningexpression(ss_sold_date_sk#62 IN dynamicpruning#63)] +PartitionFilters: [isnotnull(ss_sold_date_sk#60), dynamicpruningexpression(ss_sold_date_sk#60 IN dynamicpruning#61)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(76) CometFilter -Input [4]: [ss_item_sk#59, ss_quantity#60, ss_list_price#61, ss_sold_date_sk#62] -Condition : isnotnull(ss_item_sk#59) +(68) CometFilter +Input [4]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60] +Condition : isnotnull(ss_item_sk#57) -(77) ColumnarToRow [codegen id : 22] -Input [4]: [ss_item_sk#59, ss_quantity#60, ss_list_price#61, ss_sold_date_sk#62] +(69) ColumnarToRow [codegen id : 22] +Input [4]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60] -(78) ReusedExchange [Reuses operator id: 58] -Output [1]: [ss_item_sk#40] +(70) ReusedExchange [Reuses operator id: 50] +Output [1]: [ss_item_sk#38] -(79) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ss_item_sk#59] -Right keys [1]: [ss_item_sk#40] +(71) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [ss_item_sk#57] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None -(80) ReusedExchange [Reuses operator id: 65] -Output [4]: [i_item_sk#64, i_brand_id#65, i_class_id#66, i_category_id#67] +(72) ReusedExchange [Reuses operator id: 57] +Output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] -(81) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ss_item_sk#59] -Right keys [1]: [i_item_sk#64] +(73) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [ss_item_sk#57] +Right keys [1]: [i_item_sk#62] Join type: Inner Join condition: None -(82) Project [codegen id : 22] -Output [6]: [ss_quantity#60, ss_list_price#61, ss_sold_date_sk#62, i_brand_id#65, i_class_id#66, i_category_id#67] -Input [8]: [ss_item_sk#59, ss_quantity#60, ss_list_price#61, ss_sold_date_sk#62, i_item_sk#64, i_brand_id#65, i_class_id#66, i_category_id#67] +(74) Project [codegen id : 22] +Output [6]: [ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60, i_brand_id#63, i_class_id#64, i_category_id#65] +Input [8]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] -(83) ReusedExchange [Reuses operator id: 137] -Output [1]: [d_date_sk#68] +(75) ReusedExchange [Reuses operator id: 120] +Output [1]: [d_date_sk#66] -(84) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ss_sold_date_sk#62] -Right keys [1]: [d_date_sk#68] +(76) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [ss_sold_date_sk#60] +Right keys [1]: [d_date_sk#66] Join type: Inner Join condition: None -(85) Project [codegen id : 22] -Output [5]: [ss_quantity#60, ss_list_price#61, i_brand_id#65, i_class_id#66, i_category_id#67] -Input [7]: [ss_quantity#60, ss_list_price#61, ss_sold_date_sk#62, i_brand_id#65, i_class_id#66, i_category_id#67, d_date_sk#68] - -(86) HashAggregate [codegen id : 22] -Input [5]: [ss_quantity#60, ss_list_price#61, i_brand_id#65, i_class_id#66, i_category_id#67] -Keys [3]: [i_brand_id#65, i_class_id#66, i_category_id#67] -Functions [2]: [partial_sum((cast(ss_quantity#60 as decimal(10,0)) * ss_list_price#61)), partial_count(1)] -Aggregate Attributes [3]: [sum#69, isEmpty#70, count#71] -Results [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] - -(87) Exchange -Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] -Arguments: hashpartitioning(i_brand_id#65, i_class_id#66, i_category_id#67, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(88) HashAggregate [codegen id : 23] -Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] -Keys [3]: [i_brand_id#65, i_class_id#66, i_category_id#67] -Functions [2]: [sum((cast(ss_quantity#60 as decimal(10,0)) * ss_list_price#61)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#60 as decimal(10,0)) * ss_list_price#61))#75, count(1)#76] -Results [6]: [store AS channel#77, i_brand_id#65, i_class_id#66, i_category_id#67, sum((cast(ss_quantity#60 as decimal(10,0)) * ss_list_price#61))#75 AS sales#78, count(1)#76 AS number_sales#79] - -(89) Filter [codegen id : 23] -Input [6]: [channel#77, i_brand_id#65, i_class_id#66, i_category_id#67, sales#78, number_sales#79] -Condition : (isnotnull(sales#78) AND (cast(sales#78 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#57, [id=#58] as decimal(32,6)))) - -(90) BroadcastExchange -Input [6]: [channel#77, i_brand_id#65, i_class_id#66, i_category_id#67, sales#78, number_sales#79] +(77) Project [codegen id : 22] +Output [5]: [ss_quantity#58, ss_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65] +Input [7]: [ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60, i_brand_id#63, i_class_id#64, i_category_id#65, d_date_sk#66] + +(78) HashAggregate [codegen id : 22] +Input [5]: [ss_quantity#58, ss_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65] +Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] +Functions [2]: [partial_sum((cast(ss_quantity#58 as decimal(10,0)) * ss_list_price#59)), partial_count(1)] +Aggregate Attributes [3]: [sum#67, isEmpty#68, count#69] +Results [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] + +(79) Exchange +Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] +Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(80) HashAggregate [codegen id : 23] +Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] +Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] +Functions [2]: [sum((cast(ss_quantity#58 as decimal(10,0)) * ss_list_price#59)), count(1)] +Aggregate Attributes [2]: [sum((cast(ss_quantity#58 as decimal(10,0)) * ss_list_price#59))#73, count(1)#74] +Results [6]: [store AS channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sum((cast(ss_quantity#58 as decimal(10,0)) * ss_list_price#59))#73 AS sales#76, count(1)#74 AS number_sales#77] + +(81) Filter [codegen id : 23] +Input [6]: [channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] +Condition : (isnotnull(sales#76) AND (cast(sales#76 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#55, [id=#56] as decimal(32,6)))) + +(82) BroadcastExchange +Input [6]: [channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=8] -(91) BroadcastHashJoin [codegen id : 24] -Left keys [3]: [i_brand_id#42, i_class_id#43, i_category_id#44] -Right keys [3]: [i_brand_id#65, i_class_id#66, i_category_id#67] +(83) BroadcastHashJoin [codegen id : 24] +Left keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] +Right keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] Join type: Inner Join condition: None -(92) TakeOrderedAndProject -Input [12]: [channel#54, i_brand_id#42, i_class_id#43, i_category_id#44, sales#55, number_sales#56, channel#77, i_brand_id#65, i_class_id#66, i_category_id#67, sales#78, number_sales#79] -Arguments: 100, [i_brand_id#42 ASC NULLS FIRST, i_class_id#43 ASC NULLS FIRST, i_category_id#44 ASC NULLS FIRST], [channel#54, i_brand_id#42, i_class_id#43, i_category_id#44, sales#55, number_sales#56, channel#77, i_brand_id#65, i_class_id#66, i_category_id#67, sales#78, number_sales#79] +(84) TakeOrderedAndProject +Input [12]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sales#53, number_sales#54, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] +Arguments: 100, [i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sales#53, number_sales#54, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] ===== Subqueries ===== -Subquery:1 Hosting operator id = 74 Hosting Expression = Subquery scalar-subquery#57, [id=#58] -* HashAggregate (118) -+- Exchange (117) - +- * ColumnarToRow (116) - +- CometHashAggregate (115) - +- CometUnion (114) - :- CometProject (99) - : +- CometBroadcastHashJoin (98) - : :- CometScan parquet spark_catalog.default.store_sales (93) - : +- CometBroadcastExchange (97) - : +- CometProject (96) - : +- CometFilter (95) - : +- CometScan parquet spark_catalog.default.date_dim (94) - :- CometProject (106) - : +- CometBroadcastHashJoin (105) - : :- CometScan parquet spark_catalog.default.catalog_sales (100) - : +- CometBroadcastExchange (104) - : +- CometProject (103) - : +- CometFilter (102) - : +- CometScan parquet spark_catalog.default.date_dim (101) - +- CometProject (113) - +- CometBroadcastHashJoin (112) - :- CometScan parquet spark_catalog.default.web_sales (107) - +- CometBroadcastExchange (111) - +- CometProject (110) - +- CometFilter (109) - +- CometScan parquet spark_catalog.default.date_dim (108) - - -(93) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#80, ss_list_price#81, ss_sold_date_sk#82] +Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#55, [id=#56] +* HashAggregate (101) ++- Exchange (100) + +- * ColumnarToRow (99) + +- CometHashAggregate (98) + +- CometUnion (97) + :- CometProject (88) + : +- CometBroadcastHashJoin (87) + : :- CometScan parquet spark_catalog.default.store_sales (85) + : +- ReusedExchange (86) + :- CometProject (92) + : +- CometBroadcastHashJoin (91) + : :- CometScan parquet spark_catalog.default.catalog_sales (89) + : +- ReusedExchange (90) + +- CometProject (96) + +- CometBroadcastHashJoin (95) + :- CometScan parquet spark_catalog.default.web_sales (93) + +- ReusedExchange (94) + + +(85) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#82), dynamicpruningexpression(ss_sold_date_sk#82 IN dynamicpruning#83)] +PartitionFilters: [isnotnull(ss_sold_date_sk#80), dynamicpruningexpression(ss_sold_date_sk#80 IN dynamicpruning#81)] ReadSchema: struct -(94) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#84, d_year#85] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(95) CometFilter -Input [2]: [d_date_sk#84, d_year#85] -Condition : (((isnotnull(d_year#85) AND (d_year#85 >= 1998)) AND (d_year#85 <= 2000)) AND isnotnull(d_date_sk#84)) - -(96) CometProject -Input [2]: [d_date_sk#84, d_year#85] -Arguments: [d_date_sk#84], [d_date_sk#84] +(86) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#82] -(97) CometBroadcastExchange -Input [1]: [d_date_sk#84] -Arguments: [d_date_sk#84] +(87) CometBroadcastHashJoin +Left output [3]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80] +Right output [1]: [d_date_sk#82] +Arguments: [ss_sold_date_sk#80], [d_date_sk#82], Inner -(98) CometBroadcastHashJoin -Left output [3]: [ss_quantity#80, ss_list_price#81, ss_sold_date_sk#82] -Right output [1]: [d_date_sk#84] -Arguments: [ss_sold_date_sk#82], [d_date_sk#84], Inner +(88) CometProject +Input [4]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80, d_date_sk#82] +Arguments: [quantity#83, list_price#84], [ss_quantity#78 AS quantity#83, ss_list_price#79 AS list_price#84] -(99) CometProject -Input [4]: [ss_quantity#80, ss_list_price#81, ss_sold_date_sk#82, d_date_sk#84] -Arguments: [quantity#86, list_price#87], [ss_quantity#80 AS quantity#86, ss_list_price#81 AS list_price#87] - -(100) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#88, cs_list_price#89, cs_sold_date_sk#90] +(89) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#90), dynamicpruningexpression(cs_sold_date_sk#90 IN dynamicpruning#91)] +PartitionFilters: [isnotnull(cs_sold_date_sk#87), dynamicpruningexpression(cs_sold_date_sk#87 IN dynamicpruning#88)] ReadSchema: struct -(101) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#92, d_year#93] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(102) CometFilter -Input [2]: [d_date_sk#92, d_year#93] -Condition : (((isnotnull(d_year#93) AND (d_year#93 >= 1998)) AND (d_year#93 <= 2000)) AND isnotnull(d_date_sk#92)) +(90) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#89] -(103) CometProject -Input [2]: [d_date_sk#92, d_year#93] -Arguments: [d_date_sk#92], [d_date_sk#92] +(91) CometBroadcastHashJoin +Left output [3]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87] +Right output [1]: [d_date_sk#89] +Arguments: [cs_sold_date_sk#87], [d_date_sk#89], Inner -(104) CometBroadcastExchange -Input [1]: [d_date_sk#92] -Arguments: [d_date_sk#92] +(92) CometProject +Input [4]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87, d_date_sk#89] +Arguments: [quantity#90, list_price#91], [cs_quantity#85 AS quantity#90, cs_list_price#86 AS list_price#91] -(105) CometBroadcastHashJoin -Left output [3]: [cs_quantity#88, cs_list_price#89, cs_sold_date_sk#90] -Right output [1]: [d_date_sk#92] -Arguments: [cs_sold_date_sk#90], [d_date_sk#92], Inner - -(106) CometProject -Input [4]: [cs_quantity#88, cs_list_price#89, cs_sold_date_sk#90, d_date_sk#92] -Arguments: [quantity#94, list_price#95], [cs_quantity#88 AS quantity#94, cs_list_price#89 AS list_price#95] - -(107) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#96, ws_list_price#97, ws_sold_date_sk#98] +(93) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#98), dynamicpruningexpression(ws_sold_date_sk#98 IN dynamicpruning#99)] +PartitionFilters: [isnotnull(ws_sold_date_sk#94), dynamicpruningexpression(ws_sold_date_sk#94 IN dynamicpruning#95)] ReadSchema: struct -(108) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#100, d_year#101] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(109) CometFilter -Input [2]: [d_date_sk#100, d_year#101] -Condition : (((isnotnull(d_year#101) AND (d_year#101 >= 1998)) AND (d_year#101 <= 2000)) AND isnotnull(d_date_sk#100)) - -(110) CometProject -Input [2]: [d_date_sk#100, d_year#101] -Arguments: [d_date_sk#100], [d_date_sk#100] - -(111) CometBroadcastExchange -Input [1]: [d_date_sk#100] -Arguments: [d_date_sk#100] +(94) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#96] -(112) CometBroadcastHashJoin -Left output [3]: [ws_quantity#96, ws_list_price#97, ws_sold_date_sk#98] -Right output [1]: [d_date_sk#100] -Arguments: [ws_sold_date_sk#98], [d_date_sk#100], Inner +(95) CometBroadcastHashJoin +Left output [3]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] +Right output [1]: [d_date_sk#96] +Arguments: [ws_sold_date_sk#94], [d_date_sk#96], Inner -(113) CometProject -Input [4]: [ws_quantity#96, ws_list_price#97, ws_sold_date_sk#98, d_date_sk#100] -Arguments: [quantity#102, list_price#103], [ws_quantity#96 AS quantity#102, ws_list_price#97 AS list_price#103] +(96) CometProject +Input [4]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, d_date_sk#96] +Arguments: [quantity#97, list_price#98], [ws_quantity#92 AS quantity#97, ws_list_price#93 AS list_price#98] -(114) CometUnion -Child 0 Input [2]: [quantity#86, list_price#87] -Child 1 Input [2]: [quantity#94, list_price#95] -Child 2 Input [2]: [quantity#102, list_price#103] +(97) CometUnion +Child 0 Input [2]: [quantity#83, list_price#84] +Child 1 Input [2]: [quantity#90, list_price#91] +Child 2 Input [2]: [quantity#97, list_price#98] -(115) CometHashAggregate -Input [2]: [quantity#86, list_price#87] +(98) CometHashAggregate +Input [2]: [quantity#83, list_price#84] Keys: [] -Functions [1]: [partial_avg((cast(quantity#86 as decimal(10,0)) * list_price#87))] +Functions [1]: [partial_avg((cast(quantity#83 as decimal(10,0)) * list_price#84))] -(116) ColumnarToRow [codegen id : 1] -Input [2]: [sum#104, count#105] +(99) ColumnarToRow [codegen id : 1] +Input [2]: [sum#99, count#100] -(117) Exchange -Input [2]: [sum#104, count#105] +(100) Exchange +Input [2]: [sum#99, count#100] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] -(118) HashAggregate [codegen id : 2] -Input [2]: [sum#104, count#105] +(101) HashAggregate [codegen id : 2] +Input [2]: [sum#99, count#100] Keys: [] -Functions [1]: [avg((cast(quantity#86 as decimal(10,0)) * list_price#87))] -Aggregate Attributes [1]: [avg((cast(quantity#86 as decimal(10,0)) * list_price#87))#106] -Results [1]: [avg((cast(quantity#86 as decimal(10,0)) * list_price#87))#106 AS average_sales#107] +Functions [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))] +Aggregate Attributes [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))#101] +Results [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))#101 AS average_sales#102] -Subquery:2 Hosting operator id = 93 Hosting Expression = ss_sold_date_sk#82 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 85 Hosting Expression = ss_sold_date_sk#80 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 100 Hosting Expression = cs_sold_date_sk#90 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 89 Hosting Expression = cs_sold_date_sk#87 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#98 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 93 Hosting Expression = ws_sold_date_sk#94 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (123) -+- * ColumnarToRow (122) - +- CometProject (121) - +- CometFilter (120) - +- CometScan parquet spark_catalog.default.date_dim (119) +BroadcastExchange (106) ++- * ColumnarToRow (105) + +- CometProject (104) + +- CometFilter (103) + +- CometScan parquet spark_catalog.default.date_dim (102) -(119) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#45, d_week_seq#108] +(102) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#43, d_week_seq#103] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(120) CometFilter -Input [2]: [d_date_sk#45, d_week_seq#108] -Condition : ((isnotnull(d_week_seq#108) AND (d_week_seq#108 = Subquery scalar-subquery#109, [id=#110])) AND isnotnull(d_date_sk#45)) +(103) CometFilter +Input [2]: [d_date_sk#43, d_week_seq#103] +Condition : ((isnotnull(d_week_seq#103) AND (d_week_seq#103 = Subquery scalar-subquery#104, [id=#105])) AND isnotnull(d_date_sk#43)) -(121) CometProject -Input [2]: [d_date_sk#45, d_week_seq#108] -Arguments: [d_date_sk#45], [d_date_sk#45] +(104) CometProject +Input [2]: [d_date_sk#43, d_week_seq#103] +Arguments: [d_date_sk#43], [d_date_sk#43] -(122) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#45] +(105) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#43] -(123) BroadcastExchange -Input [1]: [d_date_sk#45] +(106) BroadcastExchange +Input [1]: [d_date_sk#43] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] -Subquery:6 Hosting operator id = 120 Hosting Expression = Subquery scalar-subquery#109, [id=#110] -* ColumnarToRow (127) -+- CometProject (126) - +- CometFilter (125) - +- CometScan parquet spark_catalog.default.date_dim (124) +Subquery:6 Hosting operator id = 103 Hosting Expression = Subquery scalar-subquery#104, [id=#105] +* ColumnarToRow (110) ++- CometProject (109) + +- CometFilter (108) + +- CometScan parquet spark_catalog.default.date_dim (107) -(124) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#111, d_year#112, d_moy#113, d_dom#114] +(107) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_week_seq#106, d_year#107, d_moy#108, d_dom#109] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(125) CometFilter -Input [4]: [d_week_seq#111, d_year#112, d_moy#113, d_dom#114] -Condition : (((((isnotnull(d_year#112) AND isnotnull(d_moy#113)) AND isnotnull(d_dom#114)) AND (d_year#112 = 1999)) AND (d_moy#113 = 12)) AND (d_dom#114 = 16)) +(108) CometFilter +Input [4]: [d_week_seq#106, d_year#107, d_moy#108, d_dom#109] +Condition : (((((isnotnull(d_year#107) AND isnotnull(d_moy#108)) AND isnotnull(d_dom#109)) AND (d_year#107 = 1999)) AND (d_moy#108 = 12)) AND (d_dom#109 = 16)) -(126) CometProject -Input [4]: [d_week_seq#111, d_year#112, d_moy#113, d_dom#114] -Arguments: [d_week_seq#111], [d_week_seq#111] +(109) CometProject +Input [4]: [d_week_seq#106, d_year#107, d_moy#108, d_dom#109] +Arguments: [d_week_seq#106], [d_week_seq#106] -(127) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#111] +(110) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#106] Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (132) -+- * ColumnarToRow (131) - +- CometProject (130) - +- CometFilter (129) - +- CometScan parquet spark_catalog.default.date_dim (128) +BroadcastExchange (115) ++- * ColumnarToRow (114) + +- CometProject (113) + +- CometFilter (112) + +- CometScan parquet spark_catalog.default.date_dim (111) -(128) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] +(111) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_year#110] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(129) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : (((isnotnull(d_year#27) AND (d_year#27 >= 1998)) AND (d_year#27 <= 2000)) AND isnotnull(d_date_sk#26)) +(112) CometFilter +Input [2]: [d_date_sk#26, d_year#110] +Condition : (((isnotnull(d_year#110) AND (d_year#110 >= 1998)) AND (d_year#110 <= 2000)) AND isnotnull(d_date_sk#26)) -(130) CometProject -Input [2]: [d_date_sk#26, d_year#27] +(113) CometProject +Input [2]: [d_date_sk#26, d_year#110] Arguments: [d_date_sk#26], [d_date_sk#26] -(131) ColumnarToRow [codegen id : 1] +(114) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] -(132) BroadcastExchange +(115) BroadcastExchange Input [1]: [d_date_sk#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] Subquery:8 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#32 IN dynamicpruning#12 +Subquery:9 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 -Subquery:10 Hosting operator id = 89 Hosting Expression = ReusedSubquery Subquery scalar-subquery#57, [id=#58] +Subquery:10 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#55, [id=#56] -Subquery:11 Hosting operator id = 75 Hosting Expression = ss_sold_date_sk#62 IN dynamicpruning#63 -BroadcastExchange (137) -+- * ColumnarToRow (136) - +- CometProject (135) - +- CometFilter (134) - +- CometScan parquet spark_catalog.default.date_dim (133) +Subquery:11 Hosting operator id = 67 Hosting Expression = ss_sold_date_sk#60 IN dynamicpruning#61 +BroadcastExchange (120) ++- * ColumnarToRow (119) + +- CometProject (118) + +- CometFilter (117) + +- CometScan parquet spark_catalog.default.date_dim (116) -(133) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#68, d_week_seq#115] +(116) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#66, d_week_seq#111] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(134) CometFilter -Input [2]: [d_date_sk#68, d_week_seq#115] -Condition : ((isnotnull(d_week_seq#115) AND (d_week_seq#115 = Subquery scalar-subquery#116, [id=#117])) AND isnotnull(d_date_sk#68)) +(117) CometFilter +Input [2]: [d_date_sk#66, d_week_seq#111] +Condition : ((isnotnull(d_week_seq#111) AND (d_week_seq#111 = Subquery scalar-subquery#112, [id=#113])) AND isnotnull(d_date_sk#66)) -(135) CometProject -Input [2]: [d_date_sk#68, d_week_seq#115] -Arguments: [d_date_sk#68], [d_date_sk#68] +(118) CometProject +Input [2]: [d_date_sk#66, d_week_seq#111] +Arguments: [d_date_sk#66], [d_date_sk#66] -(136) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#68] +(119) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#66] -(137) BroadcastExchange -Input [1]: [d_date_sk#68] +(120) BroadcastExchange +Input [1]: [d_date_sk#66] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -Subquery:12 Hosting operator id = 134 Hosting Expression = Subquery scalar-subquery#116, [id=#117] -* ColumnarToRow (141) -+- CometProject (140) - +- CometFilter (139) - +- CometScan parquet spark_catalog.default.date_dim (138) +Subquery:12 Hosting operator id = 117 Hosting Expression = Subquery scalar-subquery#112, [id=#113] +* ColumnarToRow (124) ++- CometProject (123) + +- CometFilter (122) + +- CometScan parquet spark_catalog.default.date_dim (121) -(138) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#118, d_year#119, d_moy#120, d_dom#121] +(121) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_week_seq#114, d_year#115, d_moy#116, d_dom#117] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(139) CometFilter -Input [4]: [d_week_seq#118, d_year#119, d_moy#120, d_dom#121] -Condition : (((((isnotnull(d_year#119) AND isnotnull(d_moy#120)) AND isnotnull(d_dom#121)) AND (d_year#119 = 1998)) AND (d_moy#120 = 12)) AND (d_dom#121 = 16)) +(122) CometFilter +Input [4]: [d_week_seq#114, d_year#115, d_moy#116, d_dom#117] +Condition : (((((isnotnull(d_year#115) AND isnotnull(d_moy#116)) AND isnotnull(d_dom#117)) AND (d_year#115 = 1998)) AND (d_moy#116 = 12)) AND (d_dom#117 = 16)) -(140) CometProject -Input [4]: [d_week_seq#118, d_year#119, d_moy#120, d_dom#121] -Arguments: [d_week_seq#118], [d_week_seq#118] +(123) CometProject +Input [4]: [d_week_seq#114, d_year#115, d_moy#116, d_dom#117] +Arguments: [d_week_seq#114], [d_week_seq#114] -(141) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#118] +(124) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#114] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt index e4517e1cf..799f74a36 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt @@ -6,7 +6,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (2) HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] InputAdapter - Exchange #16 + Exchange #13 WholeStageCodegen (1) ColumnarToRow InputAdapter @@ -16,26 +16,17 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange #17 - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #10 CometProject [cs_quantity,cs_list_price] [quantity,list_price] CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange #18 - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #10 CometProject [ws_quantity,ws_list_price] [quantity,list_price] CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange #19 - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #10 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #1 @@ -118,12 +109,9 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #11 - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #10 InputAdapter - BroadcastExchange #12 + BroadcastExchange #11 WholeStageCodegen (2) ColumnarToRow InputAdapter @@ -134,15 +122,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange #13 - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #14 - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + ReusedExchange [d_date_sk] #10 InputAdapter - BroadcastExchange #15 + BroadcastExchange #12 WholeStageCodegen (9) BroadcastHashJoin [i_item_sk,ss_item_sk] ColumnarToRow @@ -154,13 +137,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter - BroadcastExchange #20 + BroadcastExchange #14 WholeStageCodegen (23) Filter [sales] ReusedSubquery [average_sales] #4 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #21 + Exchange [i_brand_id,i_class_id,i_category_id] #15 WholeStageCodegen (22) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] @@ -173,7 +156,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 - BroadcastExchange #22 + BroadcastExchange #16 WholeStageCodegen (1) ColumnarToRow InputAdapter @@ -190,6 +173,6 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter ReusedExchange [ss_item_sk] #3 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 InputAdapter - ReusedExchange [d_date_sk] #22 + ReusedExchange [d_date_sk] #16 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt index d6f38b3bc..3b2aa8cbf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt @@ -1,39 +1,39 @@ == Physical Plan == -TakeOrderedAndProject (133) -+- * HashAggregate (132) - +- Exchange (131) - +- * HashAggregate (130) - +- Union (129) - :- * HashAggregate (108) - : +- Exchange (107) - : +- * HashAggregate (106) - : +- Union (105) - : :- * Filter (74) - : : +- * HashAggregate (73) - : : +- Exchange (72) - : : +- * HashAggregate (71) - : : +- * Project (70) - : : +- * BroadcastHashJoin Inner BuildRight (69) - : : :- * Project (67) - : : : +- * BroadcastHashJoin Inner BuildRight (66) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (59) +TakeOrderedAndProject (125) ++- * HashAggregate (124) + +- Exchange (123) + +- * HashAggregate (122) + +- Union (121) + :- * HashAggregate (100) + : +- Exchange (99) + : +- * HashAggregate (98) + : +- Union (97) + : :- * Filter (66) + : : +- * HashAggregate (65) + : : +- Exchange (64) + : : +- * HashAggregate (63) + : : +- * Project (62) + : : +- * BroadcastHashJoin Inner BuildRight (61) + : : :- * Project (59) + : : : +- * BroadcastHashJoin Inner BuildRight (58) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (51) : : : : :- * ColumnarToRow (3) : : : : : +- CometFilter (2) : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- BroadcastExchange (58) - : : : : +- * Project (57) - : : : : +- * BroadcastHashJoin Inner BuildRight (56) + : : : : +- BroadcastExchange (50) + : : : : +- * Project (49) + : : : : +- * BroadcastHashJoin Inner BuildRight (48) : : : : :- * ColumnarToRow (6) : : : : : +- CometFilter (5) : : : : : +- CometScan parquet spark_catalog.default.item (4) - : : : : +- BroadcastExchange (55) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (54) - : : : : :- * HashAggregate (38) - : : : : : +- Exchange (37) - : : : : : +- * ColumnarToRow (36) - : : : : : +- CometHashAggregate (35) - : : : : : +- CometProject (34) - : : : : : +- CometBroadcastHashJoin (33) + : : : : +- BroadcastExchange (47) + : : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) + : : : : :- * HashAggregate (35) + : : : : : +- Exchange (34) + : : : : : +- * ColumnarToRow (33) + : : : : : +- CometHashAggregate (32) + : : : : : +- CometProject (31) + : : : : : +- CometBroadcastHashJoin (30) : : : : : :- CometProject (28) : : : : : : +- CometBroadcastHashJoin (27) : : : : : : :- CometFilter (8) @@ -56,82 +56,74 @@ TakeOrderedAndProject (133) : : : : : : +- CometProject (20) : : : : : : +- CometFilter (19) : : : : : : +- CometScan parquet spark_catalog.default.date_dim (18) - : : : : : +- CometBroadcastExchange (32) - : : : : : +- CometProject (31) - : : : : : +- CometFilter (30) - : : : : : +- CometScan parquet spark_catalog.default.date_dim (29) - : : : : +- BroadcastExchange (53) - : : : : +- * ColumnarToRow (52) - : : : : +- CometProject (51) - : : : : +- CometBroadcastHashJoin (50) - : : : : :- CometProject (45) - : : : : : +- CometBroadcastHashJoin (44) - : : : : : :- CometFilter (40) - : : : : : : +- CometScan parquet spark_catalog.default.web_sales (39) - : : : : : +- CometBroadcastExchange (43) - : : : : : +- CometFilter (42) - : : : : : +- CometScan parquet spark_catalog.default.item (41) - : : : : +- CometBroadcastExchange (49) - : : : : +- CometProject (48) - : : : : +- CometFilter (47) - : : : : +- CometScan parquet spark_catalog.default.date_dim (46) - : : : +- BroadcastExchange (65) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (64) - : : : :- * ColumnarToRow (62) - : : : : +- CometFilter (61) - : : : : +- CometScan parquet spark_catalog.default.item (60) - : : : +- ReusedExchange (63) - : : +- ReusedExchange (68) - : :- * Filter (89) - : : +- * HashAggregate (88) - : : +- Exchange (87) - : : +- * HashAggregate (86) - : : +- * Project (85) - : : +- * BroadcastHashJoin Inner BuildRight (84) - : : :- * Project (82) - : : : +- * BroadcastHashJoin Inner BuildRight (81) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (79) - : : : : :- * ColumnarToRow (77) - : : : : : +- CometFilter (76) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (75) - : : : : +- ReusedExchange (78) - : : : +- ReusedExchange (80) - : : +- ReusedExchange (83) - : +- * Filter (104) - : +- * HashAggregate (103) - : +- Exchange (102) - : +- * HashAggregate (101) - : +- * Project (100) - : +- * BroadcastHashJoin Inner BuildRight (99) - : :- * Project (97) - : : +- * BroadcastHashJoin Inner BuildRight (96) - : : :- * BroadcastHashJoin LeftSemi BuildRight (94) - : : : :- * ColumnarToRow (92) - : : : : +- CometFilter (91) - : : : : +- CometScan parquet spark_catalog.default.web_sales (90) - : : : +- ReusedExchange (93) - : : +- ReusedExchange (95) - : +- ReusedExchange (98) - :- * HashAggregate (113) - : +- Exchange (112) - : +- * HashAggregate (111) - : +- * HashAggregate (110) - : +- ReusedExchange (109) - :- * HashAggregate (118) - : +- Exchange (117) - : +- * HashAggregate (116) - : +- * HashAggregate (115) - : +- ReusedExchange (114) - :- * HashAggregate (123) - : +- Exchange (122) - : +- * HashAggregate (121) - : +- * HashAggregate (120) - : +- ReusedExchange (119) - +- * HashAggregate (128) - +- Exchange (127) - +- * HashAggregate (126) - +- * HashAggregate (125) - +- ReusedExchange (124) + : : : : : +- ReusedExchange (29) + : : : : +- BroadcastExchange (45) + : : : : +- * ColumnarToRow (44) + : : : : +- CometProject (43) + : : : : +- CometBroadcastHashJoin (42) + : : : : :- CometProject (40) + : : : : : +- CometBroadcastHashJoin (39) + : : : : : :- CometFilter (37) + : : : : : : +- CometScan parquet spark_catalog.default.web_sales (36) + : : : : : +- ReusedExchange (38) + : : : : +- ReusedExchange (41) + : : : +- BroadcastExchange (57) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (56) + : : : :- * ColumnarToRow (54) + : : : : +- CometFilter (53) + : : : : +- CometScan parquet spark_catalog.default.item (52) + : : : +- ReusedExchange (55) + : : +- ReusedExchange (60) + : :- * Filter (81) + : : +- * HashAggregate (80) + : : +- Exchange (79) + : : +- * HashAggregate (78) + : : +- * Project (77) + : : +- * BroadcastHashJoin Inner BuildRight (76) + : : :- * Project (74) + : : : +- * BroadcastHashJoin Inner BuildRight (73) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (71) + : : : : :- * ColumnarToRow (69) + : : : : : +- CometFilter (68) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (67) + : : : : +- ReusedExchange (70) + : : : +- ReusedExchange (72) + : : +- ReusedExchange (75) + : +- * Filter (96) + : +- * HashAggregate (95) + : +- Exchange (94) + : +- * HashAggregate (93) + : +- * Project (92) + : +- * BroadcastHashJoin Inner BuildRight (91) + : :- * Project (89) + : : +- * BroadcastHashJoin Inner BuildRight (88) + : : :- * BroadcastHashJoin LeftSemi BuildRight (86) + : : : :- * ColumnarToRow (84) + : : : : +- CometFilter (83) + : : : : +- CometScan parquet spark_catalog.default.web_sales (82) + : : : +- ReusedExchange (85) + : : +- ReusedExchange (87) + : +- ReusedExchange (90) + :- * HashAggregate (105) + : +- Exchange (104) + : +- * HashAggregate (103) + : +- * HashAggregate (102) + : +- ReusedExchange (101) + :- * HashAggregate (110) + : +- Exchange (109) + : +- * HashAggregate (108) + : +- * HashAggregate (107) + : +- ReusedExchange (106) + :- * HashAggregate (115) + : +- Exchange (114) + : +- * HashAggregate (113) + : +- * HashAggregate (112) + : +- ReusedExchange (111) + +- * HashAggregate (120) + +- Exchange (119) + +- * HashAggregate (118) + +- * HashAggregate (117) + +- ReusedExchange (116) (1) Scan parquet spark_catalog.default.store_sales @@ -272,790 +264,708 @@ Arguments: [ss_item_sk#10], [i_item_sk#13], Inner Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -(29) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : (((isnotnull(d_year#27) AND (d_year#27 >= 1999)) AND (d_year#27 <= 2001)) AND isnotnull(d_date_sk#26)) +(29) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#26] -(31) CometProject -Input [2]: [d_date_sk#26, d_year#27] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(32) CometBroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: [d_date_sk#26] - -(33) CometBroadcastHashJoin +(30) CometBroadcastHashJoin Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] Right output [1]: [d_date_sk#26] Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner -(34) CometProject +(31) CometProject Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] -Arguments: [brand_id#28, class_id#29, category_id#30], [i_brand_id#14 AS brand_id#28, i_class_id#15 AS class_id#29, i_category_id#16 AS category_id#30] +Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] -(35) CometHashAggregate -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +(32) CometHashAggregate +Input [3]: [brand_id#27, class_id#28, category_id#29] +Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] -(36) ColumnarToRow [codegen id : 1] -Input [3]: [brand_id#28, class_id#29, category_id#30] +(33) ColumnarToRow [codegen id : 1] +Input [3]: [brand_id#27, class_id#28, category_id#29] -(37) Exchange -Input [3]: [brand_id#28, class_id#29, category_id#30] -Arguments: hashpartitioning(brand_id#28, class_id#29, category_id#30, 5), ENSURE_REQUIREMENTS, [plan_id=1] +(34) Exchange +Input [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(38) HashAggregate [codegen id : 3] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +(35) HashAggregate [codegen id : 3] +Input [3]: [brand_id#27, class_id#28, category_id#29] +Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#27, class_id#28, category_id#29] -(39) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#31, ws_sold_date_sk#32] +(36) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#32), dynamicpruningexpression(ws_sold_date_sk#32 IN dynamicpruning#33)] +PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(40) CometFilter -Input [2]: [ws_item_sk#31, ws_sold_date_sk#32] -Condition : isnotnull(ws_item_sk#31) +(37) CometFilter +Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] +Condition : isnotnull(ws_item_sk#30) -(41) Scan parquet spark_catalog.default.item -Output [4]: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(42) CometFilter -Input [4]: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] -Condition : isnotnull(i_item_sk#34) - -(43) CometBroadcastExchange -Input [4]: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] -Arguments: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] - -(44) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#31, ws_sold_date_sk#32] -Right output [4]: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] -Arguments: [ws_item_sk#31], [i_item_sk#34], Inner - -(45) CometProject -Input [6]: [ws_item_sk#31, ws_sold_date_sk#32, i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] -Arguments: [ws_sold_date_sk#32, i_brand_id#35, i_class_id#36, i_category_id#37], [ws_sold_date_sk#32, i_brand_id#35, i_class_id#36, i_category_id#37] - -(46) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#38, d_year#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct +(38) ReusedExchange [Reuses operator id: 15] +Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -(47) CometFilter -Input [2]: [d_date_sk#38, d_year#39] -Condition : (((isnotnull(d_year#39) AND (d_year#39 >= 1999)) AND (d_year#39 <= 2001)) AND isnotnull(d_date_sk#38)) +(39) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] +Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [ws_item_sk#30], [i_item_sk#33], Inner -(48) CometProject -Input [2]: [d_date_sk#38, d_year#39] -Arguments: [d_date_sk#38], [d_date_sk#38] +(40) CometProject +Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -(49) CometBroadcastExchange -Input [1]: [d_date_sk#38] -Arguments: [d_date_sk#38] +(41) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#37] -(50) CometBroadcastHashJoin -Left output [4]: [ws_sold_date_sk#32, i_brand_id#35, i_class_id#36, i_category_id#37] -Right output [1]: [d_date_sk#38] -Arguments: [ws_sold_date_sk#32], [d_date_sk#38], Inner +(42) CometBroadcastHashJoin +Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] +Right output [1]: [d_date_sk#37] +Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner -(51) CometProject -Input [5]: [ws_sold_date_sk#32, i_brand_id#35, i_class_id#36, i_category_id#37, d_date_sk#38] -Arguments: [i_brand_id#35, i_class_id#36, i_category_id#37], [i_brand_id#35, i_class_id#36, i_category_id#37] +(43) CometProject +Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] +Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] -(52) ColumnarToRow [codegen id : 2] -Input [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] +(44) ColumnarToRow [codegen id : 2] +Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -(53) BroadcastExchange -Input [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] +(45) BroadcastExchange +Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] -(54) BroadcastHashJoin [codegen id : 3] -Left keys [6]: [coalesce(brand_id#28, 0), isnull(brand_id#28), coalesce(class_id#29, 0), isnull(class_id#29), coalesce(category_id#30, 0), isnull(category_id#30)] -Right keys [6]: [coalesce(i_brand_id#35, 0), isnull(i_brand_id#35), coalesce(i_class_id#36, 0), isnull(i_class_id#36), coalesce(i_category_id#37, 0), isnull(i_category_id#37)] +(46) BroadcastHashJoin [codegen id : 3] +Left keys [6]: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)] +Right keys [6]: [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)] Join type: LeftSemi Join condition: None -(55) BroadcastExchange -Input [3]: [brand_id#28, class_id#29, category_id#30] +(47) BroadcastExchange +Input [3]: [brand_id#27, class_id#28, category_id#29] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=3] -(56) BroadcastHashJoin [codegen id : 4] +(48) BroadcastHashJoin [codegen id : 4] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#28, class_id#29, category_id#30] +Right keys [3]: [brand_id#27, class_id#28, category_id#29] Join type: Inner Join condition: None -(57) Project [codegen id : 4] -Output [1]: [i_item_sk#6 AS ss_item_sk#40] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#28, class_id#29, category_id#30] +(49) Project [codegen id : 4] +Output [1]: [i_item_sk#6 AS ss_item_sk#38] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] -(58) BroadcastExchange -Input [1]: [ss_item_sk#40] +(50) BroadcastExchange +Input [1]: [ss_item_sk#38] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(59) BroadcastHashJoin [codegen id : 11] +(51) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#40] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None -(60) Scan parquet spark_catalog.default.item -Output [4]: [i_item_sk#41, i_brand_id#42, i_class_id#43, i_category_id#44] +(52) Scan parquet spark_catalog.default.item +Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(61) CometFilter -Input [4]: [i_item_sk#41, i_brand_id#42, i_class_id#43, i_category_id#44] -Condition : isnotnull(i_item_sk#41) +(53) CometFilter +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Condition : isnotnull(i_item_sk#39) -(62) ColumnarToRow [codegen id : 9] -Input [4]: [i_item_sk#41, i_brand_id#42, i_class_id#43, i_category_id#44] +(54) ColumnarToRow [codegen id : 9] +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -(63) ReusedExchange [Reuses operator id: 58] -Output [1]: [ss_item_sk#40] +(55) ReusedExchange [Reuses operator id: 50] +Output [1]: [ss_item_sk#38] -(64) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [i_item_sk#41] -Right keys [1]: [ss_item_sk#40] +(56) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [i_item_sk#39] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None -(65) BroadcastExchange -Input [4]: [i_item_sk#41, i_brand_id#42, i_class_id#43, i_category_id#44] +(57) BroadcastExchange +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(66) BroadcastHashJoin [codegen id : 11] +(58) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#41] +Right keys [1]: [i_item_sk#39] Join type: Inner Join condition: None -(67) Project [codegen id : 11] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#42, i_class_id#43, i_category_id#44] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#41, i_brand_id#42, i_class_id#43, i_category_id#44] +(59) Project [codegen id : 11] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -(68) ReusedExchange [Reuses operator id: 169] -Output [1]: [d_date_sk#45] +(60) ReusedExchange [Reuses operator id: 155] +Output [1]: [d_date_sk#43] -(69) BroadcastHashJoin [codegen id : 11] +(61) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#45] +Right keys [1]: [d_date_sk#43] Join type: Inner Join condition: None -(70) Project [codegen id : 11] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#42, i_class_id#43, i_category_id#44] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#42, i_class_id#43, i_category_id#44, d_date_sk#45] +(62) Project [codegen id : 11] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -(71) HashAggregate [codegen id : 11] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#42, i_class_id#43, i_category_id#44] -Keys [3]: [i_brand_id#42, i_class_id#43, i_category_id#44] +(63) HashAggregate [codegen id : 11] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] +Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] -Aggregate Attributes [3]: [sum#46, isEmpty#47, count#48] -Results [6]: [i_brand_id#42, i_class_id#43, i_category_id#44, sum#49, isEmpty#50, count#51] +Aggregate Attributes [3]: [sum#44, isEmpty#45, count#46] +Results [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] -(72) Exchange -Input [6]: [i_brand_id#42, i_class_id#43, i_category_id#44, sum#49, isEmpty#50, count#51] -Arguments: hashpartitioning(i_brand_id#42, i_class_id#43, i_category_id#44, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(64) Exchange +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] +Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(73) HashAggregate [codegen id : 12] -Input [6]: [i_brand_id#42, i_class_id#43, i_category_id#44, sum#49, isEmpty#50, count#51] -Keys [3]: [i_brand_id#42, i_class_id#43, i_category_id#44] +(65) HashAggregate [codegen id : 12] +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] +Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#52, count(1)#53] -Results [6]: [store AS channel#54, i_brand_id#42, i_class_id#43, i_category_id#44, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#52 AS sales#55, count(1)#53 AS number_sales#56] +Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#50, count(1)#51] +Results [6]: [store AS channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#50 AS sales#53, count(1)#51 AS number_sales#54] -(74) Filter [codegen id : 12] -Input [6]: [channel#54, i_brand_id#42, i_class_id#43, i_category_id#44, sales#55, number_sales#56] -Condition : (isnotnull(sales#55) AND (cast(sales#55 as decimal(32,6)) > cast(Subquery scalar-subquery#57, [id=#58] as decimal(32,6)))) +(66) Filter [codegen id : 12] +Input [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sales#53, number_sales#54] +Condition : (isnotnull(sales#53) AND (cast(sales#53 as decimal(32,6)) > cast(Subquery scalar-subquery#55, [id=#56] as decimal(32,6)))) -(75) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#59, cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62] +(67) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#62), dynamicpruningexpression(cs_sold_date_sk#62 IN dynamicpruning#63)] +PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_sold_date_sk#60 IN dynamicpruning#61)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(76) CometFilter -Input [4]: [cs_item_sk#59, cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62] -Condition : isnotnull(cs_item_sk#59) +(68) CometFilter +Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] +Condition : isnotnull(cs_item_sk#57) -(77) ColumnarToRow [codegen id : 23] -Input [4]: [cs_item_sk#59, cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62] +(69) ColumnarToRow [codegen id : 23] +Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -(78) ReusedExchange [Reuses operator id: 58] -Output [1]: [ss_item_sk#40] +(70) ReusedExchange [Reuses operator id: 50] +Output [1]: [ss_item_sk#38] -(79) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_item_sk#59] -Right keys [1]: [ss_item_sk#40] +(71) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [cs_item_sk#57] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None -(80) ReusedExchange [Reuses operator id: 65] -Output [4]: [i_item_sk#64, i_brand_id#65, i_class_id#66, i_category_id#67] +(72) ReusedExchange [Reuses operator id: 57] +Output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] -(81) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_item_sk#59] -Right keys [1]: [i_item_sk#64] +(73) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [cs_item_sk#57] +Right keys [1]: [i_item_sk#62] Join type: Inner Join condition: None -(82) Project [codegen id : 23] -Output [6]: [cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62, i_brand_id#65, i_class_id#66, i_category_id#67] -Input [8]: [cs_item_sk#59, cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62, i_item_sk#64, i_brand_id#65, i_class_id#66, i_category_id#67] +(74) Project [codegen id : 23] +Output [6]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#63, i_class_id#64, i_category_id#65] +Input [8]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] -(83) ReusedExchange [Reuses operator id: 169] -Output [1]: [d_date_sk#68] +(75) ReusedExchange [Reuses operator id: 155] +Output [1]: [d_date_sk#66] -(84) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_sold_date_sk#62] -Right keys [1]: [d_date_sk#68] +(76) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [cs_sold_date_sk#60] +Right keys [1]: [d_date_sk#66] Join type: Inner Join condition: None -(85) Project [codegen id : 23] -Output [5]: [cs_quantity#60, cs_list_price#61, i_brand_id#65, i_class_id#66, i_category_id#67] -Input [7]: [cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62, i_brand_id#65, i_class_id#66, i_category_id#67, d_date_sk#68] - -(86) HashAggregate [codegen id : 23] -Input [5]: [cs_quantity#60, cs_list_price#61, i_brand_id#65, i_class_id#66, i_category_id#67] -Keys [3]: [i_brand_id#65, i_class_id#66, i_category_id#67] -Functions [2]: [partial_sum((cast(cs_quantity#60 as decimal(10,0)) * cs_list_price#61)), partial_count(1)] -Aggregate Attributes [3]: [sum#69, isEmpty#70, count#71] -Results [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] - -(87) Exchange -Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] -Arguments: hashpartitioning(i_brand_id#65, i_class_id#66, i_category_id#67, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(88) HashAggregate [codegen id : 24] -Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] -Keys [3]: [i_brand_id#65, i_class_id#66, i_category_id#67] -Functions [2]: [sum((cast(cs_quantity#60 as decimal(10,0)) * cs_list_price#61)), count(1)] -Aggregate Attributes [2]: [sum((cast(cs_quantity#60 as decimal(10,0)) * cs_list_price#61))#75, count(1)#76] -Results [6]: [catalog AS channel#77, i_brand_id#65, i_class_id#66, i_category_id#67, sum((cast(cs_quantity#60 as decimal(10,0)) * cs_list_price#61))#75 AS sales#78, count(1)#76 AS number_sales#79] - -(89) Filter [codegen id : 24] -Input [6]: [channel#77, i_brand_id#65, i_class_id#66, i_category_id#67, sales#78, number_sales#79] -Condition : (isnotnull(sales#78) AND (cast(sales#78 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#57, [id=#58] as decimal(32,6)))) - -(90) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#80, ws_quantity#81, ws_list_price#82, ws_sold_date_sk#83] +(77) Project [codegen id : 23] +Output [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65] +Input [7]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#63, i_class_id#64, i_category_id#65, d_date_sk#66] + +(78) HashAggregate [codegen id : 23] +Input [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65] +Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] +Functions [2]: [partial_sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), partial_count(1)] +Aggregate Attributes [3]: [sum#67, isEmpty#68, count#69] +Results [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] + +(79) Exchange +Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] +Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(80) HashAggregate [codegen id : 24] +Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] +Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] +Functions [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), count(1)] +Aggregate Attributes [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59))#73, count(1)#74] +Results [6]: [catalog AS channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59))#73 AS sales#76, count(1)#74 AS number_sales#77] + +(81) Filter [codegen id : 24] +Input [6]: [channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] +Condition : (isnotnull(sales#76) AND (cast(sales#76 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#55, [id=#56] as decimal(32,6)))) + +(82) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#83), dynamicpruningexpression(ws_sold_date_sk#83 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#82)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(91) CometFilter -Input [4]: [ws_item_sk#80, ws_quantity#81, ws_list_price#82, ws_sold_date_sk#83] -Condition : isnotnull(ws_item_sk#80) +(83) CometFilter +Input [4]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] +Condition : isnotnull(ws_item_sk#78) -(92) ColumnarToRow [codegen id : 35] -Input [4]: [ws_item_sk#80, ws_quantity#81, ws_list_price#82, ws_sold_date_sk#83] +(84) ColumnarToRow [codegen id : 35] +Input [4]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] -(93) ReusedExchange [Reuses operator id: 58] -Output [1]: [ss_item_sk#40] +(85) ReusedExchange [Reuses operator id: 50] +Output [1]: [ss_item_sk#38] -(94) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ws_item_sk#80] -Right keys [1]: [ss_item_sk#40] +(86) BroadcastHashJoin [codegen id : 35] +Left keys [1]: [ws_item_sk#78] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None -(95) ReusedExchange [Reuses operator id: 65] -Output [4]: [i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88] +(87) ReusedExchange [Reuses operator id: 57] +Output [4]: [i_item_sk#83, i_brand_id#84, i_class_id#85, i_category_id#86] -(96) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ws_item_sk#80] -Right keys [1]: [i_item_sk#85] +(88) BroadcastHashJoin [codegen id : 35] +Left keys [1]: [ws_item_sk#78] +Right keys [1]: [i_item_sk#83] Join type: Inner Join condition: None -(97) Project [codegen id : 35] -Output [6]: [ws_quantity#81, ws_list_price#82, ws_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88] -Input [8]: [ws_item_sk#80, ws_quantity#81, ws_list_price#82, ws_sold_date_sk#83, i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88] +(89) Project [codegen id : 35] +Output [6]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_brand_id#84, i_class_id#85, i_category_id#86] +Input [8]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_item_sk#83, i_brand_id#84, i_class_id#85, i_category_id#86] -(98) ReusedExchange [Reuses operator id: 169] -Output [1]: [d_date_sk#89] +(90) ReusedExchange [Reuses operator id: 155] +Output [1]: [d_date_sk#87] -(99) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ws_sold_date_sk#83] -Right keys [1]: [d_date_sk#89] +(91) BroadcastHashJoin [codegen id : 35] +Left keys [1]: [ws_sold_date_sk#81] +Right keys [1]: [d_date_sk#87] Join type: Inner Join condition: None -(100) Project [codegen id : 35] -Output [5]: [ws_quantity#81, ws_list_price#82, i_brand_id#86, i_class_id#87, i_category_id#88] -Input [7]: [ws_quantity#81, ws_list_price#82, ws_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, d_date_sk#89] - -(101) HashAggregate [codegen id : 35] -Input [5]: [ws_quantity#81, ws_list_price#82, i_brand_id#86, i_class_id#87, i_category_id#88] -Keys [3]: [i_brand_id#86, i_class_id#87, i_category_id#88] -Functions [2]: [partial_sum((cast(ws_quantity#81 as decimal(10,0)) * ws_list_price#82)), partial_count(1)] -Aggregate Attributes [3]: [sum#90, isEmpty#91, count#92] -Results [6]: [i_brand_id#86, i_class_id#87, i_category_id#88, sum#93, isEmpty#94, count#95] - -(102) Exchange -Input [6]: [i_brand_id#86, i_class_id#87, i_category_id#88, sum#93, isEmpty#94, count#95] -Arguments: hashpartitioning(i_brand_id#86, i_class_id#87, i_category_id#88, 5), ENSURE_REQUIREMENTS, [plan_id=8] - -(103) HashAggregate [codegen id : 36] -Input [6]: [i_brand_id#86, i_class_id#87, i_category_id#88, sum#93, isEmpty#94, count#95] -Keys [3]: [i_brand_id#86, i_class_id#87, i_category_id#88] -Functions [2]: [sum((cast(ws_quantity#81 as decimal(10,0)) * ws_list_price#82)), count(1)] -Aggregate Attributes [2]: [sum((cast(ws_quantity#81 as decimal(10,0)) * ws_list_price#82))#96, count(1)#97] -Results [6]: [web AS channel#98, i_brand_id#86, i_class_id#87, i_category_id#88, sum((cast(ws_quantity#81 as decimal(10,0)) * ws_list_price#82))#96 AS sales#99, count(1)#97 AS number_sales#100] - -(104) Filter [codegen id : 36] -Input [6]: [channel#98, i_brand_id#86, i_class_id#87, i_category_id#88, sales#99, number_sales#100] -Condition : (isnotnull(sales#99) AND (cast(sales#99 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#57, [id=#58] as decimal(32,6)))) - -(105) Union - -(106) HashAggregate [codegen id : 37] -Input [6]: [channel#54, i_brand_id#42, i_class_id#43, i_category_id#44, sales#55, number_sales#56] -Keys [4]: [channel#54, i_brand_id#42, i_class_id#43, i_category_id#44] -Functions [2]: [partial_sum(sales#55), partial_sum(number_sales#56)] -Aggregate Attributes [3]: [sum#101, isEmpty#102, sum#103] -Results [7]: [channel#54, i_brand_id#42, i_class_id#43, i_category_id#44, sum#104, isEmpty#105, sum#106] - -(107) Exchange -Input [7]: [channel#54, i_brand_id#42, i_class_id#43, i_category_id#44, sum#104, isEmpty#105, sum#106] -Arguments: hashpartitioning(channel#54, i_brand_id#42, i_class_id#43, i_category_id#44, 5), ENSURE_REQUIREMENTS, [plan_id=9] - -(108) HashAggregate [codegen id : 38] -Input [7]: [channel#54, i_brand_id#42, i_class_id#43, i_category_id#44, sum#104, isEmpty#105, sum#106] -Keys [4]: [channel#54, i_brand_id#42, i_class_id#43, i_category_id#44] -Functions [2]: [sum(sales#55), sum(number_sales#56)] -Aggregate Attributes [2]: [sum(sales#55)#107, sum(number_sales#56)#108] -Results [6]: [channel#54, i_brand_id#42, i_class_id#43, i_category_id#44, sum(sales#55)#107 AS sum_sales#109, sum(number_sales#56)#108 AS number_sales#110] - -(109) ReusedExchange [Reuses operator id: 107] -Output [7]: [channel#54, i_brand_id#42, i_class_id#43, i_category_id#44, sum#104, isEmpty#105, sum#106] - -(110) HashAggregate [codegen id : 76] -Input [7]: [channel#54, i_brand_id#42, i_class_id#43, i_category_id#44, sum#104, isEmpty#105, sum#106] -Keys [4]: [channel#54, i_brand_id#42, i_class_id#43, i_category_id#44] -Functions [2]: [sum(sales#55), sum(number_sales#56)] -Aggregate Attributes [2]: [sum(sales#55)#107, sum(number_sales#56)#108] -Results [5]: [channel#54, i_brand_id#42, i_class_id#43, sum(sales#55)#107 AS sum_sales#109, sum(number_sales#56)#108 AS number_sales#110] - -(111) HashAggregate [codegen id : 76] -Input [5]: [channel#54, i_brand_id#42, i_class_id#43, sum_sales#109, number_sales#110] -Keys [3]: [channel#54, i_brand_id#42, i_class_id#43] -Functions [2]: [partial_sum(sum_sales#109), partial_sum(number_sales#110)] -Aggregate Attributes [3]: [sum#111, isEmpty#112, sum#113] -Results [6]: [channel#54, i_brand_id#42, i_class_id#43, sum#114, isEmpty#115, sum#116] - -(112) Exchange -Input [6]: [channel#54, i_brand_id#42, i_class_id#43, sum#114, isEmpty#115, sum#116] -Arguments: hashpartitioning(channel#54, i_brand_id#42, i_class_id#43, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(113) HashAggregate [codegen id : 77] -Input [6]: [channel#54, i_brand_id#42, i_class_id#43, sum#114, isEmpty#115, sum#116] -Keys [3]: [channel#54, i_brand_id#42, i_class_id#43] -Functions [2]: [sum(sum_sales#109), sum(number_sales#110)] -Aggregate Attributes [2]: [sum(sum_sales#109)#117, sum(number_sales#110)#118] -Results [6]: [channel#54, i_brand_id#42, i_class_id#43, null AS i_category_id#119, sum(sum_sales#109)#117 AS sum(sum_sales)#120, sum(number_sales#110)#118 AS sum(number_sales)#121] - -(114) ReusedExchange [Reuses operator id: 107] -Output [7]: [channel#54, i_brand_id#42, i_class_id#43, i_category_id#44, sum#104, isEmpty#105, sum#106] - -(115) HashAggregate [codegen id : 115] -Input [7]: [channel#54, i_brand_id#42, i_class_id#43, i_category_id#44, sum#104, isEmpty#105, sum#106] -Keys [4]: [channel#54, i_brand_id#42, i_class_id#43, i_category_id#44] -Functions [2]: [sum(sales#55), sum(number_sales#56)] -Aggregate Attributes [2]: [sum(sales#55)#107, sum(number_sales#56)#108] -Results [4]: [channel#54, i_brand_id#42, sum(sales#55)#107 AS sum_sales#109, sum(number_sales#56)#108 AS number_sales#110] - -(116) HashAggregate [codegen id : 115] -Input [4]: [channel#54, i_brand_id#42, sum_sales#109, number_sales#110] -Keys [2]: [channel#54, i_brand_id#42] -Functions [2]: [partial_sum(sum_sales#109), partial_sum(number_sales#110)] -Aggregate Attributes [3]: [sum#122, isEmpty#123, sum#124] -Results [5]: [channel#54, i_brand_id#42, sum#125, isEmpty#126, sum#127] - -(117) Exchange -Input [5]: [channel#54, i_brand_id#42, sum#125, isEmpty#126, sum#127] -Arguments: hashpartitioning(channel#54, i_brand_id#42, 5), ENSURE_REQUIREMENTS, [plan_id=11] - -(118) HashAggregate [codegen id : 116] -Input [5]: [channel#54, i_brand_id#42, sum#125, isEmpty#126, sum#127] -Keys [2]: [channel#54, i_brand_id#42] -Functions [2]: [sum(sum_sales#109), sum(number_sales#110)] -Aggregate Attributes [2]: [sum(sum_sales#109)#128, sum(number_sales#110)#129] -Results [6]: [channel#54, i_brand_id#42, null AS i_class_id#130, null AS i_category_id#131, sum(sum_sales#109)#128 AS sum(sum_sales)#132, sum(number_sales#110)#129 AS sum(number_sales)#133] - -(119) ReusedExchange [Reuses operator id: 107] -Output [7]: [channel#54, i_brand_id#42, i_class_id#43, i_category_id#44, sum#104, isEmpty#105, sum#106] - -(120) HashAggregate [codegen id : 154] -Input [7]: [channel#54, i_brand_id#42, i_class_id#43, i_category_id#44, sum#104, isEmpty#105, sum#106] -Keys [4]: [channel#54, i_brand_id#42, i_class_id#43, i_category_id#44] -Functions [2]: [sum(sales#55), sum(number_sales#56)] -Aggregate Attributes [2]: [sum(sales#55)#107, sum(number_sales#56)#108] -Results [3]: [channel#54, sum(sales#55)#107 AS sum_sales#109, sum(number_sales#56)#108 AS number_sales#110] - -(121) HashAggregate [codegen id : 154] -Input [3]: [channel#54, sum_sales#109, number_sales#110] -Keys [1]: [channel#54] -Functions [2]: [partial_sum(sum_sales#109), partial_sum(number_sales#110)] -Aggregate Attributes [3]: [sum#134, isEmpty#135, sum#136] -Results [4]: [channel#54, sum#137, isEmpty#138, sum#139] - -(122) Exchange -Input [4]: [channel#54, sum#137, isEmpty#138, sum#139] -Arguments: hashpartitioning(channel#54, 5), ENSURE_REQUIREMENTS, [plan_id=12] - -(123) HashAggregate [codegen id : 155] -Input [4]: [channel#54, sum#137, isEmpty#138, sum#139] -Keys [1]: [channel#54] -Functions [2]: [sum(sum_sales#109), sum(number_sales#110)] -Aggregate Attributes [2]: [sum(sum_sales#109)#140, sum(number_sales#110)#141] -Results [6]: [channel#54, null AS i_brand_id#142, null AS i_class_id#143, null AS i_category_id#144, sum(sum_sales#109)#140 AS sum(sum_sales)#145, sum(number_sales#110)#141 AS sum(number_sales)#146] - -(124) ReusedExchange [Reuses operator id: 107] -Output [7]: [channel#54, i_brand_id#42, i_class_id#43, i_category_id#44, sum#104, isEmpty#105, sum#106] - -(125) HashAggregate [codegen id : 193] -Input [7]: [channel#54, i_brand_id#42, i_class_id#43, i_category_id#44, sum#104, isEmpty#105, sum#106] -Keys [4]: [channel#54, i_brand_id#42, i_class_id#43, i_category_id#44] -Functions [2]: [sum(sales#55), sum(number_sales#56)] -Aggregate Attributes [2]: [sum(sales#55)#107, sum(number_sales#56)#108] -Results [2]: [sum(sales#55)#107 AS sum_sales#109, sum(number_sales#56)#108 AS number_sales#110] - -(126) HashAggregate [codegen id : 193] -Input [2]: [sum_sales#109, number_sales#110] +(92) Project [codegen id : 35] +Output [5]: [ws_quantity#79, ws_list_price#80, i_brand_id#84, i_class_id#85, i_category_id#86] +Input [7]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_brand_id#84, i_class_id#85, i_category_id#86, d_date_sk#87] + +(93) HashAggregate [codegen id : 35] +Input [5]: [ws_quantity#79, ws_list_price#80, i_brand_id#84, i_class_id#85, i_category_id#86] +Keys [3]: [i_brand_id#84, i_class_id#85, i_category_id#86] +Functions [2]: [partial_sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80)), partial_count(1)] +Aggregate Attributes [3]: [sum#88, isEmpty#89, count#90] +Results [6]: [i_brand_id#84, i_class_id#85, i_category_id#86, sum#91, isEmpty#92, count#93] + +(94) Exchange +Input [6]: [i_brand_id#84, i_class_id#85, i_category_id#86, sum#91, isEmpty#92, count#93] +Arguments: hashpartitioning(i_brand_id#84, i_class_id#85, i_category_id#86, 5), ENSURE_REQUIREMENTS, [plan_id=8] + +(95) HashAggregate [codegen id : 36] +Input [6]: [i_brand_id#84, i_class_id#85, i_category_id#86, sum#91, isEmpty#92, count#93] +Keys [3]: [i_brand_id#84, i_class_id#85, i_category_id#86] +Functions [2]: [sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80)), count(1)] +Aggregate Attributes [2]: [sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80))#94, count(1)#95] +Results [6]: [web AS channel#96, i_brand_id#84, i_class_id#85, i_category_id#86, sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80))#94 AS sales#97, count(1)#95 AS number_sales#98] + +(96) Filter [codegen id : 36] +Input [6]: [channel#96, i_brand_id#84, i_class_id#85, i_category_id#86, sales#97, number_sales#98] +Condition : (isnotnull(sales#97) AND (cast(sales#97 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#55, [id=#56] as decimal(32,6)))) + +(97) Union + +(98) HashAggregate [codegen id : 37] +Input [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sales#53, number_sales#54] +Keys [4]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42] +Functions [2]: [partial_sum(sales#53), partial_sum(number_sales#54)] +Aggregate Attributes [3]: [sum#99, isEmpty#100, sum#101] +Results [7]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum#102, isEmpty#103, sum#104] + +(99) Exchange +Input [7]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum#102, isEmpty#103, sum#104] +Arguments: hashpartitioning(channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, [plan_id=9] + +(100) HashAggregate [codegen id : 38] +Input [7]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum#102, isEmpty#103, sum#104] +Keys [4]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42] +Functions [2]: [sum(sales#53), sum(number_sales#54)] +Aggregate Attributes [2]: [sum(sales#53)#105, sum(number_sales#54)#106] +Results [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum(sales#53)#105 AS sum_sales#107, sum(number_sales#54)#106 AS number_sales#108] + +(101) ReusedExchange [Reuses operator id: 99] +Output [7]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum#102, isEmpty#103, sum#104] + +(102) HashAggregate [codegen id : 76] +Input [7]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum#102, isEmpty#103, sum#104] +Keys [4]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42] +Functions [2]: [sum(sales#53), sum(number_sales#54)] +Aggregate Attributes [2]: [sum(sales#53)#105, sum(number_sales#54)#106] +Results [5]: [channel#52, i_brand_id#40, i_class_id#41, sum(sales#53)#105 AS sum_sales#107, sum(number_sales#54)#106 AS number_sales#108] + +(103) HashAggregate [codegen id : 76] +Input [5]: [channel#52, i_brand_id#40, i_class_id#41, sum_sales#107, number_sales#108] +Keys [3]: [channel#52, i_brand_id#40, i_class_id#41] +Functions [2]: [partial_sum(sum_sales#107), partial_sum(number_sales#108)] +Aggregate Attributes [3]: [sum#109, isEmpty#110, sum#111] +Results [6]: [channel#52, i_brand_id#40, i_class_id#41, sum#112, isEmpty#113, sum#114] + +(104) Exchange +Input [6]: [channel#52, i_brand_id#40, i_class_id#41, sum#112, isEmpty#113, sum#114] +Arguments: hashpartitioning(channel#52, i_brand_id#40, i_class_id#41, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(105) HashAggregate [codegen id : 77] +Input [6]: [channel#52, i_brand_id#40, i_class_id#41, sum#112, isEmpty#113, sum#114] +Keys [3]: [channel#52, i_brand_id#40, i_class_id#41] +Functions [2]: [sum(sum_sales#107), sum(number_sales#108)] +Aggregate Attributes [2]: [sum(sum_sales#107)#115, sum(number_sales#108)#116] +Results [6]: [channel#52, i_brand_id#40, i_class_id#41, null AS i_category_id#117, sum(sum_sales#107)#115 AS sum(sum_sales)#118, sum(number_sales#108)#116 AS sum(number_sales)#119] + +(106) ReusedExchange [Reuses operator id: 99] +Output [7]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum#102, isEmpty#103, sum#104] + +(107) HashAggregate [codegen id : 115] +Input [7]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum#102, isEmpty#103, sum#104] +Keys [4]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42] +Functions [2]: [sum(sales#53), sum(number_sales#54)] +Aggregate Attributes [2]: [sum(sales#53)#105, sum(number_sales#54)#106] +Results [4]: [channel#52, i_brand_id#40, sum(sales#53)#105 AS sum_sales#107, sum(number_sales#54)#106 AS number_sales#108] + +(108) HashAggregate [codegen id : 115] +Input [4]: [channel#52, i_brand_id#40, sum_sales#107, number_sales#108] +Keys [2]: [channel#52, i_brand_id#40] +Functions [2]: [partial_sum(sum_sales#107), partial_sum(number_sales#108)] +Aggregate Attributes [3]: [sum#120, isEmpty#121, sum#122] +Results [5]: [channel#52, i_brand_id#40, sum#123, isEmpty#124, sum#125] + +(109) Exchange +Input [5]: [channel#52, i_brand_id#40, sum#123, isEmpty#124, sum#125] +Arguments: hashpartitioning(channel#52, i_brand_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=11] + +(110) HashAggregate [codegen id : 116] +Input [5]: [channel#52, i_brand_id#40, sum#123, isEmpty#124, sum#125] +Keys [2]: [channel#52, i_brand_id#40] +Functions [2]: [sum(sum_sales#107), sum(number_sales#108)] +Aggregate Attributes [2]: [sum(sum_sales#107)#126, sum(number_sales#108)#127] +Results [6]: [channel#52, i_brand_id#40, null AS i_class_id#128, null AS i_category_id#129, sum(sum_sales#107)#126 AS sum(sum_sales)#130, sum(number_sales#108)#127 AS sum(number_sales)#131] + +(111) ReusedExchange [Reuses operator id: 99] +Output [7]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum#102, isEmpty#103, sum#104] + +(112) HashAggregate [codegen id : 154] +Input [7]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum#102, isEmpty#103, sum#104] +Keys [4]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42] +Functions [2]: [sum(sales#53), sum(number_sales#54)] +Aggregate Attributes [2]: [sum(sales#53)#105, sum(number_sales#54)#106] +Results [3]: [channel#52, sum(sales#53)#105 AS sum_sales#107, sum(number_sales#54)#106 AS number_sales#108] + +(113) HashAggregate [codegen id : 154] +Input [3]: [channel#52, sum_sales#107, number_sales#108] +Keys [1]: [channel#52] +Functions [2]: [partial_sum(sum_sales#107), partial_sum(number_sales#108)] +Aggregate Attributes [3]: [sum#132, isEmpty#133, sum#134] +Results [4]: [channel#52, sum#135, isEmpty#136, sum#137] + +(114) Exchange +Input [4]: [channel#52, sum#135, isEmpty#136, sum#137] +Arguments: hashpartitioning(channel#52, 5), ENSURE_REQUIREMENTS, [plan_id=12] + +(115) HashAggregate [codegen id : 155] +Input [4]: [channel#52, sum#135, isEmpty#136, sum#137] +Keys [1]: [channel#52] +Functions [2]: [sum(sum_sales#107), sum(number_sales#108)] +Aggregate Attributes [2]: [sum(sum_sales#107)#138, sum(number_sales#108)#139] +Results [6]: [channel#52, null AS i_brand_id#140, null AS i_class_id#141, null AS i_category_id#142, sum(sum_sales#107)#138 AS sum(sum_sales)#143, sum(number_sales#108)#139 AS sum(number_sales)#144] + +(116) ReusedExchange [Reuses operator id: 99] +Output [7]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum#102, isEmpty#103, sum#104] + +(117) HashAggregate [codegen id : 193] +Input [7]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum#102, isEmpty#103, sum#104] +Keys [4]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42] +Functions [2]: [sum(sales#53), sum(number_sales#54)] +Aggregate Attributes [2]: [sum(sales#53)#105, sum(number_sales#54)#106] +Results [2]: [sum(sales#53)#105 AS sum_sales#107, sum(number_sales#54)#106 AS number_sales#108] + +(118) HashAggregate [codegen id : 193] +Input [2]: [sum_sales#107, number_sales#108] Keys: [] -Functions [2]: [partial_sum(sum_sales#109), partial_sum(number_sales#110)] -Aggregate Attributes [3]: [sum#147, isEmpty#148, sum#149] -Results [3]: [sum#150, isEmpty#151, sum#152] +Functions [2]: [partial_sum(sum_sales#107), partial_sum(number_sales#108)] +Aggregate Attributes [3]: [sum#145, isEmpty#146, sum#147] +Results [3]: [sum#148, isEmpty#149, sum#150] -(127) Exchange -Input [3]: [sum#150, isEmpty#151, sum#152] +(119) Exchange +Input [3]: [sum#148, isEmpty#149, sum#150] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] -(128) HashAggregate [codegen id : 194] -Input [3]: [sum#150, isEmpty#151, sum#152] +(120) HashAggregate [codegen id : 194] +Input [3]: [sum#148, isEmpty#149, sum#150] Keys: [] -Functions [2]: [sum(sum_sales#109), sum(number_sales#110)] -Aggregate Attributes [2]: [sum(sum_sales#109)#153, sum(number_sales#110)#154] -Results [6]: [null AS channel#155, null AS i_brand_id#156, null AS i_class_id#157, null AS i_category_id#158, sum(sum_sales#109)#153 AS sum(sum_sales)#159, sum(number_sales#110)#154 AS sum(number_sales)#160] +Functions [2]: [sum(sum_sales#107), sum(number_sales#108)] +Aggregate Attributes [2]: [sum(sum_sales#107)#151, sum(number_sales#108)#152] +Results [6]: [null AS channel#153, null AS i_brand_id#154, null AS i_class_id#155, null AS i_category_id#156, sum(sum_sales#107)#151 AS sum(sum_sales)#157, sum(number_sales#108)#152 AS sum(number_sales)#158] -(129) Union +(121) Union -(130) HashAggregate [codegen id : 195] -Input [6]: [channel#54, i_brand_id#42, i_class_id#43, i_category_id#44, sum_sales#109, number_sales#110] -Keys [6]: [channel#54, i_brand_id#42, i_class_id#43, i_category_id#44, sum_sales#109, number_sales#110] +(122) HashAggregate [codegen id : 195] +Input [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#107, number_sales#108] +Keys [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#107, number_sales#108] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#54, i_brand_id#42, i_class_id#43, i_category_id#44, sum_sales#109, number_sales#110] +Results [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#107, number_sales#108] -(131) Exchange -Input [6]: [channel#54, i_brand_id#42, i_class_id#43, i_category_id#44, sum_sales#109, number_sales#110] -Arguments: hashpartitioning(channel#54, i_brand_id#42, i_class_id#43, i_category_id#44, sum_sales#109, number_sales#110, 5), ENSURE_REQUIREMENTS, [plan_id=14] +(123) Exchange +Input [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#107, number_sales#108] +Arguments: hashpartitioning(channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#107, number_sales#108, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(132) HashAggregate [codegen id : 196] -Input [6]: [channel#54, i_brand_id#42, i_class_id#43, i_category_id#44, sum_sales#109, number_sales#110] -Keys [6]: [channel#54, i_brand_id#42, i_class_id#43, i_category_id#44, sum_sales#109, number_sales#110] +(124) HashAggregate [codegen id : 196] +Input [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#107, number_sales#108] +Keys [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#107, number_sales#108] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#54, i_brand_id#42, i_class_id#43, i_category_id#44, sum_sales#109, number_sales#110] +Results [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#107, number_sales#108] -(133) TakeOrderedAndProject -Input [6]: [channel#54, i_brand_id#42, i_class_id#43, i_category_id#44, sum_sales#109, number_sales#110] -Arguments: 100, [channel#54 ASC NULLS FIRST, i_brand_id#42 ASC NULLS FIRST, i_class_id#43 ASC NULLS FIRST, i_category_id#44 ASC NULLS FIRST], [channel#54, i_brand_id#42, i_class_id#43, i_category_id#44, sum_sales#109, number_sales#110] +(125) TakeOrderedAndProject +Input [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#107, number_sales#108] +Arguments: 100, [channel#52 ASC NULLS FIRST, i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#107, number_sales#108] ===== Subqueries ===== -Subquery:1 Hosting operator id = 74 Hosting Expression = Subquery scalar-subquery#57, [id=#58] -* HashAggregate (159) -+- Exchange (158) - +- * ColumnarToRow (157) - +- CometHashAggregate (156) - +- CometUnion (155) - :- CometProject (140) - : +- CometBroadcastHashJoin (139) - : :- CometScan parquet spark_catalog.default.store_sales (134) - : +- CometBroadcastExchange (138) - : +- CometProject (137) - : +- CometFilter (136) - : +- CometScan parquet spark_catalog.default.date_dim (135) - :- CometProject (147) - : +- CometBroadcastHashJoin (146) - : :- CometScan parquet spark_catalog.default.catalog_sales (141) - : +- CometBroadcastExchange (145) - : +- CometProject (144) - : +- CometFilter (143) - : +- CometScan parquet spark_catalog.default.date_dim (142) - +- CometProject (154) - +- CometBroadcastHashJoin (153) - :- CometScan parquet spark_catalog.default.web_sales (148) - +- CometBroadcastExchange (152) - +- CometProject (151) - +- CometFilter (150) - +- CometScan parquet spark_catalog.default.date_dim (149) - - -(134) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#161, ss_list_price#162, ss_sold_date_sk#163] +Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#55, [id=#56] +* HashAggregate (145) ++- Exchange (144) + +- * ColumnarToRow (143) + +- CometHashAggregate (142) + +- CometUnion (141) + :- CometProject (129) + : +- CometBroadcastHashJoin (128) + : :- CometScan parquet spark_catalog.default.store_sales (126) + : +- ReusedExchange (127) + :- CometProject (136) + : +- CometBroadcastHashJoin (135) + : :- CometScan parquet spark_catalog.default.catalog_sales (130) + : +- CometBroadcastExchange (134) + : +- CometProject (133) + : +- CometFilter (132) + : +- CometScan parquet spark_catalog.default.date_dim (131) + +- CometProject (140) + +- CometBroadcastHashJoin (139) + :- CometScan parquet spark_catalog.default.web_sales (137) + +- ReusedExchange (138) + + +(126) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_quantity#159, ss_list_price#160, ss_sold_date_sk#161] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#163), dynamicpruningexpression(ss_sold_date_sk#163 IN dynamicpruning#164)] +PartitionFilters: [isnotnull(ss_sold_date_sk#161), dynamicpruningexpression(ss_sold_date_sk#161 IN dynamicpruning#162)] ReadSchema: struct -(135) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#165, d_year#166] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(136) CometFilter -Input [2]: [d_date_sk#165, d_year#166] -Condition : (((isnotnull(d_year#166) AND (d_year#166 >= 1999)) AND (d_year#166 <= 2001)) AND isnotnull(d_date_sk#165)) +(127) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#163] -(137) CometProject -Input [2]: [d_date_sk#165, d_year#166] -Arguments: [d_date_sk#165], [d_date_sk#165] +(128) CometBroadcastHashJoin +Left output [3]: [ss_quantity#159, ss_list_price#160, ss_sold_date_sk#161] +Right output [1]: [d_date_sk#163] +Arguments: [ss_sold_date_sk#161], [d_date_sk#163], Inner -(138) CometBroadcastExchange -Input [1]: [d_date_sk#165] -Arguments: [d_date_sk#165] +(129) CometProject +Input [4]: [ss_quantity#159, ss_list_price#160, ss_sold_date_sk#161, d_date_sk#163] +Arguments: [quantity#164, list_price#165], [ss_quantity#159 AS quantity#164, ss_list_price#160 AS list_price#165] -(139) CometBroadcastHashJoin -Left output [3]: [ss_quantity#161, ss_list_price#162, ss_sold_date_sk#163] -Right output [1]: [d_date_sk#165] -Arguments: [ss_sold_date_sk#163], [d_date_sk#165], Inner - -(140) CometProject -Input [4]: [ss_quantity#161, ss_list_price#162, ss_sold_date_sk#163, d_date_sk#165] -Arguments: [quantity#167, list_price#168], [ss_quantity#161 AS quantity#167, ss_list_price#162 AS list_price#168] - -(141) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#169, cs_list_price#170, cs_sold_date_sk#171] +(130) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#166, cs_list_price#167, cs_sold_date_sk#168] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#171), dynamicpruningexpression(cs_sold_date_sk#171 IN dynamicpruning#172)] +PartitionFilters: [isnotnull(cs_sold_date_sk#168), dynamicpruningexpression(cs_sold_date_sk#168 IN dynamicpruning#169)] ReadSchema: struct -(142) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#173, d_year#174] +(131) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#170, d_year#171] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(143) CometFilter -Input [2]: [d_date_sk#173, d_year#174] -Condition : (((isnotnull(d_year#174) AND (d_year#174 >= 1998)) AND (d_year#174 <= 2000)) AND isnotnull(d_date_sk#173)) +(132) CometFilter +Input [2]: [d_date_sk#170, d_year#171] +Condition : (((isnotnull(d_year#171) AND (d_year#171 >= 1998)) AND (d_year#171 <= 2000)) AND isnotnull(d_date_sk#170)) -(144) CometProject -Input [2]: [d_date_sk#173, d_year#174] -Arguments: [d_date_sk#173], [d_date_sk#173] +(133) CometProject +Input [2]: [d_date_sk#170, d_year#171] +Arguments: [d_date_sk#170], [d_date_sk#170] -(145) CometBroadcastExchange -Input [1]: [d_date_sk#173] -Arguments: [d_date_sk#173] +(134) CometBroadcastExchange +Input [1]: [d_date_sk#170] +Arguments: [d_date_sk#170] -(146) CometBroadcastHashJoin -Left output [3]: [cs_quantity#169, cs_list_price#170, cs_sold_date_sk#171] -Right output [1]: [d_date_sk#173] -Arguments: [cs_sold_date_sk#171], [d_date_sk#173], Inner +(135) CometBroadcastHashJoin +Left output [3]: [cs_quantity#166, cs_list_price#167, cs_sold_date_sk#168] +Right output [1]: [d_date_sk#170] +Arguments: [cs_sold_date_sk#168], [d_date_sk#170], Inner -(147) CometProject -Input [4]: [cs_quantity#169, cs_list_price#170, cs_sold_date_sk#171, d_date_sk#173] -Arguments: [quantity#175, list_price#176], [cs_quantity#169 AS quantity#175, cs_list_price#170 AS list_price#176] +(136) CometProject +Input [4]: [cs_quantity#166, cs_list_price#167, cs_sold_date_sk#168, d_date_sk#170] +Arguments: [quantity#172, list_price#173], [cs_quantity#166 AS quantity#172, cs_list_price#167 AS list_price#173] -(148) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#177, ws_list_price#178, ws_sold_date_sk#179] +(137) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#174, ws_list_price#175, ws_sold_date_sk#176] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#179), dynamicpruningexpression(ws_sold_date_sk#179 IN dynamicpruning#180)] +PartitionFilters: [isnotnull(ws_sold_date_sk#176), dynamicpruningexpression(ws_sold_date_sk#176 IN dynamicpruning#177)] ReadSchema: struct -(149) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#181, d_year#182] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(150) CometFilter -Input [2]: [d_date_sk#181, d_year#182] -Condition : (((isnotnull(d_year#182) AND (d_year#182 >= 1998)) AND (d_year#182 <= 2000)) AND isnotnull(d_date_sk#181)) - -(151) CometProject -Input [2]: [d_date_sk#181, d_year#182] -Arguments: [d_date_sk#181], [d_date_sk#181] - -(152) CometBroadcastExchange -Input [1]: [d_date_sk#181] -Arguments: [d_date_sk#181] +(138) ReusedExchange [Reuses operator id: 134] +Output [1]: [d_date_sk#178] -(153) CometBroadcastHashJoin -Left output [3]: [ws_quantity#177, ws_list_price#178, ws_sold_date_sk#179] -Right output [1]: [d_date_sk#181] -Arguments: [ws_sold_date_sk#179], [d_date_sk#181], Inner +(139) CometBroadcastHashJoin +Left output [3]: [ws_quantity#174, ws_list_price#175, ws_sold_date_sk#176] +Right output [1]: [d_date_sk#178] +Arguments: [ws_sold_date_sk#176], [d_date_sk#178], Inner -(154) CometProject -Input [4]: [ws_quantity#177, ws_list_price#178, ws_sold_date_sk#179, d_date_sk#181] -Arguments: [quantity#183, list_price#184], [ws_quantity#177 AS quantity#183, ws_list_price#178 AS list_price#184] +(140) CometProject +Input [4]: [ws_quantity#174, ws_list_price#175, ws_sold_date_sk#176, d_date_sk#178] +Arguments: [quantity#179, list_price#180], [ws_quantity#174 AS quantity#179, ws_list_price#175 AS list_price#180] -(155) CometUnion -Child 0 Input [2]: [quantity#167, list_price#168] -Child 1 Input [2]: [quantity#175, list_price#176] -Child 2 Input [2]: [quantity#183, list_price#184] +(141) CometUnion +Child 0 Input [2]: [quantity#164, list_price#165] +Child 1 Input [2]: [quantity#172, list_price#173] +Child 2 Input [2]: [quantity#179, list_price#180] -(156) CometHashAggregate -Input [2]: [quantity#167, list_price#168] +(142) CometHashAggregate +Input [2]: [quantity#164, list_price#165] Keys: [] -Functions [1]: [partial_avg((cast(quantity#167 as decimal(10,0)) * list_price#168))] +Functions [1]: [partial_avg((cast(quantity#164 as decimal(10,0)) * list_price#165))] -(157) ColumnarToRow [codegen id : 1] -Input [2]: [sum#185, count#186] +(143) ColumnarToRow [codegen id : 1] +Input [2]: [sum#181, count#182] -(158) Exchange -Input [2]: [sum#185, count#186] +(144) Exchange +Input [2]: [sum#181, count#182] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=15] -(159) HashAggregate [codegen id : 2] -Input [2]: [sum#185, count#186] +(145) HashAggregate [codegen id : 2] +Input [2]: [sum#181, count#182] Keys: [] -Functions [1]: [avg((cast(quantity#167 as decimal(10,0)) * list_price#168))] -Aggregate Attributes [1]: [avg((cast(quantity#167 as decimal(10,0)) * list_price#168))#187] -Results [1]: [avg((cast(quantity#167 as decimal(10,0)) * list_price#168))#187 AS average_sales#188] +Functions [1]: [avg((cast(quantity#164 as decimal(10,0)) * list_price#165))] +Aggregate Attributes [1]: [avg((cast(quantity#164 as decimal(10,0)) * list_price#165))#183] +Results [1]: [avg((cast(quantity#164 as decimal(10,0)) * list_price#165))#183 AS average_sales#184] -Subquery:2 Hosting operator id = 134 Hosting Expression = ss_sold_date_sk#163 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 126 Hosting Expression = ss_sold_date_sk#161 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 141 Hosting Expression = cs_sold_date_sk#171 IN dynamicpruning#172 -BroadcastExchange (164) -+- * ColumnarToRow (163) - +- CometProject (162) - +- CometFilter (161) - +- CometScan parquet spark_catalog.default.date_dim (160) +Subquery:3 Hosting operator id = 130 Hosting Expression = cs_sold_date_sk#168 IN dynamicpruning#169 +BroadcastExchange (150) ++- * ColumnarToRow (149) + +- CometProject (148) + +- CometFilter (147) + +- CometScan parquet spark_catalog.default.date_dim (146) -(160) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#173, d_year#174] +(146) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#170, d_year#171] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(161) CometFilter -Input [2]: [d_date_sk#173, d_year#174] -Condition : (((isnotnull(d_year#174) AND (d_year#174 >= 1998)) AND (d_year#174 <= 2000)) AND isnotnull(d_date_sk#173)) +(147) CometFilter +Input [2]: [d_date_sk#170, d_year#171] +Condition : (((isnotnull(d_year#171) AND (d_year#171 >= 1998)) AND (d_year#171 <= 2000)) AND isnotnull(d_date_sk#170)) -(162) CometProject -Input [2]: [d_date_sk#173, d_year#174] -Arguments: [d_date_sk#173], [d_date_sk#173] +(148) CometProject +Input [2]: [d_date_sk#170, d_year#171] +Arguments: [d_date_sk#170], [d_date_sk#170] -(163) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#173] +(149) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#170] -(164) BroadcastExchange -Input [1]: [d_date_sk#173] +(150) BroadcastExchange +Input [1]: [d_date_sk#170] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] -Subquery:4 Hosting operator id = 148 Hosting Expression = ws_sold_date_sk#179 IN dynamicpruning#172 +Subquery:4 Hosting operator id = 137 Hosting Expression = ws_sold_date_sk#176 IN dynamicpruning#169 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (169) -+- * ColumnarToRow (168) - +- CometProject (167) - +- CometFilter (166) - +- CometScan parquet spark_catalog.default.date_dim (165) +BroadcastExchange (155) ++- * ColumnarToRow (154) + +- CometProject (153) + +- CometFilter (152) + +- CometScan parquet spark_catalog.default.date_dim (151) -(165) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#45, d_year#189, d_moy#190] +(151) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#43, d_year#185, d_moy#186] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(166) CometFilter -Input [3]: [d_date_sk#45, d_year#189, d_moy#190] -Condition : ((((isnotnull(d_year#189) AND isnotnull(d_moy#190)) AND (d_year#189 = 2000)) AND (d_moy#190 = 11)) AND isnotnull(d_date_sk#45)) +(152) CometFilter +Input [3]: [d_date_sk#43, d_year#185, d_moy#186] +Condition : ((((isnotnull(d_year#185) AND isnotnull(d_moy#186)) AND (d_year#185 = 2000)) AND (d_moy#186 = 11)) AND isnotnull(d_date_sk#43)) -(167) CometProject -Input [3]: [d_date_sk#45, d_year#189, d_moy#190] -Arguments: [d_date_sk#45], [d_date_sk#45] +(153) CometProject +Input [3]: [d_date_sk#43, d_year#185, d_moy#186] +Arguments: [d_date_sk#43], [d_date_sk#43] -(168) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#45] +(154) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#43] -(169) BroadcastExchange -Input [1]: [d_date_sk#45] +(155) BroadcastExchange +Input [1]: [d_date_sk#43] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=17] Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (174) -+- * ColumnarToRow (173) - +- CometProject (172) - +- CometFilter (171) - +- CometScan parquet spark_catalog.default.date_dim (170) +BroadcastExchange (160) ++- * ColumnarToRow (159) + +- CometProject (158) + +- CometFilter (157) + +- CometScan parquet spark_catalog.default.date_dim (156) -(170) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] +(156) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_year#187] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(171) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : (((isnotnull(d_year#27) AND (d_year#27 >= 1999)) AND (d_year#27 <= 2001)) AND isnotnull(d_date_sk#26)) +(157) CometFilter +Input [2]: [d_date_sk#26, d_year#187] +Condition : (((isnotnull(d_year#187) AND (d_year#187 >= 1999)) AND (d_year#187 <= 2001)) AND isnotnull(d_date_sk#26)) -(172) CometProject -Input [2]: [d_date_sk#26, d_year#27] +(158) CometProject +Input [2]: [d_date_sk#26, d_year#187] Arguments: [d_date_sk#26], [d_date_sk#26] -(173) ColumnarToRow [codegen id : 1] +(159) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] -(174) BroadcastExchange +(160) BroadcastExchange Input [1]: [d_date_sk#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=18] Subquery:7 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#32 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 89 Hosting Expression = ReusedSubquery Subquery scalar-subquery#57, [id=#58] +Subquery:9 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#55, [id=#56] -Subquery:10 Hosting operator id = 75 Hosting Expression = cs_sold_date_sk#62 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 104 Hosting Expression = ReusedSubquery Subquery scalar-subquery#57, [id=#58] +Subquery:11 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#55, [id=#56] -Subquery:12 Hosting operator id = 90 Hosting Expression = ws_sold_date_sk#83 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 82 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt index f6df57782..45061c290 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num WholeStageCodegen (2) HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] InputAdapter - Exchange #18 + Exchange #15 WholeStageCodegen (1) ColumnarToRow InputAdapter @@ -31,22 +31,19 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #19 - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #12 CometProject [cs_quantity,cs_list_price] [quantity,list_price] CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #20 + BroadcastExchange #16 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #21 + CometBroadcastExchange #17 CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -54,10 +51,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #4 - CometBroadcastExchange #22 - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #17 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #3 @@ -133,12 +127,9 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #13 - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #12 InputAdapter - BroadcastExchange #14 + BroadcastExchange #13 WholeStageCodegen (2) ColumnarToRow InputAdapter @@ -149,15 +140,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #15 - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #16 - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + ReusedExchange [d_date_sk] #12 InputAdapter - BroadcastExchange #17 + BroadcastExchange #14 WholeStageCodegen (9) BroadcastHashJoin [i_item_sk,ss_item_sk] ColumnarToRow @@ -173,7 +159,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #23 + Exchange [i_brand_id,i_class_id,i_category_id] #18 WholeStageCodegen (23) HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] @@ -189,7 +175,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter ReusedExchange [ss_item_sk] #5 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #17 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 InputAdapter ReusedExchange [d_date_sk] #4 WholeStageCodegen (36) @@ -197,7 +183,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #24 + Exchange [i_brand_id,i_class_id,i_category_id] #19 WholeStageCodegen (35) HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] @@ -213,13 +199,13 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter ReusedExchange [ss_item_sk] #5 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #17 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 InputAdapter ReusedExchange [d_date_sk] #4 WholeStageCodegen (77) HashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter - Exchange [channel,i_brand_id,i_class_id] #25 + Exchange [channel,i_brand_id,i_class_id] #20 WholeStageCodegen (76) HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] @@ -228,7 +214,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num WholeStageCodegen (116) HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter - Exchange [channel,i_brand_id] #26 + Exchange [channel,i_brand_id] #21 WholeStageCodegen (115) HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] @@ -237,7 +223,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num WholeStageCodegen (155) HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter - Exchange [channel] #27 + Exchange [channel] #22 WholeStageCodegen (154) HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] @@ -246,7 +232,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num WholeStageCodegen (194) HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter - Exchange #28 + Exchange #23 WholeStageCodegen (193) HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt index b7e5a7408..80925f531 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt @@ -1,16 +1,16 @@ == Physical Plan == -TakeOrderedAndProject (50) -+- * HashAggregate (49) - +- Exchange (48) - +- * HashAggregate (47) - +- * Project (46) - +- * BroadcastHashJoin Inner BuildRight (45) - :- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (34) - : : +- * Filter (33) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (32) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (22) +TakeOrderedAndProject (44) ++- * HashAggregate (43) + +- Exchange (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * Project (34) + : +- * BroadcastHashJoin Inner BuildRight (33) + : :- * Project (28) + : : +- * Filter (27) + : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) + : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) : : : :- * ColumnarToRow (12) : : : : +- CometBroadcastHashJoin (11) : : : : :- CometFilter (2) @@ -23,32 +23,26 @@ TakeOrderedAndProject (50) : : : : +- CometProject (6) : : : : +- CometFilter (5) : : : : +- CometScan parquet spark_catalog.default.date_dim (4) - : : : +- BroadcastExchange (21) - : : : +- * ColumnarToRow (20) - : : : +- CometProject (19) - : : : +- CometBroadcastHashJoin (18) + : : : +- BroadcastExchange (18) + : : : +- * ColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometBroadcastHashJoin (15) : : : :- CometScan parquet spark_catalog.default.web_sales (13) - : : : +- CometBroadcastExchange (17) - : : : +- CometProject (16) - : : : +- CometFilter (15) - : : : +- CometScan parquet spark_catalog.default.date_dim (14) - : : +- BroadcastExchange (31) - : : +- * ColumnarToRow (30) - : : +- CometProject (29) - : : +- CometBroadcastHashJoin (28) - : : :- CometScan parquet spark_catalog.default.catalog_sales (23) - : : +- CometBroadcastExchange (27) - : : +- CometProject (26) - : : +- CometFilter (25) - : : +- CometScan parquet spark_catalog.default.date_dim (24) - : +- BroadcastExchange (38) - : +- * ColumnarToRow (37) - : +- CometFilter (36) - : +- CometScan parquet spark_catalog.default.customer_address (35) - +- BroadcastExchange (44) - +- * ColumnarToRow (43) - +- CometFilter (42) - +- CometScan parquet spark_catalog.default.customer_demographics (41) + : : : +- ReusedExchange (14) + : : +- BroadcastExchange (25) + : : +- * ColumnarToRow (24) + : : +- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometScan parquet spark_catalog.default.catalog_sales (20) + : : +- ReusedExchange (21) + : +- BroadcastExchange (32) + : +- * ColumnarToRow (31) + : +- CometFilter (30) + : +- CometScan parquet spark_catalog.default.customer_address (29) + +- BroadcastExchange (38) + +- * ColumnarToRow (37) + +- CometFilter (36) + +- CometScan parquet spark_catalog.default.customer_demographics (35) (1) Scan parquet spark_catalog.default.customer @@ -116,215 +110,183 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#14)] ReadSchema: struct -(14) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Condition : ((((isnotnull(d_year#16) AND isnotnull(d_qoy#17)) AND (d_year#16 = 2002)) AND (d_qoy#17 < 4)) AND isnotnull(d_date_sk#15)) - -(16) CometProject -Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Arguments: [d_date_sk#15], [d_date_sk#15] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#15] -Arguments: [d_date_sk#15] +(14) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#15] -(18) CometBroadcastHashJoin +(15) CometBroadcastHashJoin Left output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] Right output [1]: [d_date_sk#15] Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner -(19) CometProject +(16) CometProject Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] -(20) ColumnarToRow [codegen id : 1] +(17) ColumnarToRow [codegen id : 1] Input [1]: [ws_bill_customer_sk#12] -(21) BroadcastExchange +(18) BroadcastExchange Input [1]: [ws_bill_customer_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(22) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#12] Join type: ExistenceJoin(exists#2) Join condition: None -(23) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] +(20) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#20)] +PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] ReadSchema: struct -(24) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#21, d_year#22, d_qoy#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [3]: [d_date_sk#21, d_year#22, d_qoy#23] -Condition : ((((isnotnull(d_year#22) AND isnotnull(d_qoy#23)) AND (d_year#22 = 2002)) AND (d_qoy#23 < 4)) AND isnotnull(d_date_sk#21)) - -(26) CometProject -Input [3]: [d_date_sk#21, d_year#22, d_qoy#23] -Arguments: [d_date_sk#21], [d_date_sk#21] +(21) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#19] -(27) CometBroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: [d_date_sk#21] +(22) CometBroadcastHashJoin +Left output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Right output [1]: [d_date_sk#19] +Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner -(28) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] -Right output [1]: [d_date_sk#21] -Arguments: [cs_sold_date_sk#19], [d_date_sk#21], Inner +(23) CometProject +Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] +Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] -(29) CometProject -Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#21] -Arguments: [cs_ship_customer_sk#18], [cs_ship_customer_sk#18] +(24) ColumnarToRow [codegen id : 2] +Input [1]: [cs_ship_customer_sk#16] -(30) ColumnarToRow [codegen id : 2] -Input [1]: [cs_ship_customer_sk#18] - -(31) BroadcastExchange -Input [1]: [cs_ship_customer_sk#18] +(25) BroadcastExchange +Input [1]: [cs_ship_customer_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(32) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#18] +Right keys [1]: [cs_ship_customer_sk#16] Join type: ExistenceJoin(exists#1) Join condition: None -(33) Filter [codegen id : 5] +(27) Filter [codegen id : 5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(34) Project [codegen id : 5] +(28) Project [codegen id : 5] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(35) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#24, ca_state#25] +(29) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#20, ca_state#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(36) CometFilter -Input [2]: [ca_address_sk#24, ca_state#25] -Condition : isnotnull(ca_address_sk#24) +(30) CometFilter +Input [2]: [ca_address_sk#20, ca_state#21] +Condition : isnotnull(ca_address_sk#20) -(37) ColumnarToRow [codegen id : 3] -Input [2]: [ca_address_sk#24, ca_state#25] +(31) ColumnarToRow [codegen id : 3] +Input [2]: [ca_address_sk#20, ca_state#21] -(38) BroadcastExchange -Input [2]: [ca_address_sk#24, ca_state#25] +(32) BroadcastExchange +Input [2]: [ca_address_sk#20, ca_state#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(39) BroadcastHashJoin [codegen id : 5] +(33) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#24] +Right keys [1]: [ca_address_sk#20] Join type: Inner Join condition: None -(40) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#4, ca_state#25] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#24, ca_state#25] +(34) Project [codegen id : 5] +Output [2]: [c_current_cdemo_sk#4, ca_state#21] +Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#21] -(41) Scan parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +(35) Scan parquet spark_catalog.default.customer_demographics +Output [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(42) CometFilter -Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Condition : isnotnull(cd_demo_sk#26) +(36) CometFilter +Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Condition : isnotnull(cd_demo_sk#22) -(43) ColumnarToRow [codegen id : 4] -Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +(37) ColumnarToRow [codegen id : 4] +Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -(44) BroadcastExchange -Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +(38) BroadcastExchange +Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(45) BroadcastHashJoin [codegen id : 5] +(39) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#26] +Right keys [1]: [cd_demo_sk#22] Join type: Inner Join condition: None -(46) Project [codegen id : 5] -Output [6]: [ca_state#25, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Input [8]: [c_current_cdemo_sk#4, ca_state#25, cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] - -(47) HashAggregate [codegen id : 5] -Input [6]: [ca_state#25, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Keys [6]: [ca_state#25, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#29), partial_max(cd_dep_count#29), partial_sum(cd_dep_count#29), partial_avg(cd_dep_employed_count#30), partial_max(cd_dep_employed_count#30), partial_sum(cd_dep_employed_count#30), partial_avg(cd_dep_college_count#31), partial_max(cd_dep_college_count#31), partial_sum(cd_dep_college_count#31)] -Aggregate Attributes [13]: [count#32, sum#33, count#34, max#35, sum#36, sum#37, count#38, max#39, sum#40, sum#41, count#42, max#43, sum#44] -Results [19]: [ca_state#25, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] - -(48) Exchange -Input [19]: [ca_state#25, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] -Arguments: hashpartitioning(ca_state#25, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(49) HashAggregate [codegen id : 6] -Input [19]: [ca_state#25, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] -Keys [6]: [ca_state#25, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Functions [10]: [count(1), avg(cd_dep_count#29), max(cd_dep_count#29), sum(cd_dep_count#29), avg(cd_dep_employed_count#30), max(cd_dep_employed_count#30), sum(cd_dep_employed_count#30), avg(cd_dep_college_count#31), max(cd_dep_college_count#31), sum(cd_dep_college_count#31)] -Aggregate Attributes [10]: [count(1)#58, avg(cd_dep_count#29)#59, max(cd_dep_count#29)#60, sum(cd_dep_count#29)#61, avg(cd_dep_employed_count#30)#62, max(cd_dep_employed_count#30)#63, sum(cd_dep_employed_count#30)#64, avg(cd_dep_college_count#31)#65, max(cd_dep_college_count#31)#66, sum(cd_dep_college_count#31)#67] -Results [18]: [ca_state#25, cd_gender#27, cd_marital_status#28, cd_dep_count#29, count(1)#58 AS cnt1#68, avg(cd_dep_count#29)#59 AS avg(cd_dep_count)#69, max(cd_dep_count#29)#60 AS max(cd_dep_count)#70, sum(cd_dep_count#29)#61 AS sum(cd_dep_count)#71, cd_dep_employed_count#30, count(1)#58 AS cnt2#72, avg(cd_dep_employed_count#30)#62 AS avg(cd_dep_employed_count)#73, max(cd_dep_employed_count#30)#63 AS max(cd_dep_employed_count)#74, sum(cd_dep_employed_count#30)#64 AS sum(cd_dep_employed_count)#75, cd_dep_college_count#31, count(1)#58 AS cnt3#76, avg(cd_dep_college_count#31)#65 AS avg(cd_dep_college_count)#77, max(cd_dep_college_count#31)#66 AS max(cd_dep_college_count)#78, sum(cd_dep_college_count#31)#67 AS sum(cd_dep_college_count)#79] - -(50) TakeOrderedAndProject -Input [18]: [ca_state#25, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cnt1#68, avg(cd_dep_count)#69, max(cd_dep_count)#70, sum(cd_dep_count)#71, cd_dep_employed_count#30, cnt2#72, avg(cd_dep_employed_count)#73, max(cd_dep_employed_count)#74, sum(cd_dep_employed_count)#75, cd_dep_college_count#31, cnt3#76, avg(cd_dep_college_count)#77, max(cd_dep_college_count)#78, sum(cd_dep_college_count)#79] -Arguments: 100, [ca_state#25 ASC NULLS FIRST, cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, cd_dep_count#29 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [ca_state#25, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cnt1#68, avg(cd_dep_count)#69, max(cd_dep_count)#70, sum(cd_dep_count)#71, cd_dep_employed_count#30, cnt2#72, avg(cd_dep_employed_count)#73, max(cd_dep_employed_count)#74, sum(cd_dep_employed_count)#75, cd_dep_college_count#31, cnt3#76, avg(cd_dep_college_count)#77, max(cd_dep_college_count)#78, sum(cd_dep_college_count)#79] +(40) Project [codegen id : 5] +Output [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Input [8]: [c_current_cdemo_sk#4, ca_state#21, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] + +(41) HashAggregate [codegen id : 5] +Input [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Keys [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#25), partial_max(cd_dep_count#25), partial_sum(cd_dep_count#25), partial_avg(cd_dep_employed_count#26), partial_max(cd_dep_employed_count#26), partial_sum(cd_dep_employed_count#26), partial_avg(cd_dep_college_count#27), partial_max(cd_dep_college_count#27), partial_sum(cd_dep_college_count#27)] +Aggregate Attributes [13]: [count#28, sum#29, count#30, max#31, sum#32, sum#33, count#34, max#35, sum#36, sum#37, count#38, max#39, sum#40] +Results [19]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53] + +(42) Exchange +Input [19]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53] +Arguments: hashpartitioning(ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(43) HashAggregate [codegen id : 6] +Input [19]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53] +Keys [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Functions [10]: [count(1), avg(cd_dep_count#25), max(cd_dep_count#25), sum(cd_dep_count#25), avg(cd_dep_employed_count#26), max(cd_dep_employed_count#26), sum(cd_dep_employed_count#26), avg(cd_dep_college_count#27), max(cd_dep_college_count#27), sum(cd_dep_college_count#27)] +Aggregate Attributes [10]: [count(1)#54, avg(cd_dep_count#25)#55, max(cd_dep_count#25)#56, sum(cd_dep_count#25)#57, avg(cd_dep_employed_count#26)#58, max(cd_dep_employed_count#26)#59, sum(cd_dep_employed_count#26)#60, avg(cd_dep_college_count#27)#61, max(cd_dep_college_count#27)#62, sum(cd_dep_college_count#27)#63] +Results [18]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, count(1)#54 AS cnt1#64, avg(cd_dep_count#25)#55 AS avg(cd_dep_count)#65, max(cd_dep_count#25)#56 AS max(cd_dep_count)#66, sum(cd_dep_count#25)#57 AS sum(cd_dep_count)#67, cd_dep_employed_count#26, count(1)#54 AS cnt2#68, avg(cd_dep_employed_count#26)#58 AS avg(cd_dep_employed_count)#69, max(cd_dep_employed_count#26)#59 AS max(cd_dep_employed_count)#70, sum(cd_dep_employed_count#26)#60 AS sum(cd_dep_employed_count)#71, cd_dep_college_count#27, count(1)#54 AS cnt3#72, avg(cd_dep_college_count#27)#61 AS avg(cd_dep_college_count)#73, max(cd_dep_college_count#27)#62 AS max(cd_dep_college_count)#74, sum(cd_dep_college_count#27)#63 AS sum(cd_dep_college_count)#75] + +(44) TakeOrderedAndProject +Input [18]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#64, avg(cd_dep_count)#65, max(cd_dep_count)#66, sum(cd_dep_count)#67, cd_dep_employed_count#26, cnt2#68, avg(cd_dep_employed_count)#69, max(cd_dep_employed_count)#70, sum(cd_dep_employed_count)#71, cd_dep_college_count#27, cnt3#72, avg(cd_dep_college_count)#73, max(cd_dep_college_count)#74, sum(cd_dep_college_count)#75] +Arguments: 100, [ca_state#21 ASC NULLS FIRST, cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_dep_count#25 ASC NULLS FIRST, cd_dep_employed_count#26 ASC NULLS FIRST, cd_dep_college_count#27 ASC NULLS FIRST], [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#64, avg(cd_dep_count)#65, max(cd_dep_count)#66, sum(cd_dep_count)#67, cd_dep_employed_count#26, cnt2#68, avg(cd_dep_employed_count)#69, max(cd_dep_employed_count)#70, sum(cd_dep_employed_count)#71, cd_dep_college_count#27, cnt3#72, avg(cd_dep_college_count)#73, max(cd_dep_college_count)#74, sum(cd_dep_college_count)#75] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (55) -+- * ColumnarToRow (54) - +- CometProject (53) - +- CometFilter (52) - +- CometScan parquet spark_catalog.default.date_dim (51) +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan parquet spark_catalog.default.date_dim (45) -(51) Scan parquet spark_catalog.default.date_dim +(45) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(52) CometFilter +(46) CometFilter Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) -(53) CometProject +(47) CometProject Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(54) ColumnarToRow [codegen id : 1] +(48) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(55) BroadcastExchange +(49) BroadcastExchange Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt index a4ccd6216..a6b4add1f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt @@ -43,12 +43,9 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #6 - CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #7 + BroadcastExchange #6 WholeStageCodegen (2) ColumnarToRow InputAdapter @@ -56,19 +53,16 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #8 - CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #9 + BroadcastExchange #7 WholeStageCodegen (3) ColumnarToRow InputAdapter CometFilter [ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #10 + BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt index 844eb2559..75fed430b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt @@ -1,15 +1,15 @@ == Physical Plan == -TakeOrderedAndProject (44) -+- * HashAggregate (43) - +- Exchange (42) - +- * ColumnarToRow (41) - +- CometHashAggregate (40) - +- CometProject (39) - +- CometBroadcastHashJoin (38) - :- CometProject (34) - : +- CometBroadcastHashJoin (33) - : :- CometProject (29) - : : +- CometBroadcastHashJoin (28) +TakeOrderedAndProject (38) ++- * HashAggregate (37) + +- Exchange (36) + +- * ColumnarToRow (35) + +- CometHashAggregate (34) + +- CometProject (33) + +- CometBroadcastHashJoin (32) + :- CometProject (28) + : +- CometBroadcastHashJoin (27) + : :- CometProject (23) + : : +- CometBroadcastHashJoin (22) : : :- CometBroadcastHashJoin (11) : : : :- CometFilter (2) : : : : +- CometScan parquet spark_catalog.default.customer (1) @@ -21,28 +21,22 @@ TakeOrderedAndProject (44) : : : +- CometProject (6) : : : +- CometFilter (5) : : : +- CometScan parquet spark_catalog.default.date_dim (4) - : : +- CometBroadcastExchange (27) - : : +- CometUnion (26) - : : :- CometProject (18) - : : : +- CometBroadcastHashJoin (17) + : : +- CometBroadcastExchange (21) + : : +- CometUnion (20) + : : :- CometProject (15) + : : : +- CometBroadcastHashJoin (14) : : : :- CometScan parquet spark_catalog.default.web_sales (12) - : : : +- CometBroadcastExchange (16) - : : : +- CometProject (15) - : : : +- CometFilter (14) - : : : +- CometScan parquet spark_catalog.default.date_dim (13) - : : +- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- CometScan parquet spark_catalog.default.catalog_sales (19) - : : +- CometBroadcastExchange (23) - : : +- CometProject (22) - : : +- CometFilter (21) - : : +- CometScan parquet spark_catalog.default.date_dim (20) - : +- CometBroadcastExchange (32) - : +- CometFilter (31) - : +- CometScan parquet spark_catalog.default.customer_address (30) - +- CometBroadcastExchange (37) - +- CometFilter (36) - +- CometScan parquet spark_catalog.default.customer_demographics (35) + : : : +- ReusedExchange (13) + : : +- CometProject (19) + : : +- CometBroadcastHashJoin (18) + : : :- CometScan parquet spark_catalog.default.catalog_sales (16) + : : +- ReusedExchange (17) + : +- CometBroadcastExchange (26) + : +- CometFilter (25) + : +- CometScan parquet spark_catalog.default.customer_address (24) + +- CometBroadcastExchange (31) + +- CometFilter (30) + +- CometScan parquet spark_catalog.default.customer_demographics (29) (1) Scan parquet spark_catalog.default.customer @@ -107,191 +101,159 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] ReadSchema: struct -(13) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Condition : ((((isnotnull(d_year#14) AND isnotnull(d_qoy#15)) AND (d_year#14 = 1999)) AND (d_qoy#15 < 4)) AND isnotnull(d_date_sk#13)) - -(15) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(16) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] +(13) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#13] -(17) CometBroadcastHashJoin +(14) CometBroadcastHashJoin Left output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] Right output [1]: [d_date_sk#13] Arguments: [ws_sold_date_sk#11], [d_date_sk#13], Inner -(18) CometProject +(15) CometProject Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] -Arguments: [customsk#16], [ws_bill_customer_sk#10 AS customsk#16] +Arguments: [customsk#14], [ws_bill_customer_sk#10 AS customsk#14] -(19) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] +(16) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] ReadSchema: struct -(20) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#20, d_year#21, d_qoy#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [d_date_sk#20, d_year#21, d_qoy#22] -Condition : ((((isnotnull(d_year#21) AND isnotnull(d_qoy#22)) AND (d_year#21 = 1999)) AND (d_qoy#22 < 4)) AND isnotnull(d_date_sk#20)) - -(22) CometProject -Input [3]: [d_date_sk#20, d_year#21, d_qoy#22] -Arguments: [d_date_sk#20], [d_date_sk#20] - -(23) CometBroadcastExchange -Input [1]: [d_date_sk#20] -Arguments: [d_date_sk#20] +(17) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#18] -(24) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] -Right output [1]: [d_date_sk#20] -Arguments: [cs_sold_date_sk#18], [d_date_sk#20], Inner +(18) CometBroadcastHashJoin +Left output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +Right output [1]: [d_date_sk#18] +Arguments: [cs_sold_date_sk#16], [d_date_sk#18], Inner -(25) CometProject -Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#20] -Arguments: [customsk#23], [cs_ship_customer_sk#17 AS customsk#23] +(19) CometProject +Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#18] +Arguments: [customsk#19], [cs_ship_customer_sk#15 AS customsk#19] -(26) CometUnion -Child 0 Input [1]: [customsk#16] -Child 1 Input [1]: [customsk#23] +(20) CometUnion +Child 0 Input [1]: [customsk#14] +Child 1 Input [1]: [customsk#19] -(27) CometBroadcastExchange -Input [1]: [customsk#16] -Arguments: [customsk#16] +(21) CometBroadcastExchange +Input [1]: [customsk#14] +Arguments: [customsk#14] -(28) CometBroadcastHashJoin +(22) CometBroadcastHashJoin Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [customsk#16] -Arguments: [c_customer_sk#1], [customsk#16], LeftSemi +Right output [1]: [customsk#14] +Arguments: [c_customer_sk#1], [customsk#14], LeftSemi -(29) CometProject +(23) CometProject Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Arguments: [c_current_cdemo_sk#2, c_current_addr_sk#3], [c_current_cdemo_sk#2, c_current_addr_sk#3] -(30) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#24, ca_state#25] +(24) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#20, ca_state#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(31) CometFilter -Input [2]: [ca_address_sk#24, ca_state#25] -Condition : isnotnull(ca_address_sk#24) +(25) CometFilter +Input [2]: [ca_address_sk#20, ca_state#21] +Condition : isnotnull(ca_address_sk#20) -(32) CometBroadcastExchange -Input [2]: [ca_address_sk#24, ca_state#25] -Arguments: [ca_address_sk#24, ca_state#25] +(26) CometBroadcastExchange +Input [2]: [ca_address_sk#20, ca_state#21] +Arguments: [ca_address_sk#20, ca_state#21] -(33) CometBroadcastHashJoin +(27) CometBroadcastHashJoin Left output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [2]: [ca_address_sk#24, ca_state#25] -Arguments: [c_current_addr_sk#3], [ca_address_sk#24], Inner +Right output [2]: [ca_address_sk#20, ca_state#21] +Arguments: [c_current_addr_sk#3], [ca_address_sk#20], Inner -(34) CometProject -Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#24, ca_state#25] -Arguments: [c_current_cdemo_sk#2, ca_state#25], [c_current_cdemo_sk#2, ca_state#25] +(28) CometProject +Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20, ca_state#21] +Arguments: [c_current_cdemo_sk#2, ca_state#21], [c_current_cdemo_sk#2, ca_state#21] -(35) Scan parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +(29) Scan parquet spark_catalog.default.customer_demographics +Output [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(36) CometFilter -Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Condition : isnotnull(cd_demo_sk#26) +(30) CometFilter +Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Condition : isnotnull(cd_demo_sk#22) -(37) CometBroadcastExchange -Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Arguments: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +(31) CometBroadcastExchange +Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Arguments: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -(38) CometBroadcastHashJoin -Left output [2]: [c_current_cdemo_sk#2, ca_state#25] -Right output [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#26], Inner +(32) CometBroadcastHashJoin +Left output [2]: [c_current_cdemo_sk#2, ca_state#21] +Right output [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#22], Inner -(39) CometProject -Input [8]: [c_current_cdemo_sk#2, ca_state#25, cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Arguments: [ca_state#25, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31], [ca_state#25, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +(33) CometProject +Input [8]: [c_current_cdemo_sk#2, ca_state#21, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Arguments: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27], [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -(40) CometHashAggregate -Input [6]: [ca_state#25, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Keys [6]: [ca_state#25, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#29), partial_max(cd_dep_count#29), partial_sum(cd_dep_count#29), partial_avg(cd_dep_employed_count#30), partial_max(cd_dep_employed_count#30), partial_sum(cd_dep_employed_count#30), partial_avg(cd_dep_college_count#31), partial_max(cd_dep_college_count#31), partial_sum(cd_dep_college_count#31)] +(34) CometHashAggregate +Input [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Keys [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#25), partial_max(cd_dep_count#25), partial_sum(cd_dep_count#25), partial_avg(cd_dep_employed_count#26), partial_max(cd_dep_employed_count#26), partial_sum(cd_dep_employed_count#26), partial_avg(cd_dep_college_count#27), partial_max(cd_dep_college_count#27), partial_sum(cd_dep_college_count#27)] -(41) ColumnarToRow [codegen id : 1] -Input [19]: [ca_state#25, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#32, sum#33, count#34, max#35, sum#36, sum#37, count#38, max#39, sum#40, sum#41, count#42, max#43, sum#44] +(35) ColumnarToRow [codegen id : 1] +Input [19]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#28, sum#29, count#30, max#31, sum#32, sum#33, count#34, max#35, sum#36, sum#37, count#38, max#39, sum#40] -(42) Exchange -Input [19]: [ca_state#25, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#32, sum#33, count#34, max#35, sum#36, sum#37, count#38, max#39, sum#40, sum#41, count#42, max#43, sum#44] -Arguments: hashpartitioning(ca_state#25, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, 5), ENSURE_REQUIREMENTS, [plan_id=1] +(36) Exchange +Input [19]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#28, sum#29, count#30, max#31, sum#32, sum#33, count#34, max#35, sum#36, sum#37, count#38, max#39, sum#40] +Arguments: hashpartitioning(ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(43) HashAggregate [codegen id : 2] -Input [19]: [ca_state#25, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#32, sum#33, count#34, max#35, sum#36, sum#37, count#38, max#39, sum#40, sum#41, count#42, max#43, sum#44] -Keys [6]: [ca_state#25, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Functions [10]: [count(1), avg(cd_dep_count#29), max(cd_dep_count#29), sum(cd_dep_count#29), avg(cd_dep_employed_count#30), max(cd_dep_employed_count#30), sum(cd_dep_employed_count#30), avg(cd_dep_college_count#31), max(cd_dep_college_count#31), sum(cd_dep_college_count#31)] -Aggregate Attributes [10]: [count(1)#45, avg(cd_dep_count#29)#46, max(cd_dep_count#29)#47, sum(cd_dep_count#29)#48, avg(cd_dep_employed_count#30)#49, max(cd_dep_employed_count#30)#50, sum(cd_dep_employed_count#30)#51, avg(cd_dep_college_count#31)#52, max(cd_dep_college_count#31)#53, sum(cd_dep_college_count#31)#54] -Results [18]: [ca_state#25, cd_gender#27, cd_marital_status#28, cd_dep_count#29, count(1)#45 AS cnt1#55, avg(cd_dep_count#29)#46 AS avg(cd_dep_count)#56, max(cd_dep_count#29)#47 AS max(cd_dep_count)#57, sum(cd_dep_count#29)#48 AS sum(cd_dep_count)#58, cd_dep_employed_count#30, count(1)#45 AS cnt2#59, avg(cd_dep_employed_count#30)#49 AS avg(cd_dep_employed_count)#60, max(cd_dep_employed_count#30)#50 AS max(cd_dep_employed_count)#61, sum(cd_dep_employed_count#30)#51 AS sum(cd_dep_employed_count)#62, cd_dep_college_count#31, count(1)#45 AS cnt3#63, avg(cd_dep_college_count#31)#52 AS avg(cd_dep_college_count)#64, max(cd_dep_college_count#31)#53 AS max(cd_dep_college_count)#65, sum(cd_dep_college_count#31)#54 AS sum(cd_dep_college_count)#66] +(37) HashAggregate [codegen id : 2] +Input [19]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#28, sum#29, count#30, max#31, sum#32, sum#33, count#34, max#35, sum#36, sum#37, count#38, max#39, sum#40] +Keys [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Functions [10]: [count(1), avg(cd_dep_count#25), max(cd_dep_count#25), sum(cd_dep_count#25), avg(cd_dep_employed_count#26), max(cd_dep_employed_count#26), sum(cd_dep_employed_count#26), avg(cd_dep_college_count#27), max(cd_dep_college_count#27), sum(cd_dep_college_count#27)] +Aggregate Attributes [10]: [count(1)#41, avg(cd_dep_count#25)#42, max(cd_dep_count#25)#43, sum(cd_dep_count#25)#44, avg(cd_dep_employed_count#26)#45, max(cd_dep_employed_count#26)#46, sum(cd_dep_employed_count#26)#47, avg(cd_dep_college_count#27)#48, max(cd_dep_college_count#27)#49, sum(cd_dep_college_count#27)#50] +Results [18]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, count(1)#41 AS cnt1#51, avg(cd_dep_count#25)#42 AS avg(cd_dep_count)#52, max(cd_dep_count#25)#43 AS max(cd_dep_count)#53, sum(cd_dep_count#25)#44 AS sum(cd_dep_count)#54, cd_dep_employed_count#26, count(1)#41 AS cnt2#55, avg(cd_dep_employed_count#26)#45 AS avg(cd_dep_employed_count)#56, max(cd_dep_employed_count#26)#46 AS max(cd_dep_employed_count)#57, sum(cd_dep_employed_count#26)#47 AS sum(cd_dep_employed_count)#58, cd_dep_college_count#27, count(1)#41 AS cnt3#59, avg(cd_dep_college_count#27)#48 AS avg(cd_dep_college_count)#60, max(cd_dep_college_count#27)#49 AS max(cd_dep_college_count)#61, sum(cd_dep_college_count#27)#50 AS sum(cd_dep_college_count)#62] -(44) TakeOrderedAndProject -Input [18]: [ca_state#25, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cnt1#55, avg(cd_dep_count)#56, max(cd_dep_count)#57, sum(cd_dep_count)#58, cd_dep_employed_count#30, cnt2#59, avg(cd_dep_employed_count)#60, max(cd_dep_employed_count)#61, sum(cd_dep_employed_count)#62, cd_dep_college_count#31, cnt3#63, avg(cd_dep_college_count)#64, max(cd_dep_college_count)#65, sum(cd_dep_college_count)#66] -Arguments: 100, [ca_state#25 ASC NULLS FIRST, cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, cd_dep_count#29 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [ca_state#25, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cnt1#55, avg(cd_dep_count)#56, max(cd_dep_count)#57, sum(cd_dep_count)#58, cd_dep_employed_count#30, cnt2#59, avg(cd_dep_employed_count)#60, max(cd_dep_employed_count)#61, sum(cd_dep_employed_count)#62, cd_dep_college_count#31, cnt3#63, avg(cd_dep_college_count)#64, max(cd_dep_college_count)#65, sum(cd_dep_college_count)#66] +(38) TakeOrderedAndProject +Input [18]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#51, avg(cd_dep_count)#52, max(cd_dep_count)#53, sum(cd_dep_count)#54, cd_dep_employed_count#26, cnt2#55, avg(cd_dep_employed_count)#56, max(cd_dep_employed_count)#57, sum(cd_dep_employed_count)#58, cd_dep_college_count#27, cnt3#59, avg(cd_dep_college_count)#60, max(cd_dep_college_count)#61, sum(cd_dep_college_count)#62] +Arguments: 100, [ca_state#21 ASC NULLS FIRST, cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_dep_count#25 ASC NULLS FIRST, cd_dep_employed_count#26 ASC NULLS FIRST, cd_dep_college_count#27 ASC NULLS FIRST], [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#51, avg(cd_dep_count)#52, max(cd_dep_count)#53, sum(cd_dep_count)#54, cd_dep_employed_count#26, cnt2#55, avg(cd_dep_employed_count)#56, max(cd_dep_employed_count)#57, sum(cd_dep_employed_count)#58, cd_dep_college_count#27, cnt3#59, avg(cd_dep_college_count)#60, max(cd_dep_college_count)#61, sum(cd_dep_college_count)#62] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (49) -+- * ColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan parquet spark_catalog.default.date_dim (45) +BroadcastExchange (43) ++- * ColumnarToRow (42) + +- CometProject (41) + +- CometFilter (40) + +- CometScan parquet spark_catalog.default.date_dim (39) -(45) Scan parquet spark_catalog.default.date_dim +(39) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#7, d_year#8, d_qoy#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(46) CometFilter +(40) CometFilter Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999)) AND (d_qoy#9 < 4)) AND isnotnull(d_date_sk#7)) -(47) CometProject +(41) CometProject Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(48) ColumnarToRow [codegen id : 1] +(42) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(49) BroadcastExchange +(43) BroadcastExchange Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 19 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt index e77816028..e3a91e471 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt @@ -38,21 +38,15 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #6 - CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + ReusedExchange [d_date_sk] #4 CometProject [cs_ship_customer_sk] [customsk] CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #7 - CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #8 + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange #6 CometFilter [ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange #9 + CometBroadcastExchange #7 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt index 5440f1724..985169db2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (85) -+- * Project (84) - +- * BroadcastHashJoin Inner BuildRight (83) - :- * Project (56) - : +- * BroadcastHashJoin Inner BuildRight (55) +TakeOrderedAndProject (45) ++- * Project (44) + +- * BroadcastHashJoin Inner BuildRight (43) + :- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) : :- * Project (28) : : +- * Filter (27) : : +- Window (26) @@ -32,58 +32,18 @@ TakeOrderedAndProject (85) : : +- CometBroadcastExchange (15) : : +- CometFilter (14) : : +- CometScan parquet spark_catalog.default.store (13) - : +- BroadcastExchange (54) - : +- * Project (53) - : +- Window (52) - : +- * Sort (51) - : +- Exchange (50) - : +- * HashAggregate (49) - : +- Exchange (48) - : +- * ColumnarToRow (47) - : +- CometHashAggregate (46) - : +- CometProject (45) - : +- CometBroadcastHashJoin (44) - : :- CometProject (40) - : : +- CometBroadcastHashJoin (39) - : : :- CometProject (35) - : : : +- CometBroadcastHashJoin (34) - : : : :- CometFilter (30) - : : : : +- CometScan parquet spark_catalog.default.item (29) - : : : +- CometBroadcastExchange (33) - : : : +- CometFilter (32) - : : : +- CometScan parquet spark_catalog.default.store_sales (31) - : : +- CometBroadcastExchange (38) - : : +- CometFilter (37) - : : +- CometScan parquet spark_catalog.default.date_dim (36) - : +- CometBroadcastExchange (43) - : +- CometFilter (42) - : +- CometScan parquet spark_catalog.default.store (41) - +- BroadcastExchange (82) - +- * Project (81) - +- Window (80) - +- * Sort (79) - +- Exchange (78) - +- * HashAggregate (77) - +- Exchange (76) - +- * ColumnarToRow (75) - +- CometHashAggregate (74) - +- CometProject (73) - +- CometBroadcastHashJoin (72) - :- CometProject (68) - : +- CometBroadcastHashJoin (67) - : :- CometProject (63) - : : +- CometBroadcastHashJoin (62) - : : :- CometFilter (58) - : : : +- CometScan parquet spark_catalog.default.item (57) - : : +- CometBroadcastExchange (61) - : : +- CometFilter (60) - : : +- CometScan parquet spark_catalog.default.store_sales (59) - : +- CometBroadcastExchange (66) - : +- CometFilter (65) - : +- CometScan parquet spark_catalog.default.date_dim (64) - +- CometBroadcastExchange (71) - +- CometFilter (70) - +- CometScan parquet spark_catalog.default.store (69) + : +- BroadcastExchange (35) + : +- * Project (34) + : +- Window (33) + : +- * Sort (32) + : +- Exchange (31) + : +- * HashAggregate (30) + : +- ReusedExchange (29) + +- BroadcastExchange (42) + +- * Project (41) + +- Window (40) + +- * Sort (39) + +- ReusedExchange (38) (1) Scan parquet spark_catalog.default.item @@ -217,305 +177,104 @@ Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.0000 Output [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19] Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] -(29) Scan parquet spark_catalog.default.item -Output [3]: [i_item_sk#21, i_brand#22, i_category#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_category), IsNotNull(i_brand)] -ReadSchema: struct +(29) ReusedExchange [Reuses operator id: 20] +Output [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum#27] -(30) CometFilter -Input [3]: [i_item_sk#21, i_brand#22, i_category#23] -Condition : ((isnotnull(i_item_sk#21) AND isnotnull(i_category#23)) AND isnotnull(i_brand#22)) +(30) HashAggregate [codegen id : 6] +Input [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum#27] +Keys [6]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26] +Functions [1]: [sum(UnscaledValue(ss_sales_price#28))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#28))#16] +Results [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, MakeDecimal(sum(UnscaledValue(ss_sales_price#28))#16,17,2) AS sum_sales#17] -(31) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#24, ss_store_sk#25, ss_sales_price#26, ss_sold_date_sk#27] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#27), dynamicpruningexpression(ss_sold_date_sk#27 IN dynamicpruning#28)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct +(31) Exchange +Input [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17] +Arguments: hashpartitioning(i_category#21, i_brand#22, s_store_name#23, s_company_name#24, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(32) CometFilter -Input [4]: [ss_item_sk#24, ss_store_sk#25, ss_sales_price#26, ss_sold_date_sk#27] -Condition : (isnotnull(ss_item_sk#24) AND isnotnull(ss_store_sk#25)) +(32) Sort [codegen id : 7] +Input [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17] +Arguments: [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, s_store_name#23 ASC NULLS FIRST, s_company_name#24 ASC NULLS FIRST, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST], false, 0 -(33) CometBroadcastExchange -Input [4]: [ss_item_sk#24, ss_store_sk#25, ss_sales_price#26, ss_sold_date_sk#27] -Arguments: [ss_item_sk#24, ss_store_sk#25, ss_sales_price#26, ss_sold_date_sk#27] +(33) Window +Input [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17] +Arguments: [rank(d_year#25, d_moy#26) windowspecdefinition(i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#29], [i_category#21, i_brand#22, s_store_name#23, s_company_name#24], [d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] -(34) CometBroadcastHashJoin -Left output [3]: [i_item_sk#21, i_brand#22, i_category#23] -Right output [4]: [ss_item_sk#24, ss_store_sk#25, ss_sales_price#26, ss_sold_date_sk#27] -Arguments: [i_item_sk#21], [ss_item_sk#24], Inner +(34) Project [codegen id : 8] +Output [6]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, sum_sales#17 AS sum_sales#30, rn#29] +Input [8]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17, rn#29] -(35) CometProject -Input [7]: [i_item_sk#21, i_brand#22, i_category#23, ss_item_sk#24, ss_store_sk#25, ss_sales_price#26, ss_sold_date_sk#27] -Arguments: [i_brand#22, i_category#23, ss_store_sk#25, ss_sales_price#26, ss_sold_date_sk#27], [i_brand#22, i_category#23, ss_store_sk#25, ss_sales_price#26, ss_sold_date_sk#27] +(35) BroadcastExchange +Input [6]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, sum_sales#30, rn#29] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] -(36) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#29, d_year#30, d_moy#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(37) CometFilter -Input [3]: [d_date_sk#29, d_year#30, d_moy#31] -Condition : ((((d_year#30 = 1999) OR ((d_year#30 = 1998) AND (d_moy#31 = 12))) OR ((d_year#30 = 2000) AND (d_moy#31 = 1))) AND isnotnull(d_date_sk#29)) - -(38) CometBroadcastExchange -Input [3]: [d_date_sk#29, d_year#30, d_moy#31] -Arguments: [d_date_sk#29, d_year#30, d_moy#31] - -(39) CometBroadcastHashJoin -Left output [5]: [i_brand#22, i_category#23, ss_store_sk#25, ss_sales_price#26, ss_sold_date_sk#27] -Right output [3]: [d_date_sk#29, d_year#30, d_moy#31] -Arguments: [ss_sold_date_sk#27], [d_date_sk#29], Inner - -(40) CometProject -Input [8]: [i_brand#22, i_category#23, ss_store_sk#25, ss_sales_price#26, ss_sold_date_sk#27, d_date_sk#29, d_year#30, d_moy#31] -Arguments: [i_brand#22, i_category#23, ss_store_sk#25, ss_sales_price#26, d_year#30, d_moy#31], [i_brand#22, i_category#23, ss_store_sk#25, ss_sales_price#26, d_year#30, d_moy#31] - -(41) Scan parquet spark_catalog.default.store -Output [3]: [s_store_sk#32, s_store_name#33, s_company_name#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_company_name)] -ReadSchema: struct - -(42) CometFilter -Input [3]: [s_store_sk#32, s_store_name#33, s_company_name#34] -Condition : ((isnotnull(s_store_sk#32) AND isnotnull(s_store_name#33)) AND isnotnull(s_company_name#34)) - -(43) CometBroadcastExchange -Input [3]: [s_store_sk#32, s_store_name#33, s_company_name#34] -Arguments: [s_store_sk#32, s_store_name#33, s_company_name#34] - -(44) CometBroadcastHashJoin -Left output [6]: [i_brand#22, i_category#23, ss_store_sk#25, ss_sales_price#26, d_year#30, d_moy#31] -Right output [3]: [s_store_sk#32, s_store_name#33, s_company_name#34] -Arguments: [ss_store_sk#25], [s_store_sk#32], Inner - -(45) CometProject -Input [9]: [i_brand#22, i_category#23, ss_store_sk#25, ss_sales_price#26, d_year#30, d_moy#31, s_store_sk#32, s_store_name#33, s_company_name#34] -Arguments: [i_brand#22, i_category#23, ss_sales_price#26, d_year#30, d_moy#31, s_store_name#33, s_company_name#34], [i_brand#22, i_category#23, ss_sales_price#26, d_year#30, d_moy#31, s_store_name#33, s_company_name#34] - -(46) CometHashAggregate -Input [7]: [i_brand#22, i_category#23, ss_sales_price#26, d_year#30, d_moy#31, s_store_name#33, s_company_name#34] -Keys [6]: [i_category#23, i_brand#22, s_store_name#33, s_company_name#34, d_year#30, d_moy#31] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#26))] - -(47) ColumnarToRow [codegen id : 5] -Input [7]: [i_category#23, i_brand#22, s_store_name#33, s_company_name#34, d_year#30, d_moy#31, sum#35] - -(48) Exchange -Input [7]: [i_category#23, i_brand#22, s_store_name#33, s_company_name#34, d_year#30, d_moy#31, sum#35] -Arguments: hashpartitioning(i_category#23, i_brand#22, s_store_name#33, s_company_name#34, d_year#30, d_moy#31, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(49) HashAggregate [codegen id : 6] -Input [7]: [i_category#23, i_brand#22, s_store_name#33, s_company_name#34, d_year#30, d_moy#31, sum#35] -Keys [6]: [i_category#23, i_brand#22, s_store_name#33, s_company_name#34, d_year#30, d_moy#31] -Functions [1]: [sum(UnscaledValue(ss_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#26))#16] -Results [7]: [i_category#23, i_brand#22, s_store_name#33, s_company_name#34, d_year#30, d_moy#31, MakeDecimal(sum(UnscaledValue(ss_sales_price#26))#16,17,2) AS sum_sales#17] - -(50) Exchange -Input [7]: [i_category#23, i_brand#22, s_store_name#33, s_company_name#34, d_year#30, d_moy#31, sum_sales#17] -Arguments: hashpartitioning(i_category#23, i_brand#22, s_store_name#33, s_company_name#34, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(51) Sort [codegen id : 7] -Input [7]: [i_category#23, i_brand#22, s_store_name#33, s_company_name#34, d_year#30, d_moy#31, sum_sales#17] -Arguments: [i_category#23 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, s_store_name#33 ASC NULLS FIRST, s_company_name#34 ASC NULLS FIRST, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST], false, 0 - -(52) Window -Input [7]: [i_category#23, i_brand#22, s_store_name#33, s_company_name#34, d_year#30, d_moy#31, sum_sales#17] -Arguments: [rank(d_year#30, d_moy#31) windowspecdefinition(i_category#23, i_brand#22, s_store_name#33, s_company_name#34, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#23, i_brand#22, s_store_name#33, s_company_name#34], [d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] - -(53) Project [codegen id : 8] -Output [6]: [i_category#23, i_brand#22, s_store_name#33, s_company_name#34, sum_sales#17 AS sum_sales#37, rn#36] -Input [8]: [i_category#23, i_brand#22, s_store_name#33, s_company_name#34, d_year#30, d_moy#31, sum_sales#17, rn#36] - -(54) BroadcastExchange -Input [6]: [i_category#23, i_brand#22, s_store_name#33, s_company_name#34, sum_sales#37, rn#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=5] - -(55) BroadcastHashJoin [codegen id : 13] +(36) BroadcastHashJoin [codegen id : 13] Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#19] -Right keys [5]: [i_category#23, i_brand#22, s_store_name#33, s_company_name#34, (rn#36 + 1)] +Right keys [5]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, (rn#29 + 1)] Join type: Inner Join condition: None -(56) Project [codegen id : 13] -Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#37] -Input [15]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#23, i_brand#22, s_store_name#33, s_company_name#34, sum_sales#37, rn#36] - -(57) Scan parquet spark_catalog.default.item -Output [3]: [i_item_sk#38, i_brand#39, i_category#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_category), IsNotNull(i_brand)] -ReadSchema: struct - -(58) CometFilter -Input [3]: [i_item_sk#38, i_brand#39, i_category#40] -Condition : ((isnotnull(i_item_sk#38) AND isnotnull(i_category#40)) AND isnotnull(i_brand#39)) +(37) Project [codegen id : 13] +Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#30] +Input [15]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#21, i_brand#22, s_store_name#23, s_company_name#24, sum_sales#30, rn#29] -(59) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#41, ss_store_sk#42, ss_sales_price#43, ss_sold_date_sk#44] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#44), dynamicpruningexpression(ss_sold_date_sk#44 IN dynamicpruning#45)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct +(38) ReusedExchange [Reuses operator id: 31] +Output [7]: [i_category#31, i_brand#32, s_store_name#33, s_company_name#34, d_year#35, d_moy#36, sum_sales#17] -(60) CometFilter -Input [4]: [ss_item_sk#41, ss_store_sk#42, ss_sales_price#43, ss_sold_date_sk#44] -Condition : (isnotnull(ss_item_sk#41) AND isnotnull(ss_store_sk#42)) +(39) Sort [codegen id : 11] +Input [7]: [i_category#31, i_brand#32, s_store_name#33, s_company_name#34, d_year#35, d_moy#36, sum_sales#17] +Arguments: [i_category#31 ASC NULLS FIRST, i_brand#32 ASC NULLS FIRST, s_store_name#33 ASC NULLS FIRST, s_company_name#34 ASC NULLS FIRST, d_year#35 ASC NULLS FIRST, d_moy#36 ASC NULLS FIRST], false, 0 -(61) CometBroadcastExchange -Input [4]: [ss_item_sk#41, ss_store_sk#42, ss_sales_price#43, ss_sold_date_sk#44] -Arguments: [ss_item_sk#41, ss_store_sk#42, ss_sales_price#43, ss_sold_date_sk#44] +(40) Window +Input [7]: [i_category#31, i_brand#32, s_store_name#33, s_company_name#34, d_year#35, d_moy#36, sum_sales#17] +Arguments: [rank(d_year#35, d_moy#36) windowspecdefinition(i_category#31, i_brand#32, s_store_name#33, s_company_name#34, d_year#35 ASC NULLS FIRST, d_moy#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#31, i_brand#32, s_store_name#33, s_company_name#34], [d_year#35 ASC NULLS FIRST, d_moy#36 ASC NULLS FIRST] -(62) CometBroadcastHashJoin -Left output [3]: [i_item_sk#38, i_brand#39, i_category#40] -Right output [4]: [ss_item_sk#41, ss_store_sk#42, ss_sales_price#43, ss_sold_date_sk#44] -Arguments: [i_item_sk#38], [ss_item_sk#41], Inner +(41) Project [codegen id : 12] +Output [6]: [i_category#31, i_brand#32, s_store_name#33, s_company_name#34, sum_sales#17 AS sum_sales#38, rn#37] +Input [8]: [i_category#31, i_brand#32, s_store_name#33, s_company_name#34, d_year#35, d_moy#36, sum_sales#17, rn#37] -(63) CometProject -Input [7]: [i_item_sk#38, i_brand#39, i_category#40, ss_item_sk#41, ss_store_sk#42, ss_sales_price#43, ss_sold_date_sk#44] -Arguments: [i_brand#39, i_category#40, ss_store_sk#42, ss_sales_price#43, ss_sold_date_sk#44], [i_brand#39, i_category#40, ss_store_sk#42, ss_sales_price#43, ss_sold_date_sk#44] +(42) BroadcastExchange +Input [6]: [i_category#31, i_brand#32, s_store_name#33, s_company_name#34, sum_sales#38, rn#37] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] -(64) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#46, d_year#47, d_moy#48] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(65) CometFilter -Input [3]: [d_date_sk#46, d_year#47, d_moy#48] -Condition : ((((d_year#47 = 1999) OR ((d_year#47 = 1998) AND (d_moy#48 = 12))) OR ((d_year#47 = 2000) AND (d_moy#48 = 1))) AND isnotnull(d_date_sk#46)) - -(66) CometBroadcastExchange -Input [3]: [d_date_sk#46, d_year#47, d_moy#48] -Arguments: [d_date_sk#46, d_year#47, d_moy#48] - -(67) CometBroadcastHashJoin -Left output [5]: [i_brand#39, i_category#40, ss_store_sk#42, ss_sales_price#43, ss_sold_date_sk#44] -Right output [3]: [d_date_sk#46, d_year#47, d_moy#48] -Arguments: [ss_sold_date_sk#44], [d_date_sk#46], Inner - -(68) CometProject -Input [8]: [i_brand#39, i_category#40, ss_store_sk#42, ss_sales_price#43, ss_sold_date_sk#44, d_date_sk#46, d_year#47, d_moy#48] -Arguments: [i_brand#39, i_category#40, ss_store_sk#42, ss_sales_price#43, d_year#47, d_moy#48], [i_brand#39, i_category#40, ss_store_sk#42, ss_sales_price#43, d_year#47, d_moy#48] - -(69) Scan parquet spark_catalog.default.store -Output [3]: [s_store_sk#49, s_store_name#50, s_company_name#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_company_name)] -ReadSchema: struct - -(70) CometFilter -Input [3]: [s_store_sk#49, s_store_name#50, s_company_name#51] -Condition : ((isnotnull(s_store_sk#49) AND isnotnull(s_store_name#50)) AND isnotnull(s_company_name#51)) - -(71) CometBroadcastExchange -Input [3]: [s_store_sk#49, s_store_name#50, s_company_name#51] -Arguments: [s_store_sk#49, s_store_name#50, s_company_name#51] - -(72) CometBroadcastHashJoin -Left output [6]: [i_brand#39, i_category#40, ss_store_sk#42, ss_sales_price#43, d_year#47, d_moy#48] -Right output [3]: [s_store_sk#49, s_store_name#50, s_company_name#51] -Arguments: [ss_store_sk#42], [s_store_sk#49], Inner - -(73) CometProject -Input [9]: [i_brand#39, i_category#40, ss_store_sk#42, ss_sales_price#43, d_year#47, d_moy#48, s_store_sk#49, s_store_name#50, s_company_name#51] -Arguments: [i_brand#39, i_category#40, ss_sales_price#43, d_year#47, d_moy#48, s_store_name#50, s_company_name#51], [i_brand#39, i_category#40, ss_sales_price#43, d_year#47, d_moy#48, s_store_name#50, s_company_name#51] - -(74) CometHashAggregate -Input [7]: [i_brand#39, i_category#40, ss_sales_price#43, d_year#47, d_moy#48, s_store_name#50, s_company_name#51] -Keys [6]: [i_category#40, i_brand#39, s_store_name#50, s_company_name#51, d_year#47, d_moy#48] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#43))] - -(75) ColumnarToRow [codegen id : 9] -Input [7]: [i_category#40, i_brand#39, s_store_name#50, s_company_name#51, d_year#47, d_moy#48, sum#52] - -(76) Exchange -Input [7]: [i_category#40, i_brand#39, s_store_name#50, s_company_name#51, d_year#47, d_moy#48, sum#52] -Arguments: hashpartitioning(i_category#40, i_brand#39, s_store_name#50, s_company_name#51, d_year#47, d_moy#48, 5), ENSURE_REQUIREMENTS, [plan_id=6] - -(77) HashAggregate [codegen id : 10] -Input [7]: [i_category#40, i_brand#39, s_store_name#50, s_company_name#51, d_year#47, d_moy#48, sum#52] -Keys [6]: [i_category#40, i_brand#39, s_store_name#50, s_company_name#51, d_year#47, d_moy#48] -Functions [1]: [sum(UnscaledValue(ss_sales_price#43))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#43))#16] -Results [7]: [i_category#40, i_brand#39, s_store_name#50, s_company_name#51, d_year#47, d_moy#48, MakeDecimal(sum(UnscaledValue(ss_sales_price#43))#16,17,2) AS sum_sales#17] - -(78) Exchange -Input [7]: [i_category#40, i_brand#39, s_store_name#50, s_company_name#51, d_year#47, d_moy#48, sum_sales#17] -Arguments: hashpartitioning(i_category#40, i_brand#39, s_store_name#50, s_company_name#51, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(79) Sort [codegen id : 11] -Input [7]: [i_category#40, i_brand#39, s_store_name#50, s_company_name#51, d_year#47, d_moy#48, sum_sales#17] -Arguments: [i_category#40 ASC NULLS FIRST, i_brand#39 ASC NULLS FIRST, s_store_name#50 ASC NULLS FIRST, s_company_name#51 ASC NULLS FIRST, d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST], false, 0 - -(80) Window -Input [7]: [i_category#40, i_brand#39, s_store_name#50, s_company_name#51, d_year#47, d_moy#48, sum_sales#17] -Arguments: [rank(d_year#47, d_moy#48) windowspecdefinition(i_category#40, i_brand#39, s_store_name#50, s_company_name#51, d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#53], [i_category#40, i_brand#39, s_store_name#50, s_company_name#51], [d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST] - -(81) Project [codegen id : 12] -Output [6]: [i_category#40, i_brand#39, s_store_name#50, s_company_name#51, sum_sales#17 AS sum_sales#54, rn#53] -Input [8]: [i_category#40, i_brand#39, s_store_name#50, s_company_name#51, d_year#47, d_moy#48, sum_sales#17, rn#53] - -(82) BroadcastExchange -Input [6]: [i_category#40, i_brand#39, s_store_name#50, s_company_name#51, sum_sales#54, rn#53] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=8] - -(83) BroadcastHashJoin [codegen id : 13] +(43) BroadcastHashJoin [codegen id : 13] Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#19] -Right keys [5]: [i_category#40, i_brand#39, s_store_name#50, s_company_name#51, (rn#53 - 1)] +Right keys [5]: [i_category#31, i_brand#32, s_store_name#33, s_company_name#34, (rn#37 - 1)] Join type: Inner Join condition: None -(84) Project [codegen id : 13] -Output [7]: [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, sum_sales#37 AS psum#55, sum_sales#54 AS nsum#56] -Input [16]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#37, i_category#40, i_brand#39, s_store_name#50, s_company_name#51, sum_sales#54, rn#53] +(44) Project [codegen id : 13] +Output [7]: [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, sum_sales#30 AS psum#39, sum_sales#38 AS nsum#40] +Input [16]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#30, i_category#31, i_brand#32, s_store_name#33, s_company_name#34, sum_sales#38, rn#37] -(85) TakeOrderedAndProject -Input [7]: [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#55, nsum#56] -Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#55, nsum#56] +(45) TakeOrderedAndProject +Input [7]: [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#39, nsum#40] +Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#39, nsum#40] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (89) -+- * ColumnarToRow (88) - +- CometFilter (87) - +- CometScan parquet spark_catalog.default.date_dim (86) +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) -(86) Scan parquet spark_catalog.default.date_dim +(46) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#9, d_year#10, d_moy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(87) CometFilter +(47) CometFilter Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) -(88) ColumnarToRow [codegen id : 1] +(48) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -(89) BroadcastExchange +(49) BroadcastExchange Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#27 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 59 Hosting Expression = ss_sold_date_sk#44 IN dynamicpruning#8 +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-v2_7/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt index bda1dda8d..a85302cc5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt @@ -61,31 +61,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, WholeStageCodegen (6) HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #9 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometFilter [i_item_sk,i_category,i_brand] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #10 - CometFilter [ss_item_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #11 - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #12 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 InputAdapter - BroadcastExchange #13 + BroadcastExchange #9 WholeStageCodegen (12) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter @@ -93,30 +71,4 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, WholeStageCodegen (11) Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name] #14 - WholeStageCodegen (10) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] - InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #15 - WholeStageCodegen (9) - ColumnarToRow - InputAdapter - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometFilter [i_item_sk,i_category,i_brand] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #16 - CometFilter [ss_item_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #17 - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #18 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt index e26d5280c..b1d558bc1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (83) -+- * HashAggregate (82) - +- Exchange (81) - +- * HashAggregate (80) - +- Union (79) +TakeOrderedAndProject (77) ++- * HashAggregate (76) + +- Exchange (75) + +- * HashAggregate (74) + +- Union (73) :- * Project (26) : +- * Filter (25) : +- Window (24) @@ -30,19 +30,19 @@ TakeOrderedAndProject (83) : +- CometProject (12) : +- CometFilter (11) : +- CometScan parquet spark_catalog.default.date_dim (10) - :- * Project (52) - : +- * Filter (51) - : +- Window (50) - : +- * Sort (49) - : +- Window (48) - : +- * Sort (47) - : +- Exchange (46) - : +- * HashAggregate (45) - : +- Exchange (44) - : +- * ColumnarToRow (43) - : +- CometHashAggregate (42) - : +- CometProject (41) - : +- CometBroadcastHashJoin (40) + :- * Project (49) + : +- * Filter (48) + : +- Window (47) + : +- * Sort (46) + : +- Window (45) + : +- * Sort (44) + : +- Exchange (43) + : +- * HashAggregate (42) + : +- Exchange (41) + : +- * ColumnarToRow (40) + : +- CometHashAggregate (39) + : +- CometProject (38) + : +- CometBroadcastHashJoin (37) : :- CometProject (35) : : +- CometBroadcastHashJoin (34) : : :- CometBroadcastExchange (30) @@ -52,36 +52,30 @@ TakeOrderedAndProject (83) : : +- CometProject (33) : : +- CometFilter (32) : : +- CometScan parquet spark_catalog.default.catalog_returns (31) - : +- CometBroadcastExchange (39) - : +- CometProject (38) - : +- CometFilter (37) - : +- CometScan parquet spark_catalog.default.date_dim (36) - +- * Project (78) - +- * Filter (77) - +- Window (76) - +- * Sort (75) - +- Window (74) - +- * Sort (73) - +- Exchange (72) - +- * HashAggregate (71) - +- Exchange (70) - +- * ColumnarToRow (69) - +- CometHashAggregate (68) - +- CometProject (67) - +- CometBroadcastHashJoin (66) - :- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometBroadcastExchange (56) - : : +- CometProject (55) - : : +- CometFilter (54) - : : +- CometScan parquet spark_catalog.default.store_sales (53) - : +- CometProject (59) - : +- CometFilter (58) - : +- CometScan parquet spark_catalog.default.store_returns (57) - +- CometBroadcastExchange (65) - +- CometProject (64) - +- CometFilter (63) - +- CometScan parquet spark_catalog.default.date_dim (62) + : +- ReusedExchange (36) + +- * Project (72) + +- * Filter (71) + +- Window (70) + +- * Sort (69) + +- Window (68) + +- * Sort (67) + +- Exchange (66) + +- * HashAggregate (65) + +- Exchange (64) + +- * ColumnarToRow (63) + +- CometHashAggregate (62) + +- CometProject (61) + +- CometBroadcastHashJoin (60) + :- CometProject (58) + : +- CometBroadcastHashJoin (57) + : :- CometBroadcastExchange (53) + : : +- CometProject (52) + : : +- CometFilter (51) + : : +- CometScan parquet spark_catalog.default.store_sales (50) + : +- CometProject (56) + : +- CometFilter (55) + : +- CometScan parquet spark_catalog.default.store_returns (54) + +- ReusedExchange (59) (1) Scan parquet spark_catalog.default.web_sales @@ -247,258 +241,226 @@ Arguments: [cs_order_number#33, cs_item_sk#32], [cr_order_number#40, cr_item_sk# Input [9]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42] Arguments: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_return_quantity#41, cr_return_amount#42], [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_return_quantity#41, cr_return_amount#42] -(36) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#44, d_year#45, d_moy#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(37) CometFilter -Input [3]: [d_date_sk#44, d_year#45, d_moy#46] -Condition : ((((isnotnull(d_year#45) AND isnotnull(d_moy#46)) AND (d_year#45 = 2001)) AND (d_moy#46 = 12)) AND isnotnull(d_date_sk#44)) - -(38) CometProject -Input [3]: [d_date_sk#44, d_year#45, d_moy#46] -Arguments: [d_date_sk#44], [d_date_sk#44] +(36) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_date_sk#44] -(39) CometBroadcastExchange -Input [1]: [d_date_sk#44] -Arguments: [d_date_sk#44] - -(40) CometBroadcastHashJoin +(37) CometBroadcastHashJoin Left output [6]: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_return_quantity#41, cr_return_amount#42] Right output [1]: [d_date_sk#44] Arguments: [cs_sold_date_sk#37], [d_date_sk#44], Inner -(41) CometProject +(38) CometProject Input [7]: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_return_quantity#41, cr_return_amount#42, d_date_sk#44] Arguments: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cr_return_quantity#41, cr_return_amount#42], [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cr_return_quantity#41, cr_return_amount#42] -(42) CometHashAggregate +(39) CometHashAggregate Input [5]: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cr_return_quantity#41, cr_return_amount#42] Keys [1]: [cs_item_sk#32] Functions [4]: [partial_sum(coalesce(cr_return_quantity#41, 0)), partial_sum(coalesce(cs_quantity#34, 0)), partial_sum(coalesce(cast(cr_return_amount#42 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#35 as decimal(12,2)), 0.00))] -(43) ColumnarToRow [codegen id : 6] -Input [7]: [cs_item_sk#32, sum#47, sum#48, sum#49, isEmpty#50, sum#51, isEmpty#52] +(40) ColumnarToRow [codegen id : 6] +Input [7]: [cs_item_sk#32, sum#45, sum#46, sum#47, isEmpty#48, sum#49, isEmpty#50] -(44) Exchange -Input [7]: [cs_item_sk#32, sum#47, sum#48, sum#49, isEmpty#50, sum#51, isEmpty#52] +(41) Exchange +Input [7]: [cs_item_sk#32, sum#45, sum#46, sum#47, isEmpty#48, sum#49, isEmpty#50] Arguments: hashpartitioning(cs_item_sk#32, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(45) HashAggregate [codegen id : 7] -Input [7]: [cs_item_sk#32, sum#47, sum#48, sum#49, isEmpty#50, sum#51, isEmpty#52] +(42) HashAggregate [codegen id : 7] +Input [7]: [cs_item_sk#32, sum#45, sum#46, sum#47, isEmpty#48, sum#49, isEmpty#50] Keys [1]: [cs_item_sk#32] Functions [4]: [sum(coalesce(cr_return_quantity#41, 0)), sum(coalesce(cs_quantity#34, 0)), sum(coalesce(cast(cr_return_amount#42 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#35 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#41, 0))#53, sum(coalesce(cs_quantity#34, 0))#54, sum(coalesce(cast(cr_return_amount#42 as decimal(12,2)), 0.00))#55, sum(coalesce(cast(cs_net_paid#35 as decimal(12,2)), 0.00))#56] -Results [3]: [cs_item_sk#32 AS item#57, (cast(sum(coalesce(cr_return_quantity#41, 0))#53 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#34, 0))#54 as decimal(15,4))) AS return_ratio#58, (cast(sum(coalesce(cast(cr_return_amount#42 as decimal(12,2)), 0.00))#55 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#35 as decimal(12,2)), 0.00))#56 as decimal(15,4))) AS currency_ratio#59] +Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#41, 0))#51, sum(coalesce(cs_quantity#34, 0))#52, sum(coalesce(cast(cr_return_amount#42 as decimal(12,2)), 0.00))#53, sum(coalesce(cast(cs_net_paid#35 as decimal(12,2)), 0.00))#54] +Results [3]: [cs_item_sk#32 AS item#55, (cast(sum(coalesce(cr_return_quantity#41, 0))#51 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#34, 0))#52 as decimal(15,4))) AS return_ratio#56, (cast(sum(coalesce(cast(cr_return_amount#42 as decimal(12,2)), 0.00))#53 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#35 as decimal(12,2)), 0.00))#54 as decimal(15,4))) AS currency_ratio#57] -(46) Exchange -Input [3]: [item#57, return_ratio#58, currency_ratio#59] +(43) Exchange +Input [3]: [item#55, return_ratio#56, currency_ratio#57] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] -(47) Sort [codegen id : 8] -Input [3]: [item#57, return_ratio#58, currency_ratio#59] -Arguments: [return_ratio#58 ASC NULLS FIRST], false, 0 +(44) Sort [codegen id : 8] +Input [3]: [item#55, return_ratio#56, currency_ratio#57] +Arguments: [return_ratio#56 ASC NULLS FIRST], false, 0 -(48) Window -Input [3]: [item#57, return_ratio#58, currency_ratio#59] -Arguments: [rank(return_ratio#58) windowspecdefinition(return_ratio#58 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#60], [return_ratio#58 ASC NULLS FIRST] +(45) Window +Input [3]: [item#55, return_ratio#56, currency_ratio#57] +Arguments: [rank(return_ratio#56) windowspecdefinition(return_ratio#56 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#58], [return_ratio#56 ASC NULLS FIRST] -(49) Sort [codegen id : 9] -Input [4]: [item#57, return_ratio#58, currency_ratio#59, return_rank#60] -Arguments: [currency_ratio#59 ASC NULLS FIRST], false, 0 +(46) Sort [codegen id : 9] +Input [4]: [item#55, return_ratio#56, currency_ratio#57, return_rank#58] +Arguments: [currency_ratio#57 ASC NULLS FIRST], false, 0 -(50) Window -Input [4]: [item#57, return_ratio#58, currency_ratio#59, return_rank#60] -Arguments: [rank(currency_ratio#59) windowspecdefinition(currency_ratio#59 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#61], [currency_ratio#59 ASC NULLS FIRST] +(47) Window +Input [4]: [item#55, return_ratio#56, currency_ratio#57, return_rank#58] +Arguments: [rank(currency_ratio#57) windowspecdefinition(currency_ratio#57 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#59], [currency_ratio#57 ASC NULLS FIRST] -(51) Filter [codegen id : 10] -Input [5]: [item#57, return_ratio#58, currency_ratio#59, return_rank#60, currency_rank#61] -Condition : ((return_rank#60 <= 10) OR (currency_rank#61 <= 10)) +(48) Filter [codegen id : 10] +Input [5]: [item#55, return_ratio#56, currency_ratio#57, return_rank#58, currency_rank#59] +Condition : ((return_rank#58 <= 10) OR (currency_rank#59 <= 10)) -(52) Project [codegen id : 10] -Output [5]: [catalog AS channel#62, item#57, return_ratio#58, return_rank#60, currency_rank#61] -Input [5]: [item#57, return_ratio#58, currency_ratio#59, return_rank#60, currency_rank#61] +(49) Project [codegen id : 10] +Output [5]: [catalog AS channel#60, item#55, return_ratio#56, return_rank#58, currency_rank#59] +Input [5]: [item#55, return_ratio#56, currency_ratio#57, return_rank#58, currency_rank#59] -(53) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_net_profit#67, ss_sold_date_sk#68] +(50) Scan parquet spark_catalog.default.store_sales +Output [6]: [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_net_profit#65, ss_sold_date_sk#66] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#68), dynamicpruningexpression(ss_sold_date_sk#68 IN dynamicpruning#69)] +PartitionFilters: [isnotnull(ss_sold_date_sk#66), dynamicpruningexpression(ss_sold_date_sk#66 IN dynamicpruning#67)] PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(54) CometFilter -Input [6]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_net_profit#67, ss_sold_date_sk#68] -Condition : (((((((isnotnull(ss_net_profit#67) AND isnotnull(ss_net_paid#66)) AND isnotnull(ss_quantity#65)) AND (ss_net_profit#67 > 1.00)) AND (ss_net_paid#66 > 0.00)) AND (ss_quantity#65 > 0)) AND isnotnull(ss_ticket_number#64)) AND isnotnull(ss_item_sk#63)) +(51) CometFilter +Input [6]: [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_net_profit#65, ss_sold_date_sk#66] +Condition : (((((((isnotnull(ss_net_profit#65) AND isnotnull(ss_net_paid#64)) AND isnotnull(ss_quantity#63)) AND (ss_net_profit#65 > 1.00)) AND (ss_net_paid#64 > 0.00)) AND (ss_quantity#63 > 0)) AND isnotnull(ss_ticket_number#62)) AND isnotnull(ss_item_sk#61)) -(55) CometProject -Input [6]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_net_profit#67, ss_sold_date_sk#68] -Arguments: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68], [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68] +(52) CometProject +Input [6]: [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_net_profit#65, ss_sold_date_sk#66] +Arguments: [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66], [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66] -(56) CometBroadcastExchange -Input [5]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68] -Arguments: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68] +(53) CometBroadcastExchange +Input [5]: [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66] +Arguments: [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66] -(57) Scan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73, sr_returned_date_sk#74] +(54) Scan parquet spark_catalog.default.store_returns +Output [5]: [sr_item_sk#68, sr_ticket_number#69, sr_return_quantity#70, sr_return_amt#71, sr_returned_date_sk#72] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(58) CometFilter -Input [5]: [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73, sr_returned_date_sk#74] -Condition : (((isnotnull(sr_return_amt#73) AND (sr_return_amt#73 > 10000.00)) AND isnotnull(sr_ticket_number#71)) AND isnotnull(sr_item_sk#70)) +(55) CometFilter +Input [5]: [sr_item_sk#68, sr_ticket_number#69, sr_return_quantity#70, sr_return_amt#71, sr_returned_date_sk#72] +Condition : (((isnotnull(sr_return_amt#71) AND (sr_return_amt#71 > 10000.00)) AND isnotnull(sr_ticket_number#69)) AND isnotnull(sr_item_sk#68)) -(59) CometProject -Input [5]: [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73, sr_returned_date_sk#74] -Arguments: [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73], [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73] +(56) CometProject +Input [5]: [sr_item_sk#68, sr_ticket_number#69, sr_return_quantity#70, sr_return_amt#71, sr_returned_date_sk#72] +Arguments: [sr_item_sk#68, sr_ticket_number#69, sr_return_quantity#70, sr_return_amt#71], [sr_item_sk#68, sr_ticket_number#69, sr_return_quantity#70, sr_return_amt#71] -(60) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68] -Right output [4]: [sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73] -Arguments: [ss_ticket_number#64, ss_item_sk#63], [sr_ticket_number#71, sr_item_sk#70], Inner +(57) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66] +Right output [4]: [sr_item_sk#68, sr_ticket_number#69, sr_return_quantity#70, sr_return_amt#71] +Arguments: [ss_ticket_number#62, ss_item_sk#61], [sr_ticket_number#69, sr_item_sk#68], Inner -(61) CometProject -Input [9]: [ss_item_sk#63, ss_ticket_number#64, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68, sr_item_sk#70, sr_ticket_number#71, sr_return_quantity#72, sr_return_amt#73] -Arguments: [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68, sr_return_quantity#72, sr_return_amt#73], [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68, sr_return_quantity#72, sr_return_amt#73] +(58) CometProject +Input [9]: [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66, sr_item_sk#68, sr_ticket_number#69, sr_return_quantity#70, sr_return_amt#71] +Arguments: [ss_item_sk#61, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66, sr_return_quantity#70, sr_return_amt#71], [ss_item_sk#61, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66, sr_return_quantity#70, sr_return_amt#71] -(62) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#75, d_year#76, d_moy#77] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(63) CometFilter -Input [3]: [d_date_sk#75, d_year#76, d_moy#77] -Condition : ((((isnotnull(d_year#76) AND isnotnull(d_moy#77)) AND (d_year#76 = 2001)) AND (d_moy#77 = 12)) AND isnotnull(d_date_sk#75)) - -(64) CometProject -Input [3]: [d_date_sk#75, d_year#76, d_moy#77] -Arguments: [d_date_sk#75], [d_date_sk#75] - -(65) CometBroadcastExchange -Input [1]: [d_date_sk#75] -Arguments: [d_date_sk#75] - -(66) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68, sr_return_quantity#72, sr_return_amt#73] -Right output [1]: [d_date_sk#75] -Arguments: [ss_sold_date_sk#68], [d_date_sk#75], Inner +(59) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_date_sk#73] -(67) CometProject -Input [7]: [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, ss_sold_date_sk#68, sr_return_quantity#72, sr_return_amt#73, d_date_sk#75] -Arguments: [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, sr_return_quantity#72, sr_return_amt#73], [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, sr_return_quantity#72, sr_return_amt#73] - -(68) CometHashAggregate -Input [5]: [ss_item_sk#63, ss_quantity#65, ss_net_paid#66, sr_return_quantity#72, sr_return_amt#73] -Keys [1]: [ss_item_sk#63] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#72, 0)), partial_sum(coalesce(ss_quantity#65, 0)), partial_sum(coalesce(cast(sr_return_amt#73 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#66 as decimal(12,2)), 0.00))] - -(69) ColumnarToRow [codegen id : 11] -Input [7]: [ss_item_sk#63, sum#78, sum#79, sum#80, isEmpty#81, sum#82, isEmpty#83] - -(70) Exchange -Input [7]: [ss_item_sk#63, sum#78, sum#79, sum#80, isEmpty#81, sum#82, isEmpty#83] -Arguments: hashpartitioning(ss_item_sk#63, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(71) HashAggregate [codegen id : 12] -Input [7]: [ss_item_sk#63, sum#78, sum#79, sum#80, isEmpty#81, sum#82, isEmpty#83] -Keys [1]: [ss_item_sk#63] -Functions [4]: [sum(coalesce(sr_return_quantity#72, 0)), sum(coalesce(ss_quantity#65, 0)), sum(coalesce(cast(sr_return_amt#73 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#66 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#72, 0))#84, sum(coalesce(ss_quantity#65, 0))#85, sum(coalesce(cast(sr_return_amt#73 as decimal(12,2)), 0.00))#86, sum(coalesce(cast(ss_net_paid#66 as decimal(12,2)), 0.00))#87] -Results [3]: [ss_item_sk#63 AS item#88, (cast(sum(coalesce(sr_return_quantity#72, 0))#84 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#65, 0))#85 as decimal(15,4))) AS return_ratio#89, (cast(sum(coalesce(cast(sr_return_amt#73 as decimal(12,2)), 0.00))#86 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#66 as decimal(12,2)), 0.00))#87 as decimal(15,4))) AS currency_ratio#90] +(60) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#61, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66, sr_return_quantity#70, sr_return_amt#71] +Right output [1]: [d_date_sk#73] +Arguments: [ss_sold_date_sk#66], [d_date_sk#73], Inner -(72) Exchange -Input [3]: [item#88, return_ratio#89, currency_ratio#90] +(61) CometProject +Input [7]: [ss_item_sk#61, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66, sr_return_quantity#70, sr_return_amt#71, d_date_sk#73] +Arguments: [ss_item_sk#61, ss_quantity#63, ss_net_paid#64, sr_return_quantity#70, sr_return_amt#71], [ss_item_sk#61, ss_quantity#63, ss_net_paid#64, sr_return_quantity#70, sr_return_amt#71] + +(62) CometHashAggregate +Input [5]: [ss_item_sk#61, ss_quantity#63, ss_net_paid#64, sr_return_quantity#70, sr_return_amt#71] +Keys [1]: [ss_item_sk#61] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#70, 0)), partial_sum(coalesce(ss_quantity#63, 0)), partial_sum(coalesce(cast(sr_return_amt#71 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#64 as decimal(12,2)), 0.00))] + +(63) ColumnarToRow [codegen id : 11] +Input [7]: [ss_item_sk#61, sum#74, sum#75, sum#76, isEmpty#77, sum#78, isEmpty#79] + +(64) Exchange +Input [7]: [ss_item_sk#61, sum#74, sum#75, sum#76, isEmpty#77, sum#78, isEmpty#79] +Arguments: hashpartitioning(ss_item_sk#61, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(65) HashAggregate [codegen id : 12] +Input [7]: [ss_item_sk#61, sum#74, sum#75, sum#76, isEmpty#77, sum#78, isEmpty#79] +Keys [1]: [ss_item_sk#61] +Functions [4]: [sum(coalesce(sr_return_quantity#70, 0)), sum(coalesce(ss_quantity#63, 0)), sum(coalesce(cast(sr_return_amt#71 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#64 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#70, 0))#80, sum(coalesce(ss_quantity#63, 0))#81, sum(coalesce(cast(sr_return_amt#71 as decimal(12,2)), 0.00))#82, sum(coalesce(cast(ss_net_paid#64 as decimal(12,2)), 0.00))#83] +Results [3]: [ss_item_sk#61 AS item#84, (cast(sum(coalesce(sr_return_quantity#70, 0))#80 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#63, 0))#81 as decimal(15,4))) AS return_ratio#85, (cast(sum(coalesce(cast(sr_return_amt#71 as decimal(12,2)), 0.00))#82 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#64 as decimal(12,2)), 0.00))#83 as decimal(15,4))) AS currency_ratio#86] + +(66) Exchange +Input [3]: [item#84, return_ratio#85, currency_ratio#86] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(73) Sort [codegen id : 13] -Input [3]: [item#88, return_ratio#89, currency_ratio#90] -Arguments: [return_ratio#89 ASC NULLS FIRST], false, 0 +(67) Sort [codegen id : 13] +Input [3]: [item#84, return_ratio#85, currency_ratio#86] +Arguments: [return_ratio#85 ASC NULLS FIRST], false, 0 -(74) Window -Input [3]: [item#88, return_ratio#89, currency_ratio#90] -Arguments: [rank(return_ratio#89) windowspecdefinition(return_ratio#89 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#91], [return_ratio#89 ASC NULLS FIRST] +(68) Window +Input [3]: [item#84, return_ratio#85, currency_ratio#86] +Arguments: [rank(return_ratio#85) windowspecdefinition(return_ratio#85 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#87], [return_ratio#85 ASC NULLS FIRST] -(75) Sort [codegen id : 14] -Input [4]: [item#88, return_ratio#89, currency_ratio#90, return_rank#91] -Arguments: [currency_ratio#90 ASC NULLS FIRST], false, 0 +(69) Sort [codegen id : 14] +Input [4]: [item#84, return_ratio#85, currency_ratio#86, return_rank#87] +Arguments: [currency_ratio#86 ASC NULLS FIRST], false, 0 -(76) Window -Input [4]: [item#88, return_ratio#89, currency_ratio#90, return_rank#91] -Arguments: [rank(currency_ratio#90) windowspecdefinition(currency_ratio#90 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#92], [currency_ratio#90 ASC NULLS FIRST] +(70) Window +Input [4]: [item#84, return_ratio#85, currency_ratio#86, return_rank#87] +Arguments: [rank(currency_ratio#86) windowspecdefinition(currency_ratio#86 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#88], [currency_ratio#86 ASC NULLS FIRST] -(77) Filter [codegen id : 15] -Input [5]: [item#88, return_ratio#89, currency_ratio#90, return_rank#91, currency_rank#92] -Condition : ((return_rank#91 <= 10) OR (currency_rank#92 <= 10)) +(71) Filter [codegen id : 15] +Input [5]: [item#84, return_ratio#85, currency_ratio#86, return_rank#87, currency_rank#88] +Condition : ((return_rank#87 <= 10) OR (currency_rank#88 <= 10)) -(78) Project [codegen id : 15] -Output [5]: [store AS channel#93, item#88, return_ratio#89, return_rank#91, currency_rank#92] -Input [5]: [item#88, return_ratio#89, currency_ratio#90, return_rank#91, currency_rank#92] +(72) Project [codegen id : 15] +Output [5]: [store AS channel#89, item#84, return_ratio#85, return_rank#87, currency_rank#88] +Input [5]: [item#84, return_ratio#85, currency_ratio#86, return_rank#87, currency_rank#88] -(79) Union +(73) Union -(80) HashAggregate [codegen id : 16] +(74) HashAggregate [codegen id : 16] Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] Keys [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] Functions: [] Aggregate Attributes: [] Results [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] -(81) Exchange +(75) Exchange Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] Arguments: hashpartitioning(channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(82) HashAggregate [codegen id : 17] +(76) HashAggregate [codegen id : 17] Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] Keys [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] Functions: [] Aggregate Attributes: [] Results [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] -(83) TakeOrderedAndProject +(77) TakeOrderedAndProject Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] Arguments: 100, [channel#31 ASC NULLS FIRST, return_rank#29 ASC NULLS FIRST, currency_rank#30 ASC NULLS FIRST, item#26 ASC NULLS FIRST], [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (88) -+- * ColumnarToRow (87) - +- CometProject (86) - +- CometFilter (85) - +- CometScan parquet spark_catalog.default.date_dim (84) +BroadcastExchange (82) ++- * ColumnarToRow (81) + +- CometProject (80) + +- CometFilter (79) + +- CometScan parquet spark_catalog.default.date_dim (78) -(84) Scan parquet spark_catalog.default.date_dim +(78) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#13, d_year#14, d_moy#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] ReadSchema: struct -(85) CometFilter +(79) CometFilter Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) -(86) CometProject +(80) CometProject Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(87) ColumnarToRow [codegen id : 1] +(81) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(88) BroadcastExchange +(82) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#37 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 53 Hosting Expression = ss_sold_date_sk#68 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#66 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt index 1ded798bd..8d7b158d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt @@ -84,10 +84,7 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] CometFilter [cr_return_amount,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] - CometBroadcastExchange #10 - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + ReusedExchange [d_date_sk] #6 WholeStageCodegen (15) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] @@ -100,11 +97,11 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (13) Sort [return_ratio] InputAdapter - Exchange #11 + Exchange #10 WholeStageCodegen (12) HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange [ss_item_sk] #12 + Exchange [ss_item_sk] #11 WholeStageCodegen (11) ColumnarToRow InputAdapter @@ -113,7 +110,7 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometBroadcastExchange #13 + CometBroadcastExchange #12 CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] @@ -121,7 +118,4 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] CometFilter [sr_return_amt,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] - CometBroadcastExchange #14 - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt index 346471fe7..ba2f8634b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt @@ -1,23 +1,23 @@ == Physical Plan == -TakeOrderedAndProject (180) -+- * Filter (179) - +- * HashAggregate (178) - +- * HashAggregate (177) - +- * Project (176) - +- * BroadcastHashJoin Inner BuildRight (175) - :- Window (86) - : +- * Sort (85) - : +- Exchange (84) - : +- * Project (83) - : +- * Filter (82) - : +- * SortMergeJoin FullOuter (81) - : :- * Sort (40) - : : +- Exchange (39) - : : +- * HashAggregate (38) - : : +- Exchange (37) - : : +- * HashAggregate (36) - : : +- * Project (35) - : : +- * BroadcastHashJoin Inner BuildRight (34) +TakeOrderedAndProject (70) ++- * Filter (69) + +- * HashAggregate (68) + +- * HashAggregate (67) + +- * Project (66) + +- * BroadcastHashJoin Inner BuildRight (65) + :- Window (59) + : +- * Sort (58) + : +- Exchange (57) + : +- * Project (56) + : +- * Filter (55) + : +- * SortMergeJoin FullOuter (54) + : :- * Sort (28) + : : +- Exchange (27) + : : +- * HashAggregate (26) + : : +- Exchange (25) + : : +- * HashAggregate (24) + : : +- * Project (23) + : : +- * BroadcastHashJoin Inner BuildRight (22) : : :- * Project (16) : : : +- Window (15) : : : +- * Sort (14) @@ -34,151 +34,41 @@ TakeOrderedAndProject (180) : : : +- CometProject (5) : : : +- CometFilter (4) : : : +- CometScan parquet spark_catalog.default.date_dim (3) - : : +- BroadcastExchange (33) - : : +- * Project (32) - : : +- Window (31) - : : +- * Sort (30) - : : +- Exchange (29) - : : +- * HashAggregate (28) - : : +- Exchange (27) - : : +- * ColumnarToRow (26) - : : +- CometHashAggregate (25) - : : +- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometFilter (18) - : : : +- CometScan parquet spark_catalog.default.web_sales (17) - : : +- CometBroadcastExchange (22) - : : +- CometProject (21) - : : +- CometFilter (20) - : : +- CometScan parquet spark_catalog.default.date_dim (19) - : +- * Sort (80) - : +- Exchange (79) - : +- * HashAggregate (78) - : +- Exchange (77) - : +- * HashAggregate (76) - : +- * Project (75) - : +- * BroadcastHashJoin Inner BuildRight (74) - : :- * Project (56) - : : +- Window (55) - : : +- * Sort (54) - : : +- Exchange (53) - : : +- * HashAggregate (52) - : : +- Exchange (51) - : : +- * ColumnarToRow (50) - : : +- CometHashAggregate (49) - : : +- CometProject (48) - : : +- CometBroadcastHashJoin (47) - : : :- CometFilter (42) - : : : +- CometScan parquet spark_catalog.default.store_sales (41) - : : +- CometBroadcastExchange (46) - : : +- CometProject (45) - : : +- CometFilter (44) - : : +- CometScan parquet spark_catalog.default.date_dim (43) - : +- BroadcastExchange (73) - : +- * Project (72) - : +- Window (71) - : +- * Sort (70) - : +- Exchange (69) - : +- * HashAggregate (68) - : +- Exchange (67) - : +- * ColumnarToRow (66) - : +- CometHashAggregate (65) - : +- CometProject (64) - : +- CometBroadcastHashJoin (63) - : :- CometFilter (58) - : : +- CometScan parquet spark_catalog.default.store_sales (57) - : +- CometBroadcastExchange (62) - : +- CometProject (61) - : +- CometFilter (60) - : +- CometScan parquet spark_catalog.default.date_dim (59) - +- BroadcastExchange (174) - +- * Project (173) - +- Window (172) - +- * Sort (171) - +- Exchange (170) - +- * Project (169) - +- * Filter (168) - +- * SortMergeJoin FullOuter (167) - :- * Sort (126) - : +- Exchange (125) - : +- * HashAggregate (124) - : +- Exchange (123) - : +- * HashAggregate (122) - : +- * Project (121) - : +- * BroadcastHashJoin Inner BuildRight (120) - : :- * Project (102) - : : +- Window (101) - : : +- * Sort (100) - : : +- Exchange (99) - : : +- * HashAggregate (98) - : : +- Exchange (97) - : : +- * ColumnarToRow (96) - : : +- CometHashAggregate (95) - : : +- CometProject (94) - : : +- CometBroadcastHashJoin (93) - : : :- CometFilter (88) - : : : +- CometScan parquet spark_catalog.default.web_sales (87) - : : +- CometBroadcastExchange (92) - : : +- CometProject (91) - : : +- CometFilter (90) - : : +- CometScan parquet spark_catalog.default.date_dim (89) - : +- BroadcastExchange (119) - : +- * Project (118) - : +- Window (117) - : +- * Sort (116) - : +- Exchange (115) - : +- * HashAggregate (114) - : +- Exchange (113) - : +- * ColumnarToRow (112) - : +- CometHashAggregate (111) - : +- CometProject (110) - : +- CometBroadcastHashJoin (109) - : :- CometFilter (104) - : : +- CometScan parquet spark_catalog.default.web_sales (103) - : +- CometBroadcastExchange (108) - : +- CometProject (107) - : +- CometFilter (106) - : +- CometScan parquet spark_catalog.default.date_dim (105) - +- * Sort (166) - +- Exchange (165) - +- * HashAggregate (164) - +- Exchange (163) - +- * HashAggregate (162) - +- * Project (161) - +- * BroadcastHashJoin Inner BuildRight (160) - :- * Project (142) - : +- Window (141) - : +- * Sort (140) - : +- Exchange (139) - : +- * HashAggregate (138) - : +- Exchange (137) - : +- * ColumnarToRow (136) - : +- CometHashAggregate (135) - : +- CometProject (134) - : +- CometBroadcastHashJoin (133) - : :- CometFilter (128) - : : +- CometScan parquet spark_catalog.default.store_sales (127) - : +- CometBroadcastExchange (132) - : +- CometProject (131) - : +- CometFilter (130) - : +- CometScan parquet spark_catalog.default.date_dim (129) - +- BroadcastExchange (159) - +- * Project (158) - +- Window (157) - +- * Sort (156) - +- Exchange (155) - +- * HashAggregate (154) - +- Exchange (153) - +- * ColumnarToRow (152) - +- CometHashAggregate (151) - +- CometProject (150) - +- CometBroadcastHashJoin (149) - :- CometFilter (144) - : +- CometScan parquet spark_catalog.default.store_sales (143) - +- CometBroadcastExchange (148) - +- CometProject (147) - +- CometFilter (146) - +- CometScan parquet spark_catalog.default.date_dim (145) + : : +- BroadcastExchange (21) + : : +- * Project (20) + : : +- Window (19) + : : +- * Sort (18) + : : +- ReusedExchange (17) + : +- * Sort (53) + : +- Exchange (52) + : +- * HashAggregate (51) + : +- Exchange (50) + : +- * HashAggregate (49) + : +- * Project (48) + : +- * BroadcastHashJoin Inner BuildRight (47) + : :- * Project (41) + : : +- Window (40) + : : +- * Sort (39) + : : +- Exchange (38) + : : +- * HashAggregate (37) + : : +- Exchange (36) + : : +- * ColumnarToRow (35) + : : +- CometHashAggregate (34) + : : +- CometProject (33) + : : +- CometBroadcastHashJoin (32) + : : :- CometFilter (30) + : : : +- CometScan parquet spark_catalog.default.store_sales (29) + : : +- ReusedExchange (31) + : +- BroadcastExchange (46) + : +- * Project (45) + : +- Window (44) + : +- * Sort (43) + : +- ReusedExchange (42) + +- BroadcastExchange (64) + +- * Project (63) + +- Window (62) + +- * Sort (61) + +- ReusedExchange (60) (1) Scan parquet spark_catalog.default.web_sales @@ -256,827 +146,284 @@ Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS F Output [4]: [item_sk#10, d_date#6, sumws#11, rk#12] Input [5]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1, rk#12] -(17) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#13, ws_sales_price#14, ws_sold_date_sk#15] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#15), dynamicpruningexpression(ws_sold_date_sk#15 IN dynamicpruning#16)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(18) CometFilter -Input [3]: [ws_item_sk#13, ws_sales_price#14, ws_sold_date_sk#15] -Condition : isnotnull(ws_item_sk#13) - -(19) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_date#18, d_month_seq#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [3]: [d_date_sk#17, d_date#18, d_month_seq#19] -Condition : (((isnotnull(d_month_seq#19) AND (d_month_seq#19 >= 1212)) AND (d_month_seq#19 <= 1223)) AND isnotnull(d_date_sk#17)) - -(21) CometProject -Input [3]: [d_date_sk#17, d_date#18, d_month_seq#19] -Arguments: [d_date_sk#17, d_date#18], [d_date_sk#17, d_date#18] - -(22) CometBroadcastExchange -Input [2]: [d_date_sk#17, d_date#18] -Arguments: [d_date_sk#17, d_date#18] - -(23) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#13, ws_sales_price#14, ws_sold_date_sk#15] -Right output [2]: [d_date_sk#17, d_date#18] -Arguments: [ws_sold_date_sk#15], [d_date_sk#17], Inner - -(24) CometProject -Input [5]: [ws_item_sk#13, ws_sales_price#14, ws_sold_date_sk#15, d_date_sk#17, d_date#18] -Arguments: [ws_item_sk#13, ws_sales_price#14, d_date#18], [ws_item_sk#13, ws_sales_price#14, d_date#18] - -(25) CometHashAggregate -Input [3]: [ws_item_sk#13, ws_sales_price#14, d_date#18] -Keys [2]: [ws_item_sk#13, d_date#18] -Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#14))] - -(26) ColumnarToRow [codegen id : 4] -Input [3]: [ws_item_sk#13, d_date#18, sum#20] - -(27) Exchange -Input [3]: [ws_item_sk#13, d_date#18, sum#20] -Arguments: hashpartitioning(ws_item_sk#13, d_date#18, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(17) ReusedExchange [Reuses operator id: 13] +Output [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] -(28) HashAggregate [codegen id : 5] -Input [3]: [ws_item_sk#13, d_date#18, sum#20] -Keys [2]: [ws_item_sk#13, d_date#18] -Functions [1]: [sum(UnscaledValue(ws_sales_price#14))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#14))#9] -Results [4]: [ws_item_sk#13 AS item_sk#10, d_date#18, MakeDecimal(sum(UnscaledValue(ws_sales_price#14))#9,17,2) AS sumws#11, ws_item_sk#13] +(18) Sort [codegen id : 6] +Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] +Arguments: [ws_item_sk#14 ASC NULLS FIRST, d_date#13 ASC NULLS FIRST], false, 0 -(29) Exchange -Input [4]: [item_sk#10, d_date#18, sumws#11, ws_item_sk#13] -Arguments: hashpartitioning(ws_item_sk#13, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(19) Window +Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] +Arguments: [row_number() windowspecdefinition(ws_item_sk#14, d_date#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#15], [ws_item_sk#14], [d_date#13 ASC NULLS FIRST] -(30) Sort [codegen id : 6] -Input [4]: [item_sk#10, d_date#18, sumws#11, ws_item_sk#13] -Arguments: [ws_item_sk#13 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST], false, 0 +(20) Project [codegen id : 7] +Output [3]: [item_sk#10 AS item_sk#16, sumws#11 AS sumws#17, rk#15] +Input [5]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14, rk#15] -(31) Window -Input [4]: [item_sk#10, d_date#18, sumws#11, ws_item_sk#13] -Arguments: [row_number() windowspecdefinition(ws_item_sk#13, d_date#18 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#21], [ws_item_sk#13], [d_date#18 ASC NULLS FIRST] +(21) BroadcastExchange +Input [3]: [item_sk#16, sumws#17, rk#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(32) Project [codegen id : 7] -Output [3]: [item_sk#10 AS item_sk#22, sumws#11 AS sumws#23, rk#21] -Input [5]: [item_sk#10, d_date#18, sumws#11, ws_item_sk#13, rk#21] - -(33) BroadcastExchange -Input [3]: [item_sk#22, sumws#23, rk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(34) BroadcastHashJoin [codegen id : 8] +(22) BroadcastHashJoin [codegen id : 8] Left keys [1]: [item_sk#10] -Right keys [1]: [item_sk#22] +Right keys [1]: [item_sk#16] Join type: Inner -Join condition: (rk#12 >= rk#21) +Join condition: (rk#12 >= rk#15) -(35) Project [codegen id : 8] -Output [4]: [item_sk#10, d_date#6, sumws#11, sumws#23] -Input [7]: [item_sk#10, d_date#6, sumws#11, rk#12, item_sk#22, sumws#23, rk#21] +(23) Project [codegen id : 8] +Output [4]: [item_sk#10, d_date#6, sumws#11, sumws#17] +Input [7]: [item_sk#10, d_date#6, sumws#11, rk#12, item_sk#16, sumws#17, rk#15] -(36) HashAggregate [codegen id : 8] -Input [4]: [item_sk#10, d_date#6, sumws#11, sumws#23] +(24) HashAggregate [codegen id : 8] +Input [4]: [item_sk#10, d_date#6, sumws#11, sumws#17] Keys [3]: [item_sk#10, d_date#6, sumws#11] -Functions [1]: [partial_sum(sumws#23)] -Aggregate Attributes [2]: [sum#24, isEmpty#25] -Results [5]: [item_sk#10, d_date#6, sumws#11, sum#26, isEmpty#27] +Functions [1]: [partial_sum(sumws#17)] +Aggregate Attributes [2]: [sum#18, isEmpty#19] +Results [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] -(37) Exchange -Input [5]: [item_sk#10, d_date#6, sumws#11, sum#26, isEmpty#27] -Arguments: hashpartitioning(item_sk#10, d_date#6, sumws#11, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(25) Exchange +Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] +Arguments: hashpartitioning(item_sk#10, d_date#6, sumws#11, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(38) HashAggregate [codegen id : 9] -Input [5]: [item_sk#10, d_date#6, sumws#11, sum#26, isEmpty#27] +(26) HashAggregate [codegen id : 9] +Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] Keys [3]: [item_sk#10, d_date#6, sumws#11] -Functions [1]: [sum(sumws#23)] -Aggregate Attributes [1]: [sum(sumws#23)#28] -Results [3]: [item_sk#10, d_date#6, sum(sumws#23)#28 AS cume_sales#29] +Functions [1]: [sum(sumws#17)] +Aggregate Attributes [1]: [sum(sumws#17)#22] +Results [3]: [item_sk#10, d_date#6, sum(sumws#17)#22 AS cume_sales#23] -(39) Exchange -Input [3]: [item_sk#10, d_date#6, cume_sales#29] -Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(27) Exchange +Input [3]: [item_sk#10, d_date#6, cume_sales#23] +Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(40) Sort [codegen id : 10] -Input [3]: [item_sk#10, d_date#6, cume_sales#29] +(28) Sort [codegen id : 10] +Input [3]: [item_sk#10, d_date#6, cume_sales#23] Arguments: [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 -(41) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] +(29) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#32), dynamicpruningexpression(ss_sold_date_sk#32 IN dynamicpruning#33)] +PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(42) CometFilter -Input [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] -Condition : isnotnull(ss_item_sk#30) - -(43) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#34, d_date#35, d_month_seq#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(44) CometFilter -Input [3]: [d_date_sk#34, d_date#35, d_month_seq#36] -Condition : (((isnotnull(d_month_seq#36) AND (d_month_seq#36 >= 1212)) AND (d_month_seq#36 <= 1223)) AND isnotnull(d_date_sk#34)) +(30) CometFilter +Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] +Condition : isnotnull(ss_item_sk#24) -(45) CometProject -Input [3]: [d_date_sk#34, d_date#35, d_month_seq#36] -Arguments: [d_date_sk#34, d_date#35], [d_date_sk#34, d_date#35] +(31) ReusedExchange [Reuses operator id: 6] +Output [2]: [d_date_sk#28, d_date#29] -(46) CometBroadcastExchange -Input [2]: [d_date_sk#34, d_date#35] -Arguments: [d_date_sk#34, d_date#35] +(32) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] +Right output [2]: [d_date_sk#28, d_date#29] +Arguments: [ss_sold_date_sk#26], [d_date_sk#28], Inner -(47) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] -Right output [2]: [d_date_sk#34, d_date#35] -Arguments: [ss_sold_date_sk#32], [d_date_sk#34], Inner +(33) CometProject +Input [5]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#28, d_date#29] +Arguments: [ss_item_sk#24, ss_sales_price#25, d_date#29], [ss_item_sk#24, ss_sales_price#25, d_date#29] -(48) CometProject -Input [5]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32, d_date_sk#34, d_date#35] -Arguments: [ss_item_sk#30, ss_sales_price#31, d_date#35], [ss_item_sk#30, ss_sales_price#31, d_date#35] +(34) CometHashAggregate +Input [3]: [ss_item_sk#24, ss_sales_price#25, d_date#29] +Keys [2]: [ss_item_sk#24, d_date#29] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#25))] -(49) CometHashAggregate -Input [3]: [ss_item_sk#30, ss_sales_price#31, d_date#35] -Keys [2]: [ss_item_sk#30, d_date#35] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#31))] +(35) ColumnarToRow [codegen id : 11] +Input [3]: [ss_item_sk#24, d_date#29, sum#30] -(50) ColumnarToRow [codegen id : 11] -Input [3]: [ss_item_sk#30, d_date#35, sum#37] +(36) Exchange +Input [3]: [ss_item_sk#24, d_date#29, sum#30] +Arguments: hashpartitioning(ss_item_sk#24, d_date#29, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(51) Exchange -Input [3]: [ss_item_sk#30, d_date#35, sum#37] -Arguments: hashpartitioning(ss_item_sk#30, d_date#35, 5), ENSURE_REQUIREMENTS, [plan_id=8] +(37) HashAggregate [codegen id : 12] +Input [3]: [ss_item_sk#24, d_date#29, sum#30] +Keys [2]: [ss_item_sk#24, d_date#29] +Functions [1]: [sum(UnscaledValue(ss_sales_price#25))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#31] +Results [4]: [ss_item_sk#24 AS item_sk#32, d_date#29, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#31,17,2) AS sumss#33, ss_item_sk#24] -(52) HashAggregate [codegen id : 12] -Input [3]: [ss_item_sk#30, d_date#35, sum#37] -Keys [2]: [ss_item_sk#30, d_date#35] -Functions [1]: [sum(UnscaledValue(ss_sales_price#31))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#31))#38] -Results [4]: [ss_item_sk#30 AS item_sk#39, d_date#35, MakeDecimal(sum(UnscaledValue(ss_sales_price#31))#38,17,2) AS sumss#40, ss_item_sk#30] +(38) Exchange +Input [4]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24] +Arguments: hashpartitioning(ss_item_sk#24, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(53) Exchange -Input [4]: [item_sk#39, d_date#35, sumss#40, ss_item_sk#30] -Arguments: hashpartitioning(ss_item_sk#30, 5), ENSURE_REQUIREMENTS, [plan_id=9] +(39) Sort [codegen id : 13] +Input [4]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24] +Arguments: [ss_item_sk#24 ASC NULLS FIRST, d_date#29 ASC NULLS FIRST], false, 0 -(54) Sort [codegen id : 13] -Input [4]: [item_sk#39, d_date#35, sumss#40, ss_item_sk#30] -Arguments: [ss_item_sk#30 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST], false, 0 +(40) Window +Input [4]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24] +Arguments: [row_number() windowspecdefinition(ss_item_sk#24, d_date#29 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#34], [ss_item_sk#24], [d_date#29 ASC NULLS FIRST] -(55) Window -Input [4]: [item_sk#39, d_date#35, sumss#40, ss_item_sk#30] -Arguments: [row_number() windowspecdefinition(ss_item_sk#30, d_date#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#41], [ss_item_sk#30], [d_date#35 ASC NULLS FIRST] +(41) Project [codegen id : 18] +Output [4]: [item_sk#32, d_date#29, sumss#33, rk#34] +Input [5]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24, rk#34] -(56) Project [codegen id : 18] -Output [4]: [item_sk#39, d_date#35, sumss#40, rk#41] -Input [5]: [item_sk#39, d_date#35, sumss#40, ss_item_sk#30, rk#41] +(42) ReusedExchange [Reuses operator id: 38] +Output [4]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36] -(57) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#42, ss_sales_price#43, ss_sold_date_sk#44] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#44), dynamicpruningexpression(ss_sold_date_sk#44 IN dynamicpruning#45)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(58) CometFilter -Input [3]: [ss_item_sk#42, ss_sales_price#43, ss_sold_date_sk#44] -Condition : isnotnull(ss_item_sk#42) - -(59) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#46, d_date#47, d_month_seq#48] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct +(43) Sort [codegen id : 16] +Input [4]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36] +Arguments: [ss_item_sk#36 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST], false, 0 -(60) CometFilter -Input [3]: [d_date_sk#46, d_date#47, d_month_seq#48] -Condition : (((isnotnull(d_month_seq#48) AND (d_month_seq#48 >= 1212)) AND (d_month_seq#48 <= 1223)) AND isnotnull(d_date_sk#46)) +(44) Window +Input [4]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36] +Arguments: [row_number() windowspecdefinition(ss_item_sk#36, d_date#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#37], [ss_item_sk#36], [d_date#35 ASC NULLS FIRST] -(61) CometProject -Input [3]: [d_date_sk#46, d_date#47, d_month_seq#48] -Arguments: [d_date_sk#46, d_date#47], [d_date_sk#46, d_date#47] +(45) Project [codegen id : 17] +Output [3]: [item_sk#32 AS item_sk#38, sumss#33 AS sumss#39, rk#37] +Input [5]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36, rk#37] -(62) CometBroadcastExchange -Input [2]: [d_date_sk#46, d_date#47] -Arguments: [d_date_sk#46, d_date#47] +(46) BroadcastExchange +Input [3]: [item_sk#38, sumss#39, rk#37] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -(63) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#42, ss_sales_price#43, ss_sold_date_sk#44] -Right output [2]: [d_date_sk#46, d_date#47] -Arguments: [ss_sold_date_sk#44], [d_date_sk#46], Inner - -(64) CometProject -Input [5]: [ss_item_sk#42, ss_sales_price#43, ss_sold_date_sk#44, d_date_sk#46, d_date#47] -Arguments: [ss_item_sk#42, ss_sales_price#43, d_date#47], [ss_item_sk#42, ss_sales_price#43, d_date#47] - -(65) CometHashAggregate -Input [3]: [ss_item_sk#42, ss_sales_price#43, d_date#47] -Keys [2]: [ss_item_sk#42, d_date#47] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#43))] - -(66) ColumnarToRow [codegen id : 14] -Input [3]: [ss_item_sk#42, d_date#47, sum#49] - -(67) Exchange -Input [3]: [ss_item_sk#42, d_date#47, sum#49] -Arguments: hashpartitioning(ss_item_sk#42, d_date#47, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(68) HashAggregate [codegen id : 15] -Input [3]: [ss_item_sk#42, d_date#47, sum#49] -Keys [2]: [ss_item_sk#42, d_date#47] -Functions [1]: [sum(UnscaledValue(ss_sales_price#43))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#43))#38] -Results [4]: [ss_item_sk#42 AS item_sk#39, d_date#47, MakeDecimal(sum(UnscaledValue(ss_sales_price#43))#38,17,2) AS sumss#40, ss_item_sk#42] - -(69) Exchange -Input [4]: [item_sk#39, d_date#47, sumss#40, ss_item_sk#42] -Arguments: hashpartitioning(ss_item_sk#42, 5), ENSURE_REQUIREMENTS, [plan_id=11] - -(70) Sort [codegen id : 16] -Input [4]: [item_sk#39, d_date#47, sumss#40, ss_item_sk#42] -Arguments: [ss_item_sk#42 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST], false, 0 - -(71) Window -Input [4]: [item_sk#39, d_date#47, sumss#40, ss_item_sk#42] -Arguments: [row_number() windowspecdefinition(ss_item_sk#42, d_date#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#50], [ss_item_sk#42], [d_date#47 ASC NULLS FIRST] - -(72) Project [codegen id : 17] -Output [3]: [item_sk#39 AS item_sk#51, sumss#40 AS sumss#52, rk#50] -Input [5]: [item_sk#39, d_date#47, sumss#40, ss_item_sk#42, rk#50] - -(73) BroadcastExchange -Input [3]: [item_sk#51, sumss#52, rk#50] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - -(74) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_sk#39] -Right keys [1]: [item_sk#51] +(47) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [item_sk#32] +Right keys [1]: [item_sk#38] Join type: Inner -Join condition: (rk#41 >= rk#50) - -(75) Project [codegen id : 18] -Output [4]: [item_sk#39, d_date#35, sumss#40, sumss#52] -Input [7]: [item_sk#39, d_date#35, sumss#40, rk#41, item_sk#51, sumss#52, rk#50] - -(76) HashAggregate [codegen id : 18] -Input [4]: [item_sk#39, d_date#35, sumss#40, sumss#52] -Keys [3]: [item_sk#39, d_date#35, sumss#40] -Functions [1]: [partial_sum(sumss#52)] -Aggregate Attributes [2]: [sum#53, isEmpty#54] -Results [5]: [item_sk#39, d_date#35, sumss#40, sum#55, isEmpty#56] - -(77) Exchange -Input [5]: [item_sk#39, d_date#35, sumss#40, sum#55, isEmpty#56] -Arguments: hashpartitioning(item_sk#39, d_date#35, sumss#40, 5), ENSURE_REQUIREMENTS, [plan_id=13] - -(78) HashAggregate [codegen id : 19] -Input [5]: [item_sk#39, d_date#35, sumss#40, sum#55, isEmpty#56] -Keys [3]: [item_sk#39, d_date#35, sumss#40] -Functions [1]: [sum(sumss#52)] -Aggregate Attributes [1]: [sum(sumss#52)#57] -Results [3]: [item_sk#39, d_date#35, sum(sumss#52)#57 AS cume_sales#58] - -(79) Exchange -Input [3]: [item_sk#39, d_date#35, cume_sales#58] -Arguments: hashpartitioning(item_sk#39, d_date#35, 5), ENSURE_REQUIREMENTS, [plan_id=14] - -(80) Sort [codegen id : 20] -Input [3]: [item_sk#39, d_date#35, cume_sales#58] -Arguments: [item_sk#39 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST], false, 0 - -(81) SortMergeJoin [codegen id : 21] +Join condition: (rk#34 >= rk#37) + +(48) Project [codegen id : 18] +Output [4]: [item_sk#32, d_date#29, sumss#33, sumss#39] +Input [7]: [item_sk#32, d_date#29, sumss#33, rk#34, item_sk#38, sumss#39, rk#37] + +(49) HashAggregate [codegen id : 18] +Input [4]: [item_sk#32, d_date#29, sumss#33, sumss#39] +Keys [3]: [item_sk#32, d_date#29, sumss#33] +Functions [1]: [partial_sum(sumss#39)] +Aggregate Attributes [2]: [sum#40, isEmpty#41] +Results [5]: [item_sk#32, d_date#29, sumss#33, sum#42, isEmpty#43] + +(50) Exchange +Input [5]: [item_sk#32, d_date#29, sumss#33, sum#42, isEmpty#43] +Arguments: hashpartitioning(item_sk#32, d_date#29, sumss#33, 5), ENSURE_REQUIREMENTS, [plan_id=9] + +(51) HashAggregate [codegen id : 19] +Input [5]: [item_sk#32, d_date#29, sumss#33, sum#42, isEmpty#43] +Keys [3]: [item_sk#32, d_date#29, sumss#33] +Functions [1]: [sum(sumss#39)] +Aggregate Attributes [1]: [sum(sumss#39)#44] +Results [3]: [item_sk#32, d_date#29, sum(sumss#39)#44 AS cume_sales#45] + +(52) Exchange +Input [3]: [item_sk#32, d_date#29, cume_sales#45] +Arguments: hashpartitioning(item_sk#32, d_date#29, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(53) Sort [codegen id : 20] +Input [3]: [item_sk#32, d_date#29, cume_sales#45] +Arguments: [item_sk#32 ASC NULLS FIRST, d_date#29 ASC NULLS FIRST], false, 0 + +(54) SortMergeJoin [codegen id : 21] Left keys [2]: [item_sk#10, d_date#6] -Right keys [2]: [item_sk#39, d_date#35] +Right keys [2]: [item_sk#32, d_date#29] Join type: FullOuter Join condition: None -(82) Filter [codegen id : 21] -Input [6]: [item_sk#10, d_date#6, cume_sales#29, item_sk#39, d_date#35, cume_sales#58] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#39 END) +(55) Filter [codegen id : 21] +Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#32, d_date#29, cume_sales#45] +Condition : isnotnull(CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#32 END) -(83) Project [codegen id : 21] -Output [4]: [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#39 END AS item_sk#59, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#35 END AS d_date#60, cume_sales#29 AS web_sales#61, cume_sales#58 AS store_sales#62] -Input [6]: [item_sk#10, d_date#6, cume_sales#29, item_sk#39, d_date#35, cume_sales#58] +(56) Project [codegen id : 21] +Output [4]: [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#32 END AS item_sk#46, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#29 END AS d_date#47, cume_sales#23 AS web_sales#48, cume_sales#45 AS store_sales#49] +Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#32, d_date#29, cume_sales#45] -(84) Exchange -Input [4]: [item_sk#59, d_date#60, web_sales#61, store_sales#62] -Arguments: hashpartitioning(item_sk#59, 5), ENSURE_REQUIREMENTS, [plan_id=15] +(57) Exchange +Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] +Arguments: hashpartitioning(item_sk#46, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(85) Sort [codegen id : 22] -Input [4]: [item_sk#59, d_date#60, web_sales#61, store_sales#62] -Arguments: [item_sk#59 ASC NULLS FIRST, d_date#60 ASC NULLS FIRST], false, 0 +(58) Sort [codegen id : 22] +Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] +Arguments: [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST], false, 0 -(86) Window -Input [4]: [item_sk#59, d_date#60, web_sales#61, store_sales#62] -Arguments: [row_number() windowspecdefinition(item_sk#59, d_date#60 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#63], [item_sk#59], [d_date#60 ASC NULLS FIRST] +(59) Window +Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] +Arguments: [row_number() windowspecdefinition(item_sk#46, d_date#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#50], [item_sk#46], [d_date#47 ASC NULLS FIRST] -(87) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#64, ws_sales_price#65, ws_sold_date_sk#66] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#66), dynamicpruningexpression(ws_sold_date_sk#66 IN dynamicpruning#67)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct +(60) ReusedExchange [Reuses operator id: 57] +Output [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -(88) CometFilter -Input [3]: [ws_item_sk#64, ws_sales_price#65, ws_sold_date_sk#66] -Condition : isnotnull(ws_item_sk#64) +(61) Sort [codegen id : 44] +Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] +Arguments: [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST], false, 0 -(89) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#68, d_date#69, d_month_seq#70] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(90) CometFilter -Input [3]: [d_date_sk#68, d_date#69, d_month_seq#70] -Condition : (((isnotnull(d_month_seq#70) AND (d_month_seq#70 >= 1212)) AND (d_month_seq#70 <= 1223)) AND isnotnull(d_date_sk#68)) - -(91) CometProject -Input [3]: [d_date_sk#68, d_date#69, d_month_seq#70] -Arguments: [d_date_sk#68, d_date#69], [d_date_sk#68, d_date#69] - -(92) CometBroadcastExchange -Input [2]: [d_date_sk#68, d_date#69] -Arguments: [d_date_sk#68, d_date#69] - -(93) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#64, ws_sales_price#65, ws_sold_date_sk#66] -Right output [2]: [d_date_sk#68, d_date#69] -Arguments: [ws_sold_date_sk#66], [d_date_sk#68], Inner - -(94) CometProject -Input [5]: [ws_item_sk#64, ws_sales_price#65, ws_sold_date_sk#66, d_date_sk#68, d_date#69] -Arguments: [ws_item_sk#64, ws_sales_price#65, d_date#69], [ws_item_sk#64, ws_sales_price#65, d_date#69] - -(95) CometHashAggregate -Input [3]: [ws_item_sk#64, ws_sales_price#65, d_date#69] -Keys [2]: [ws_item_sk#64, d_date#69] -Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#65))] - -(96) ColumnarToRow [codegen id : 23] -Input [3]: [ws_item_sk#64, d_date#69, sum#71] - -(97) Exchange -Input [3]: [ws_item_sk#64, d_date#69, sum#71] -Arguments: hashpartitioning(ws_item_sk#64, d_date#69, 5), ENSURE_REQUIREMENTS, [plan_id=16] +(62) Window +Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] +Arguments: [row_number() windowspecdefinition(item_sk#46, d_date#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#51], [item_sk#46], [d_date#47 ASC NULLS FIRST] -(98) HashAggregate [codegen id : 24] -Input [3]: [ws_item_sk#64, d_date#69, sum#71] -Keys [2]: [ws_item_sk#64, d_date#69] -Functions [1]: [sum(UnscaledValue(ws_sales_price#65))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#65))#9] -Results [4]: [ws_item_sk#64 AS item_sk#10, d_date#69, MakeDecimal(sum(UnscaledValue(ws_sales_price#65))#9,17,2) AS sumws#11, ws_item_sk#64] - -(99) Exchange -Input [4]: [item_sk#10, d_date#69, sumws#11, ws_item_sk#64] -Arguments: hashpartitioning(ws_item_sk#64, 5), ENSURE_REQUIREMENTS, [plan_id=17] - -(100) Sort [codegen id : 25] -Input [4]: [item_sk#10, d_date#69, sumws#11, ws_item_sk#64] -Arguments: [ws_item_sk#64 ASC NULLS FIRST, d_date#69 ASC NULLS FIRST], false, 0 - -(101) Window -Input [4]: [item_sk#10, d_date#69, sumws#11, ws_item_sk#64] -Arguments: [row_number() windowspecdefinition(ws_item_sk#64, d_date#69 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#12], [ws_item_sk#64], [d_date#69 ASC NULLS FIRST] - -(102) Project [codegen id : 30] -Output [4]: [item_sk#10, d_date#69, sumws#11, rk#12] -Input [5]: [item_sk#10, d_date#69, sumws#11, ws_item_sk#64, rk#12] - -(103) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#72, ws_sales_price#73, ws_sold_date_sk#74] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#74), dynamicpruningexpression(ws_sold_date_sk#74 IN dynamicpruning#75)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct +(63) Project [codegen id : 45] +Output [4]: [item_sk#46 AS item_sk#52, web_sales#48 AS web_sales#53, store_sales#49 AS store_sales#54, rk#51] +Input [5]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, rk#51] -(104) CometFilter -Input [3]: [ws_item_sk#72, ws_sales_price#73, ws_sold_date_sk#74] -Condition : isnotnull(ws_item_sk#72) - -(105) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#76, d_date#77, d_month_seq#78] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(106) CometFilter -Input [3]: [d_date_sk#76, d_date#77, d_month_seq#78] -Condition : (((isnotnull(d_month_seq#78) AND (d_month_seq#78 >= 1212)) AND (d_month_seq#78 <= 1223)) AND isnotnull(d_date_sk#76)) - -(107) CometProject -Input [3]: [d_date_sk#76, d_date#77, d_month_seq#78] -Arguments: [d_date_sk#76, d_date#77], [d_date_sk#76, d_date#77] - -(108) CometBroadcastExchange -Input [2]: [d_date_sk#76, d_date#77] -Arguments: [d_date_sk#76, d_date#77] - -(109) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#72, ws_sales_price#73, ws_sold_date_sk#74] -Right output [2]: [d_date_sk#76, d_date#77] -Arguments: [ws_sold_date_sk#74], [d_date_sk#76], Inner - -(110) CometProject -Input [5]: [ws_item_sk#72, ws_sales_price#73, ws_sold_date_sk#74, d_date_sk#76, d_date#77] -Arguments: [ws_item_sk#72, ws_sales_price#73, d_date#77], [ws_item_sk#72, ws_sales_price#73, d_date#77] - -(111) CometHashAggregate -Input [3]: [ws_item_sk#72, ws_sales_price#73, d_date#77] -Keys [2]: [ws_item_sk#72, d_date#77] -Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#73))] - -(112) ColumnarToRow [codegen id : 26] -Input [3]: [ws_item_sk#72, d_date#77, sum#79] - -(113) Exchange -Input [3]: [ws_item_sk#72, d_date#77, sum#79] -Arguments: hashpartitioning(ws_item_sk#72, d_date#77, 5), ENSURE_REQUIREMENTS, [plan_id=18] - -(114) HashAggregate [codegen id : 27] -Input [3]: [ws_item_sk#72, d_date#77, sum#79] -Keys [2]: [ws_item_sk#72, d_date#77] -Functions [1]: [sum(UnscaledValue(ws_sales_price#73))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#73))#9] -Results [4]: [ws_item_sk#72 AS item_sk#10, d_date#77, MakeDecimal(sum(UnscaledValue(ws_sales_price#73))#9,17,2) AS sumws#11, ws_item_sk#72] - -(115) Exchange -Input [4]: [item_sk#10, d_date#77, sumws#11, ws_item_sk#72] -Arguments: hashpartitioning(ws_item_sk#72, 5), ENSURE_REQUIREMENTS, [plan_id=19] - -(116) Sort [codegen id : 28] -Input [4]: [item_sk#10, d_date#77, sumws#11, ws_item_sk#72] -Arguments: [ws_item_sk#72 ASC NULLS FIRST, d_date#77 ASC NULLS FIRST], false, 0 - -(117) Window -Input [4]: [item_sk#10, d_date#77, sumws#11, ws_item_sk#72] -Arguments: [row_number() windowspecdefinition(ws_item_sk#72, d_date#77 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#21], [ws_item_sk#72], [d_date#77 ASC NULLS FIRST] - -(118) Project [codegen id : 29] -Output [3]: [item_sk#10 AS item_sk#22, sumws#11 AS sumws#23, rk#21] -Input [5]: [item_sk#10, d_date#77, sumws#11, ws_item_sk#72, rk#21] - -(119) BroadcastExchange -Input [3]: [item_sk#22, sumws#23, rk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=20] - -(120) BroadcastHashJoin [codegen id : 30] -Left keys [1]: [item_sk#10] -Right keys [1]: [item_sk#22] -Join type: Inner -Join condition: (rk#12 >= rk#21) - -(121) Project [codegen id : 30] -Output [4]: [item_sk#10, d_date#69, sumws#11, sumws#23] -Input [7]: [item_sk#10, d_date#69, sumws#11, rk#12, item_sk#22, sumws#23, rk#21] - -(122) HashAggregate [codegen id : 30] -Input [4]: [item_sk#10, d_date#69, sumws#11, sumws#23] -Keys [3]: [item_sk#10, d_date#69, sumws#11] -Functions [1]: [partial_sum(sumws#23)] -Aggregate Attributes [2]: [sum#24, isEmpty#25] -Results [5]: [item_sk#10, d_date#69, sumws#11, sum#26, isEmpty#27] - -(123) Exchange -Input [5]: [item_sk#10, d_date#69, sumws#11, sum#26, isEmpty#27] -Arguments: hashpartitioning(item_sk#10, d_date#69, sumws#11, 5), ENSURE_REQUIREMENTS, [plan_id=21] - -(124) HashAggregate [codegen id : 31] -Input [5]: [item_sk#10, d_date#69, sumws#11, sum#26, isEmpty#27] -Keys [3]: [item_sk#10, d_date#69, sumws#11] -Functions [1]: [sum(sumws#23)] -Aggregate Attributes [1]: [sum(sumws#23)#28] -Results [3]: [item_sk#10, d_date#69, sum(sumws#23)#28 AS cume_sales#29] - -(125) Exchange -Input [3]: [item_sk#10, d_date#69, cume_sales#29] -Arguments: hashpartitioning(item_sk#10, d_date#69, 5), ENSURE_REQUIREMENTS, [plan_id=22] - -(126) Sort [codegen id : 32] -Input [3]: [item_sk#10, d_date#69, cume_sales#29] -Arguments: [item_sk#10 ASC NULLS FIRST, d_date#69 ASC NULLS FIRST], false, 0 - -(127) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#80, ss_sales_price#81, ss_sold_date_sk#82] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#82), dynamicpruningexpression(ss_sold_date_sk#82 IN dynamicpruning#83)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(128) CometFilter -Input [3]: [ss_item_sk#80, ss_sales_price#81, ss_sold_date_sk#82] -Condition : isnotnull(ss_item_sk#80) - -(129) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#84, d_date#85, d_month_seq#86] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(130) CometFilter -Input [3]: [d_date_sk#84, d_date#85, d_month_seq#86] -Condition : (((isnotnull(d_month_seq#86) AND (d_month_seq#86 >= 1212)) AND (d_month_seq#86 <= 1223)) AND isnotnull(d_date_sk#84)) - -(131) CometProject -Input [3]: [d_date_sk#84, d_date#85, d_month_seq#86] -Arguments: [d_date_sk#84, d_date#85], [d_date_sk#84, d_date#85] - -(132) CometBroadcastExchange -Input [2]: [d_date_sk#84, d_date#85] -Arguments: [d_date_sk#84, d_date#85] - -(133) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#80, ss_sales_price#81, ss_sold_date_sk#82] -Right output [2]: [d_date_sk#84, d_date#85] -Arguments: [ss_sold_date_sk#82], [d_date_sk#84], Inner - -(134) CometProject -Input [5]: [ss_item_sk#80, ss_sales_price#81, ss_sold_date_sk#82, d_date_sk#84, d_date#85] -Arguments: [ss_item_sk#80, ss_sales_price#81, d_date#85], [ss_item_sk#80, ss_sales_price#81, d_date#85] - -(135) CometHashAggregate -Input [3]: [ss_item_sk#80, ss_sales_price#81, d_date#85] -Keys [2]: [ss_item_sk#80, d_date#85] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#81))] - -(136) ColumnarToRow [codegen id : 33] -Input [3]: [ss_item_sk#80, d_date#85, sum#87] - -(137) Exchange -Input [3]: [ss_item_sk#80, d_date#85, sum#87] -Arguments: hashpartitioning(ss_item_sk#80, d_date#85, 5), ENSURE_REQUIREMENTS, [plan_id=23] - -(138) HashAggregate [codegen id : 34] -Input [3]: [ss_item_sk#80, d_date#85, sum#87] -Keys [2]: [ss_item_sk#80, d_date#85] -Functions [1]: [sum(UnscaledValue(ss_sales_price#81))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#81))#38] -Results [4]: [ss_item_sk#80 AS item_sk#39, d_date#85, MakeDecimal(sum(UnscaledValue(ss_sales_price#81))#38,17,2) AS sumss#40, ss_item_sk#80] - -(139) Exchange -Input [4]: [item_sk#39, d_date#85, sumss#40, ss_item_sk#80] -Arguments: hashpartitioning(ss_item_sk#80, 5), ENSURE_REQUIREMENTS, [plan_id=24] - -(140) Sort [codegen id : 35] -Input [4]: [item_sk#39, d_date#85, sumss#40, ss_item_sk#80] -Arguments: [ss_item_sk#80 ASC NULLS FIRST, d_date#85 ASC NULLS FIRST], false, 0 - -(141) Window -Input [4]: [item_sk#39, d_date#85, sumss#40, ss_item_sk#80] -Arguments: [row_number() windowspecdefinition(ss_item_sk#80, d_date#85 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#41], [ss_item_sk#80], [d_date#85 ASC NULLS FIRST] - -(142) Project [codegen id : 40] -Output [4]: [item_sk#39, d_date#85, sumss#40, rk#41] -Input [5]: [item_sk#39, d_date#85, sumss#40, ss_item_sk#80, rk#41] - -(143) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#88, ss_sales_price#89, ss_sold_date_sk#90] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#90), dynamicpruningexpression(ss_sold_date_sk#90 IN dynamicpruning#91)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(144) CometFilter -Input [3]: [ss_item_sk#88, ss_sales_price#89, ss_sold_date_sk#90] -Condition : isnotnull(ss_item_sk#88) - -(145) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#92, d_date#93, d_month_seq#94] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(146) CometFilter -Input [3]: [d_date_sk#92, d_date#93, d_month_seq#94] -Condition : (((isnotnull(d_month_seq#94) AND (d_month_seq#94 >= 1212)) AND (d_month_seq#94 <= 1223)) AND isnotnull(d_date_sk#92)) - -(147) CometProject -Input [3]: [d_date_sk#92, d_date#93, d_month_seq#94] -Arguments: [d_date_sk#92, d_date#93], [d_date_sk#92, d_date#93] - -(148) CometBroadcastExchange -Input [2]: [d_date_sk#92, d_date#93] -Arguments: [d_date_sk#92, d_date#93] - -(149) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#88, ss_sales_price#89, ss_sold_date_sk#90] -Right output [2]: [d_date_sk#92, d_date#93] -Arguments: [ss_sold_date_sk#90], [d_date_sk#92], Inner - -(150) CometProject -Input [5]: [ss_item_sk#88, ss_sales_price#89, ss_sold_date_sk#90, d_date_sk#92, d_date#93] -Arguments: [ss_item_sk#88, ss_sales_price#89, d_date#93], [ss_item_sk#88, ss_sales_price#89, d_date#93] - -(151) CometHashAggregate -Input [3]: [ss_item_sk#88, ss_sales_price#89, d_date#93] -Keys [2]: [ss_item_sk#88, d_date#93] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#89))] - -(152) ColumnarToRow [codegen id : 36] -Input [3]: [ss_item_sk#88, d_date#93, sum#95] - -(153) Exchange -Input [3]: [ss_item_sk#88, d_date#93, sum#95] -Arguments: hashpartitioning(ss_item_sk#88, d_date#93, 5), ENSURE_REQUIREMENTS, [plan_id=25] - -(154) HashAggregate [codegen id : 37] -Input [3]: [ss_item_sk#88, d_date#93, sum#95] -Keys [2]: [ss_item_sk#88, d_date#93] -Functions [1]: [sum(UnscaledValue(ss_sales_price#89))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#89))#38] -Results [4]: [ss_item_sk#88 AS item_sk#39, d_date#93, MakeDecimal(sum(UnscaledValue(ss_sales_price#89))#38,17,2) AS sumss#40, ss_item_sk#88] - -(155) Exchange -Input [4]: [item_sk#39, d_date#93, sumss#40, ss_item_sk#88] -Arguments: hashpartitioning(ss_item_sk#88, 5), ENSURE_REQUIREMENTS, [plan_id=26] - -(156) Sort [codegen id : 38] -Input [4]: [item_sk#39, d_date#93, sumss#40, ss_item_sk#88] -Arguments: [ss_item_sk#88 ASC NULLS FIRST, d_date#93 ASC NULLS FIRST], false, 0 - -(157) Window -Input [4]: [item_sk#39, d_date#93, sumss#40, ss_item_sk#88] -Arguments: [row_number() windowspecdefinition(ss_item_sk#88, d_date#93 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#50], [ss_item_sk#88], [d_date#93 ASC NULLS FIRST] - -(158) Project [codegen id : 39] -Output [3]: [item_sk#39 AS item_sk#51, sumss#40 AS sumss#52, rk#50] -Input [5]: [item_sk#39, d_date#93, sumss#40, ss_item_sk#88, rk#50] - -(159) BroadcastExchange -Input [3]: [item_sk#51, sumss#52, rk#50] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=27] - -(160) BroadcastHashJoin [codegen id : 40] -Left keys [1]: [item_sk#39] -Right keys [1]: [item_sk#51] -Join type: Inner -Join condition: (rk#41 >= rk#50) - -(161) Project [codegen id : 40] -Output [4]: [item_sk#39, d_date#85, sumss#40, sumss#52] -Input [7]: [item_sk#39, d_date#85, sumss#40, rk#41, item_sk#51, sumss#52, rk#50] - -(162) HashAggregate [codegen id : 40] -Input [4]: [item_sk#39, d_date#85, sumss#40, sumss#52] -Keys [3]: [item_sk#39, d_date#85, sumss#40] -Functions [1]: [partial_sum(sumss#52)] -Aggregate Attributes [2]: [sum#53, isEmpty#54] -Results [5]: [item_sk#39, d_date#85, sumss#40, sum#55, isEmpty#56] - -(163) Exchange -Input [5]: [item_sk#39, d_date#85, sumss#40, sum#55, isEmpty#56] -Arguments: hashpartitioning(item_sk#39, d_date#85, sumss#40, 5), ENSURE_REQUIREMENTS, [plan_id=28] - -(164) HashAggregate [codegen id : 41] -Input [5]: [item_sk#39, d_date#85, sumss#40, sum#55, isEmpty#56] -Keys [3]: [item_sk#39, d_date#85, sumss#40] -Functions [1]: [sum(sumss#52)] -Aggregate Attributes [1]: [sum(sumss#52)#57] -Results [3]: [item_sk#39, d_date#85, sum(sumss#52)#57 AS cume_sales#58] - -(165) Exchange -Input [3]: [item_sk#39, d_date#85, cume_sales#58] -Arguments: hashpartitioning(item_sk#39, d_date#85, 5), ENSURE_REQUIREMENTS, [plan_id=29] - -(166) Sort [codegen id : 42] -Input [3]: [item_sk#39, d_date#85, cume_sales#58] -Arguments: [item_sk#39 ASC NULLS FIRST, d_date#85 ASC NULLS FIRST], false, 0 - -(167) SortMergeJoin [codegen id : 43] -Left keys [2]: [item_sk#10, d_date#69] -Right keys [2]: [item_sk#39, d_date#85] -Join type: FullOuter -Join condition: None - -(168) Filter [codegen id : 43] -Input [6]: [item_sk#10, d_date#69, cume_sales#29, item_sk#39, d_date#85, cume_sales#58] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#39 END) - -(169) Project [codegen id : 43] -Output [4]: [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#39 END AS item_sk#59, CASE WHEN isnotnull(d_date#69) THEN d_date#69 ELSE d_date#85 END AS d_date#60, cume_sales#29 AS web_sales#61, cume_sales#58 AS store_sales#62] -Input [6]: [item_sk#10, d_date#69, cume_sales#29, item_sk#39, d_date#85, cume_sales#58] - -(170) Exchange -Input [4]: [item_sk#59, d_date#60, web_sales#61, store_sales#62] -Arguments: hashpartitioning(item_sk#59, 5), ENSURE_REQUIREMENTS, [plan_id=30] - -(171) Sort [codegen id : 44] -Input [4]: [item_sk#59, d_date#60, web_sales#61, store_sales#62] -Arguments: [item_sk#59 ASC NULLS FIRST, d_date#60 ASC NULLS FIRST], false, 0 - -(172) Window -Input [4]: [item_sk#59, d_date#60, web_sales#61, store_sales#62] -Arguments: [row_number() windowspecdefinition(item_sk#59, d_date#60 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#96], [item_sk#59], [d_date#60 ASC NULLS FIRST] - -(173) Project [codegen id : 45] -Output [4]: [item_sk#59 AS item_sk#97, web_sales#61 AS web_sales#98, store_sales#62 AS store_sales#99, rk#96] -Input [5]: [item_sk#59, d_date#60, web_sales#61, store_sales#62, rk#96] - -(174) BroadcastExchange -Input [4]: [item_sk#97, web_sales#98, store_sales#99, rk#96] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=31] +(64) BroadcastExchange +Input [4]: [item_sk#52, web_sales#53, store_sales#54, rk#51] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -(175) BroadcastHashJoin [codegen id : 46] -Left keys [1]: [item_sk#59] -Right keys [1]: [item_sk#97] +(65) BroadcastHashJoin [codegen id : 46] +Left keys [1]: [item_sk#46] +Right keys [1]: [item_sk#52] Join type: Inner -Join condition: (rk#63 >= rk#96) - -(176) Project [codegen id : 46] -Output [6]: [item_sk#59, d_date#60, web_sales#61, store_sales#62, web_sales#98, store_sales#99] -Input [9]: [item_sk#59, d_date#60, web_sales#61, store_sales#62, rk#63, item_sk#97, web_sales#98, store_sales#99, rk#96] - -(177) HashAggregate [codegen id : 46] -Input [6]: [item_sk#59, d_date#60, web_sales#61, store_sales#62, web_sales#98, store_sales#99] -Keys [4]: [item_sk#59, d_date#60, web_sales#61, store_sales#62] -Functions [2]: [partial_max(web_sales#98), partial_max(store_sales#99)] -Aggregate Attributes [2]: [max#100, max#101] -Results [6]: [item_sk#59, d_date#60, web_sales#61, store_sales#62, max#102, max#103] - -(178) HashAggregate [codegen id : 46] -Input [6]: [item_sk#59, d_date#60, web_sales#61, store_sales#62, max#102, max#103] -Keys [4]: [item_sk#59, d_date#60, web_sales#61, store_sales#62] -Functions [2]: [max(web_sales#98), max(store_sales#99)] -Aggregate Attributes [2]: [max(web_sales#98)#104, max(store_sales#99)#105] -Results [6]: [item_sk#59, d_date#60, web_sales#61, store_sales#62, max(web_sales#98)#104 AS web_cumulative#106, max(store_sales#99)#105 AS store_cumulative#107] - -(179) Filter [codegen id : 46] -Input [6]: [item_sk#59, d_date#60, web_sales#61, store_sales#62, web_cumulative#106, store_cumulative#107] -Condition : ((isnotnull(web_cumulative#106) AND isnotnull(store_cumulative#107)) AND (web_cumulative#106 > store_cumulative#107)) - -(180) TakeOrderedAndProject -Input [6]: [item_sk#59, d_date#60, web_sales#61, store_sales#62, web_cumulative#106, store_cumulative#107] -Arguments: 100, [item_sk#59 ASC NULLS FIRST, d_date#60 ASC NULLS FIRST], [item_sk#59, d_date#60, web_sales#61, store_sales#62, web_cumulative#106, store_cumulative#107] +Join condition: (rk#50 >= rk#51) + +(66) Project [codegen id : 46] +Output [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_sales#53, store_sales#54] +Input [9]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, rk#50, item_sk#52, web_sales#53, store_sales#54, rk#51] + +(67) HashAggregate [codegen id : 46] +Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_sales#53, store_sales#54] +Keys [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] +Functions [2]: [partial_max(web_sales#53), partial_max(store_sales#54)] +Aggregate Attributes [2]: [max#55, max#56] +Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#57, max#58] + +(68) HashAggregate [codegen id : 46] +Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#57, max#58] +Keys [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] +Functions [2]: [max(web_sales#53), max(store_sales#54)] +Aggregate Attributes [2]: [max(web_sales#53)#59, max(store_sales#54)#60] +Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max(web_sales#53)#59 AS web_cumulative#61, max(store_sales#54)#60 AS store_cumulative#62] + +(69) Filter [codegen id : 46] +Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#61, store_cumulative#62] +Condition : ((isnotnull(web_cumulative#61) AND isnotnull(store_cumulative#62)) AND (web_cumulative#61 > store_cumulative#62)) + +(70) TakeOrderedAndProject +Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#61, store_cumulative#62] +Arguments: 100, [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST], [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#61, store_cumulative#62] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (185) -+- * ColumnarToRow (184) - +- CometProject (183) - +- CometFilter (182) - +- CometScan parquet spark_catalog.default.date_dim (181) +BroadcastExchange (75) ++- * ColumnarToRow (74) + +- CometProject (73) + +- CometFilter (72) + +- CometScan parquet spark_catalog.default.date_dim (71) -(181) Scan parquet spark_catalog.default.date_dim +(71) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(182) CometFilter +(72) CometFilter Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#5)) -(183) CometProject +(73) CometProject Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(184) ColumnarToRow [codegen id : 1] +(74) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(185) BroadcastExchange +(75) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=32] - -Subquery:2 Hosting operator id = 17 Hosting Expression = ws_sold_date_sk#15 IN dynamicpruning#4 - -Subquery:3 Hosting operator id = 41 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#4 - -Subquery:4 Hosting operator id = 57 Hosting Expression = ss_sold_date_sk#44 IN dynamicpruning#4 - -Subquery:5 Hosting operator id = 87 Hosting Expression = ws_sold_date_sk#66 IN dynamicpruning#4 - -Subquery:6 Hosting operator id = 103 Hosting Expression = ws_sold_date_sk#74 IN dynamicpruning#4 - -Subquery:7 Hosting operator id = 127 Hosting Expression = ss_sold_date_sk#82 IN dynamicpruning#4 +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:8 Hosting operator id = 143 Hosting Expression = ss_sold_date_sk#90 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 29 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt index f587cfbaf..78dd29c2a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt @@ -68,33 +68,16 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (6) Sort [ws_item_sk,d_date] InputAdapter - Exchange [ws_item_sk] #9 - WholeStageCodegen (5) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] - InputAdapter - Exchange [ws_item_sk,d_date] #10 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometHashAggregate [ws_item_sk,d_date,ws_sales_price] - CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #11 - CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 InputAdapter WholeStageCodegen (20) Sort [item_sk,d_date] InputAdapter - Exchange [item_sk,d_date] #12 + Exchange [item_sk,d_date] #9 WholeStageCodegen (19) HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] InputAdapter - Exchange [item_sk,d_date,sumss] #13 + Exchange [item_sk,d_date,sumss] #10 WholeStageCodegen (18) HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] Project [item_sk,d_date,sumss,sumss] @@ -105,11 +88,11 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (13) Sort [ss_item_sk,d_date] InputAdapter - Exchange [ss_item_sk] #14 + Exchange [ss_item_sk] #11 WholeStageCodegen (12) HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] InputAdapter - Exchange [ss_item_sk,d_date] #15 + Exchange [ss_item_sk,d_date] #12 WholeStageCodegen (11) ColumnarToRow InputAdapter @@ -119,12 +102,9 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #16 - CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + ReusedExchange [d_date_sk,d_date] #7 InputAdapter - BroadcastExchange #17 + BroadcastExchange #13 WholeStageCodegen (17) Project [item_sk,sumss,rk] InputAdapter @@ -132,26 +112,9 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (16) Sort [ss_item_sk,d_date] InputAdapter - Exchange [ss_item_sk] #18 - WholeStageCodegen (15) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] - InputAdapter - Exchange [ss_item_sk,d_date] #19 - WholeStageCodegen (14) - ColumnarToRow - InputAdapter - CometHashAggregate [ss_item_sk,d_date,ss_sales_price] - CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #20 - CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 InputAdapter - BroadcastExchange #21 + BroadcastExchange #14 WholeStageCodegen (45) Project [item_sk,web_sales,store_sales,rk] InputAdapter @@ -159,136 +122,4 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (44) Sort [item_sk,d_date] InputAdapter - Exchange [item_sk] #22 - WholeStageCodegen (43) - Project [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] - Filter [item_sk,item_sk] - SortMergeJoin [item_sk,d_date,item_sk,d_date] - InputAdapter - WholeStageCodegen (32) - Sort [item_sk,d_date] - InputAdapter - Exchange [item_sk,d_date] #23 - WholeStageCodegen (31) - HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] - InputAdapter - Exchange [item_sk,d_date,sumws] #24 - WholeStageCodegen (30) - HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumws,sumws] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (25) - Sort [ws_item_sk,d_date] - InputAdapter - Exchange [ws_item_sk] #25 - WholeStageCodegen (24) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] - InputAdapter - Exchange [ws_item_sk,d_date] #26 - WholeStageCodegen (23) - ColumnarToRow - InputAdapter - CometHashAggregate [ws_item_sk,d_date,ws_sales_price] - CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #27 - CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - BroadcastExchange #28 - WholeStageCodegen (29) - Project [item_sk,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (28) - Sort [ws_item_sk,d_date] - InputAdapter - Exchange [ws_item_sk] #29 - WholeStageCodegen (27) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] - InputAdapter - Exchange [ws_item_sk,d_date] #30 - WholeStageCodegen (26) - ColumnarToRow - InputAdapter - CometHashAggregate [ws_item_sk,d_date,ws_sales_price] - CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #31 - CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - WholeStageCodegen (42) - Sort [item_sk,d_date] - InputAdapter - Exchange [item_sk,d_date] #32 - WholeStageCodegen (41) - HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] - InputAdapter - Exchange [item_sk,d_date,sumss] #33 - WholeStageCodegen (40) - HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumss,sumss] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (35) - Sort [ss_item_sk,d_date] - InputAdapter - Exchange [ss_item_sk] #34 - WholeStageCodegen (34) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] - InputAdapter - Exchange [ss_item_sk,d_date] #35 - WholeStageCodegen (33) - ColumnarToRow - InputAdapter - CometHashAggregate [ss_item_sk,d_date,ss_sales_price] - CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #36 - CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - BroadcastExchange #37 - WholeStageCodegen (39) - Project [item_sk,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (38) - Sort [ss_item_sk,d_date] - InputAdapter - Exchange [ss_item_sk] #38 - WholeStageCodegen (37) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] - InputAdapter - Exchange [ss_item_sk,d_date] #39 - WholeStageCodegen (36) - ColumnarToRow - InputAdapter - CometHashAggregate [ss_item_sk,d_date,ss_sales_price] - CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #40 - CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt index 02d0803eb..f7fa9d271 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (85) -+- * Project (84) - +- * BroadcastHashJoin Inner BuildRight (83) - :- * Project (56) - : +- * BroadcastHashJoin Inner BuildRight (55) +TakeOrderedAndProject (45) ++- * Project (44) + +- * BroadcastHashJoin Inner BuildRight (43) + :- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) : :- * Project (28) : : +- * Filter (27) : : +- Window (26) @@ -32,58 +32,18 @@ TakeOrderedAndProject (85) : : +- CometBroadcastExchange (15) : : +- CometFilter (14) : : +- CometScan parquet spark_catalog.default.call_center (13) - : +- BroadcastExchange (54) - : +- * Project (53) - : +- Window (52) - : +- * Sort (51) - : +- Exchange (50) - : +- * HashAggregate (49) - : +- Exchange (48) - : +- * ColumnarToRow (47) - : +- CometHashAggregate (46) - : +- CometProject (45) - : +- CometBroadcastHashJoin (44) - : :- CometProject (40) - : : +- CometBroadcastHashJoin (39) - : : :- CometProject (35) - : : : +- CometBroadcastHashJoin (34) - : : : :- CometFilter (30) - : : : : +- CometScan parquet spark_catalog.default.item (29) - : : : +- CometBroadcastExchange (33) - : : : +- CometFilter (32) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (31) - : : +- CometBroadcastExchange (38) - : : +- CometFilter (37) - : : +- CometScan parquet spark_catalog.default.date_dim (36) - : +- CometBroadcastExchange (43) - : +- CometFilter (42) - : +- CometScan parquet spark_catalog.default.call_center (41) - +- BroadcastExchange (82) - +- * Project (81) - +- Window (80) - +- * Sort (79) - +- Exchange (78) - +- * HashAggregate (77) - +- Exchange (76) - +- * ColumnarToRow (75) - +- CometHashAggregate (74) - +- CometProject (73) - +- CometBroadcastHashJoin (72) - :- CometProject (68) - : +- CometBroadcastHashJoin (67) - : :- CometProject (63) - : : +- CometBroadcastHashJoin (62) - : : :- CometFilter (58) - : : : +- CometScan parquet spark_catalog.default.item (57) - : : +- CometBroadcastExchange (61) - : : +- CometFilter (60) - : : +- CometScan parquet spark_catalog.default.catalog_sales (59) - : +- CometBroadcastExchange (66) - : +- CometFilter (65) - : +- CometScan parquet spark_catalog.default.date_dim (64) - +- CometBroadcastExchange (71) - +- CometFilter (70) - +- CometScan parquet spark_catalog.default.call_center (69) + : +- BroadcastExchange (35) + : +- * Project (34) + : +- Window (33) + : +- * Sort (32) + : +- Exchange (31) + : +- * HashAggregate (30) + : +- ReusedExchange (29) + +- BroadcastExchange (42) + +- * Project (41) + +- Window (40) + +- * Sort (39) + +- ReusedExchange (38) (1) Scan parquet spark_catalog.default.item @@ -217,305 +177,104 @@ Condition : ((isnotnull(avg_monthly_sales#19) AND (avg_monthly_sales#19 > 0.0000 Output [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18] Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17, rn#18, avg_monthly_sales#19] -(29) Scan parquet spark_catalog.default.item -Output [3]: [i_item_sk#20, i_brand#21, i_category#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_category), IsNotNull(i_brand)] -ReadSchema: struct +(29) ReusedExchange [Reuses operator id: 20] +Output [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum#25] -(30) CometFilter -Input [3]: [i_item_sk#20, i_brand#21, i_category#22] -Condition : ((isnotnull(i_item_sk#20) AND isnotnull(i_category#22)) AND isnotnull(i_brand#21)) +(30) HashAggregate [codegen id : 6] +Input [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum#25] +Keys [5]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24] +Functions [1]: [sum(UnscaledValue(cs_sales_price#26))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#26))#15] +Results [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, MakeDecimal(sum(UnscaledValue(cs_sales_price#26))#15,17,2) AS sum_sales#16] -(31) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#23, cs_item_sk#24, cs_sales_price#25, cs_sold_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#26), dynamicpruningexpression(cs_sold_date_sk#26 IN dynamicpruning#27)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_call_center_sk)] -ReadSchema: struct +(31) Exchange +Input [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16] +Arguments: hashpartitioning(i_category#20, i_brand#21, cc_name#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(32) CometFilter -Input [4]: [cs_call_center_sk#23, cs_item_sk#24, cs_sales_price#25, cs_sold_date_sk#26] -Condition : (isnotnull(cs_item_sk#24) AND isnotnull(cs_call_center_sk#23)) +(32) Sort [codegen id : 7] +Input [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16] +Arguments: [i_category#20 ASC NULLS FIRST, i_brand#21 ASC NULLS FIRST, cc_name#22 ASC NULLS FIRST, d_year#23 ASC NULLS FIRST, d_moy#24 ASC NULLS FIRST], false, 0 -(33) CometBroadcastExchange -Input [4]: [cs_call_center_sk#23, cs_item_sk#24, cs_sales_price#25, cs_sold_date_sk#26] -Arguments: [cs_call_center_sk#23, cs_item_sk#24, cs_sales_price#25, cs_sold_date_sk#26] +(33) Window +Input [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16] +Arguments: [rank(d_year#23, d_moy#24) windowspecdefinition(i_category#20, i_brand#21, cc_name#22, d_year#23 ASC NULLS FIRST, d_moy#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#27], [i_category#20, i_brand#21, cc_name#22], [d_year#23 ASC NULLS FIRST, d_moy#24 ASC NULLS FIRST] -(34) CometBroadcastHashJoin -Left output [3]: [i_item_sk#20, i_brand#21, i_category#22] -Right output [4]: [cs_call_center_sk#23, cs_item_sk#24, cs_sales_price#25, cs_sold_date_sk#26] -Arguments: [i_item_sk#20], [cs_item_sk#24], Inner +(34) Project [codegen id : 8] +Output [5]: [i_category#20, i_brand#21, cc_name#22, sum_sales#16 AS sum_sales#28, rn#27] +Input [7]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16, rn#27] -(35) CometProject -Input [7]: [i_item_sk#20, i_brand#21, i_category#22, cs_call_center_sk#23, cs_item_sk#24, cs_sales_price#25, cs_sold_date_sk#26] -Arguments: [i_brand#21, i_category#22, cs_call_center_sk#23, cs_sales_price#25, cs_sold_date_sk#26], [i_brand#21, i_category#22, cs_call_center_sk#23, cs_sales_price#25, cs_sold_date_sk#26] +(35) BroadcastExchange +Input [5]: [i_category#20, i_brand#21, cc_name#22, sum_sales#28, rn#27] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] -(36) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#28, d_year#29, d_moy#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(37) CometFilter -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Condition : ((((d_year#29 = 1999) OR ((d_year#29 = 1998) AND (d_moy#30 = 12))) OR ((d_year#29 = 2000) AND (d_moy#30 = 1))) AND isnotnull(d_date_sk#28)) - -(38) CometBroadcastExchange -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Arguments: [d_date_sk#28, d_year#29, d_moy#30] - -(39) CometBroadcastHashJoin -Left output [5]: [i_brand#21, i_category#22, cs_call_center_sk#23, cs_sales_price#25, cs_sold_date_sk#26] -Right output [3]: [d_date_sk#28, d_year#29, d_moy#30] -Arguments: [cs_sold_date_sk#26], [d_date_sk#28], Inner - -(40) CometProject -Input [8]: [i_brand#21, i_category#22, cs_call_center_sk#23, cs_sales_price#25, cs_sold_date_sk#26, d_date_sk#28, d_year#29, d_moy#30] -Arguments: [i_brand#21, i_category#22, cs_call_center_sk#23, cs_sales_price#25, d_year#29, d_moy#30], [i_brand#21, i_category#22, cs_call_center_sk#23, cs_sales_price#25, d_year#29, d_moy#30] - -(41) Scan parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#31, cc_name#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk), IsNotNull(cc_name)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [cc_call_center_sk#31, cc_name#32] -Condition : (isnotnull(cc_call_center_sk#31) AND isnotnull(cc_name#32)) - -(43) CometBroadcastExchange -Input [2]: [cc_call_center_sk#31, cc_name#32] -Arguments: [cc_call_center_sk#31, cc_name#32] - -(44) CometBroadcastHashJoin -Left output [6]: [i_brand#21, i_category#22, cs_call_center_sk#23, cs_sales_price#25, d_year#29, d_moy#30] -Right output [2]: [cc_call_center_sk#31, cc_name#32] -Arguments: [cs_call_center_sk#23], [cc_call_center_sk#31], Inner - -(45) CometProject -Input [8]: [i_brand#21, i_category#22, cs_call_center_sk#23, cs_sales_price#25, d_year#29, d_moy#30, cc_call_center_sk#31, cc_name#32] -Arguments: [i_brand#21, i_category#22, cs_sales_price#25, d_year#29, d_moy#30, cc_name#32], [i_brand#21, i_category#22, cs_sales_price#25, d_year#29, d_moy#30, cc_name#32] - -(46) CometHashAggregate -Input [6]: [i_brand#21, i_category#22, cs_sales_price#25, d_year#29, d_moy#30, cc_name#32] -Keys [5]: [i_category#22, i_brand#21, cc_name#32, d_year#29, d_moy#30] -Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#25))] - -(47) ColumnarToRow [codegen id : 5] -Input [6]: [i_category#22, i_brand#21, cc_name#32, d_year#29, d_moy#30, sum#33] - -(48) Exchange -Input [6]: [i_category#22, i_brand#21, cc_name#32, d_year#29, d_moy#30, sum#33] -Arguments: hashpartitioning(i_category#22, i_brand#21, cc_name#32, d_year#29, d_moy#30, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(49) HashAggregate [codegen id : 6] -Input [6]: [i_category#22, i_brand#21, cc_name#32, d_year#29, d_moy#30, sum#33] -Keys [5]: [i_category#22, i_brand#21, cc_name#32, d_year#29, d_moy#30] -Functions [1]: [sum(UnscaledValue(cs_sales_price#25))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#25))#15] -Results [6]: [i_category#22, i_brand#21, cc_name#32, d_year#29, d_moy#30, MakeDecimal(sum(UnscaledValue(cs_sales_price#25))#15,17,2) AS sum_sales#16] - -(50) Exchange -Input [6]: [i_category#22, i_brand#21, cc_name#32, d_year#29, d_moy#30, sum_sales#16] -Arguments: hashpartitioning(i_category#22, i_brand#21, cc_name#32, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(51) Sort [codegen id : 7] -Input [6]: [i_category#22, i_brand#21, cc_name#32, d_year#29, d_moy#30, sum_sales#16] -Arguments: [i_category#22 ASC NULLS FIRST, i_brand#21 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST], false, 0 - -(52) Window -Input [6]: [i_category#22, i_brand#21, cc_name#32, d_year#29, d_moy#30, sum_sales#16] -Arguments: [rank(d_year#29, d_moy#30) windowspecdefinition(i_category#22, i_brand#21, cc_name#32, d_year#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#34], [i_category#22, i_brand#21, cc_name#32], [d_year#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST] - -(53) Project [codegen id : 8] -Output [5]: [i_category#22, i_brand#21, cc_name#32, sum_sales#16 AS sum_sales#35, rn#34] -Input [7]: [i_category#22, i_brand#21, cc_name#32, d_year#29, d_moy#30, sum_sales#16, rn#34] - -(54) BroadcastExchange -Input [5]: [i_category#22, i_brand#21, cc_name#32, sum_sales#35, rn#34] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=5] - -(55) BroadcastHashJoin [codegen id : 13] +(36) BroadcastHashJoin [codegen id : 13] Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#18] -Right keys [4]: [i_category#22, i_brand#21, cc_name#32, (rn#34 + 1)] +Right keys [4]: [i_category#20, i_brand#21, cc_name#22, (rn#27 + 1)] Join type: Inner Join condition: None -(56) Project [codegen id : 13] -Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, sum_sales#35] -Input [13]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, i_category#22, i_brand#21, cc_name#32, sum_sales#35, rn#34] - -(57) Scan parquet spark_catalog.default.item -Output [3]: [i_item_sk#36, i_brand#37, i_category#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_category), IsNotNull(i_brand)] -ReadSchema: struct - -(58) CometFilter -Input [3]: [i_item_sk#36, i_brand#37, i_category#38] -Condition : ((isnotnull(i_item_sk#36) AND isnotnull(i_category#38)) AND isnotnull(i_brand#37)) +(37) Project [codegen id : 13] +Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, sum_sales#28] +Input [13]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, i_category#20, i_brand#21, cc_name#22, sum_sales#28, rn#27] -(59) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#39, cs_item_sk#40, cs_sales_price#41, cs_sold_date_sk#42] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#42), dynamicpruningexpression(cs_sold_date_sk#42 IN dynamicpruning#43)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_call_center_sk)] -ReadSchema: struct +(38) ReusedExchange [Reuses operator id: 31] +Output [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#16] -(60) CometFilter -Input [4]: [cs_call_center_sk#39, cs_item_sk#40, cs_sales_price#41, cs_sold_date_sk#42] -Condition : (isnotnull(cs_item_sk#40) AND isnotnull(cs_call_center_sk#39)) +(39) Sort [codegen id : 11] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#16] +Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST], false, 0 -(61) CometBroadcastExchange -Input [4]: [cs_call_center_sk#39, cs_item_sk#40, cs_sales_price#41, cs_sold_date_sk#42] -Arguments: [cs_call_center_sk#39, cs_item_sk#40, cs_sales_price#41, cs_sold_date_sk#42] +(40) Window +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#16] +Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#29, i_brand#30, cc_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#34], [i_category#29, i_brand#30, cc_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] -(62) CometBroadcastHashJoin -Left output [3]: [i_item_sk#36, i_brand#37, i_category#38] -Right output [4]: [cs_call_center_sk#39, cs_item_sk#40, cs_sales_price#41, cs_sold_date_sk#42] -Arguments: [i_item_sk#36], [cs_item_sk#40], Inner +(41) Project [codegen id : 12] +Output [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#16 AS sum_sales#35, rn#34] +Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#16, rn#34] -(63) CometProject -Input [7]: [i_item_sk#36, i_brand#37, i_category#38, cs_call_center_sk#39, cs_item_sk#40, cs_sales_price#41, cs_sold_date_sk#42] -Arguments: [i_brand#37, i_category#38, cs_call_center_sk#39, cs_sales_price#41, cs_sold_date_sk#42], [i_brand#37, i_category#38, cs_call_center_sk#39, cs_sales_price#41, cs_sold_date_sk#42] +(42) BroadcastExchange +Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#35, rn#34] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] -(64) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#44, d_year#45, d_moy#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(65) CometFilter -Input [3]: [d_date_sk#44, d_year#45, d_moy#46] -Condition : ((((d_year#45 = 1999) OR ((d_year#45 = 1998) AND (d_moy#46 = 12))) OR ((d_year#45 = 2000) AND (d_moy#46 = 1))) AND isnotnull(d_date_sk#44)) - -(66) CometBroadcastExchange -Input [3]: [d_date_sk#44, d_year#45, d_moy#46] -Arguments: [d_date_sk#44, d_year#45, d_moy#46] - -(67) CometBroadcastHashJoin -Left output [5]: [i_brand#37, i_category#38, cs_call_center_sk#39, cs_sales_price#41, cs_sold_date_sk#42] -Right output [3]: [d_date_sk#44, d_year#45, d_moy#46] -Arguments: [cs_sold_date_sk#42], [d_date_sk#44], Inner - -(68) CometProject -Input [8]: [i_brand#37, i_category#38, cs_call_center_sk#39, cs_sales_price#41, cs_sold_date_sk#42, d_date_sk#44, d_year#45, d_moy#46] -Arguments: [i_brand#37, i_category#38, cs_call_center_sk#39, cs_sales_price#41, d_year#45, d_moy#46], [i_brand#37, i_category#38, cs_call_center_sk#39, cs_sales_price#41, d_year#45, d_moy#46] - -(69) Scan parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#47, cc_name#48] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk), IsNotNull(cc_name)] -ReadSchema: struct - -(70) CometFilter -Input [2]: [cc_call_center_sk#47, cc_name#48] -Condition : (isnotnull(cc_call_center_sk#47) AND isnotnull(cc_name#48)) - -(71) CometBroadcastExchange -Input [2]: [cc_call_center_sk#47, cc_name#48] -Arguments: [cc_call_center_sk#47, cc_name#48] - -(72) CometBroadcastHashJoin -Left output [6]: [i_brand#37, i_category#38, cs_call_center_sk#39, cs_sales_price#41, d_year#45, d_moy#46] -Right output [2]: [cc_call_center_sk#47, cc_name#48] -Arguments: [cs_call_center_sk#39], [cc_call_center_sk#47], Inner - -(73) CometProject -Input [8]: [i_brand#37, i_category#38, cs_call_center_sk#39, cs_sales_price#41, d_year#45, d_moy#46, cc_call_center_sk#47, cc_name#48] -Arguments: [i_brand#37, i_category#38, cs_sales_price#41, d_year#45, d_moy#46, cc_name#48], [i_brand#37, i_category#38, cs_sales_price#41, d_year#45, d_moy#46, cc_name#48] - -(74) CometHashAggregate -Input [6]: [i_brand#37, i_category#38, cs_sales_price#41, d_year#45, d_moy#46, cc_name#48] -Keys [5]: [i_category#38, i_brand#37, cc_name#48, d_year#45, d_moy#46] -Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#41))] - -(75) ColumnarToRow [codegen id : 9] -Input [6]: [i_category#38, i_brand#37, cc_name#48, d_year#45, d_moy#46, sum#49] - -(76) Exchange -Input [6]: [i_category#38, i_brand#37, cc_name#48, d_year#45, d_moy#46, sum#49] -Arguments: hashpartitioning(i_category#38, i_brand#37, cc_name#48, d_year#45, d_moy#46, 5), ENSURE_REQUIREMENTS, [plan_id=6] - -(77) HashAggregate [codegen id : 10] -Input [6]: [i_category#38, i_brand#37, cc_name#48, d_year#45, d_moy#46, sum#49] -Keys [5]: [i_category#38, i_brand#37, cc_name#48, d_year#45, d_moy#46] -Functions [1]: [sum(UnscaledValue(cs_sales_price#41))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#41))#15] -Results [6]: [i_category#38, i_brand#37, cc_name#48, d_year#45, d_moy#46, MakeDecimal(sum(UnscaledValue(cs_sales_price#41))#15,17,2) AS sum_sales#16] - -(78) Exchange -Input [6]: [i_category#38, i_brand#37, cc_name#48, d_year#45, d_moy#46, sum_sales#16] -Arguments: hashpartitioning(i_category#38, i_brand#37, cc_name#48, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(79) Sort [codegen id : 11] -Input [6]: [i_category#38, i_brand#37, cc_name#48, d_year#45, d_moy#46, sum_sales#16] -Arguments: [i_category#38 ASC NULLS FIRST, i_brand#37 ASC NULLS FIRST, cc_name#48 ASC NULLS FIRST, d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST], false, 0 - -(80) Window -Input [6]: [i_category#38, i_brand#37, cc_name#48, d_year#45, d_moy#46, sum_sales#16] -Arguments: [rank(d_year#45, d_moy#46) windowspecdefinition(i_category#38, i_brand#37, cc_name#48, d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#50], [i_category#38, i_brand#37, cc_name#48], [d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST] - -(81) Project [codegen id : 12] -Output [5]: [i_category#38, i_brand#37, cc_name#48, sum_sales#16 AS sum_sales#51, rn#50] -Input [7]: [i_category#38, i_brand#37, cc_name#48, d_year#45, d_moy#46, sum_sales#16, rn#50] - -(82) BroadcastExchange -Input [5]: [i_category#38, i_brand#37, cc_name#48, sum_sales#51, rn#50] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=8] - -(83) BroadcastHashJoin [codegen id : 13] +(43) BroadcastHashJoin [codegen id : 13] Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#18] -Right keys [4]: [i_category#38, i_brand#37, cc_name#48, (rn#50 - 1)] +Right keys [4]: [i_category#29, i_brand#30, cc_name#31, (rn#34 - 1)] Join type: Inner Join condition: None -(84) Project [codegen id : 13] -Output [8]: [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#19, sum_sales#16, sum_sales#35 AS psum#52, sum_sales#51 AS nsum#53] -Input [14]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, sum_sales#35, i_category#38, i_brand#37, cc_name#48, sum_sales#51, rn#50] +(44) Project [codegen id : 13] +Output [8]: [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#19, sum_sales#16, sum_sales#28 AS psum#36, sum_sales#35 AS nsum#37] +Input [14]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, sum_sales#28, i_category#29, i_brand#30, cc_name#31, sum_sales#35, rn#34] -(85) TakeOrderedAndProject -Input [8]: [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#19, sum_sales#16, psum#52, nsum#53] -Arguments: 100, [(sum_sales#16 - avg_monthly_sales#19) ASC NULLS FIRST, d_year#10 ASC NULLS FIRST], [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#19, sum_sales#16, psum#52, nsum#53] +(45) TakeOrderedAndProject +Input [8]: [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#19, sum_sales#16, psum#36, nsum#37] +Arguments: 100, [(sum_sales#16 - avg_monthly_sales#19) ASC NULLS FIRST, d_year#10 ASC NULLS FIRST], [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#19, sum_sales#16, psum#36, nsum#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = cs_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (89) -+- * ColumnarToRow (88) - +- CometFilter (87) - +- CometScan parquet spark_catalog.default.date_dim (86) +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) -(86) Scan parquet spark_catalog.default.date_dim +(46) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#9, d_year#10, d_moy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(87) CometFilter +(47) CometFilter Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) -(88) ColumnarToRow [codegen id : 1] +(48) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -(89) BroadcastExchange +(49) BroadcastExchange Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 31 Hosting Expression = cs_sold_date_sk#26 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 59 Hosting Expression = cs_sold_date_sk#42 IN dynamicpruning#8 +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-v2_7/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt index a90a98278..c2c8e089f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt @@ -61,31 +61,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m WholeStageCodegen (6) HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] InputAdapter - Exchange [i_category,i_brand,cc_name,d_year,d_moy] #9 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,cs_item_sk] - CometFilter [i_item_sk,i_category,i_brand] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #10 - CometFilter [cs_item_sk,cs_call_center_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #11 - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #12 - CometFilter [cc_call_center_sk,cc_name] - CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 InputAdapter - BroadcastExchange #13 + BroadcastExchange #9 WholeStageCodegen (12) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter @@ -93,30 +71,4 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m WholeStageCodegen (11) Sort [i_category,i_brand,cc_name,d_year,d_moy] InputAdapter - Exchange [i_category,i_brand,cc_name] #14 - WholeStageCodegen (10) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] - InputAdapter - Exchange [i_category,i_brand,cc_name,d_year,d_moy] #15 - WholeStageCodegen (9) - ColumnarToRow - InputAdapter - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,cs_item_sk] - CometFilter [i_item_sk,i_category,i_brand] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #16 - CometFilter [cs_item_sk,cs_call_center_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #17 - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #18 - CometFilter [cc_call_center_sk,cc_name] - CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt index 218c39f85..dc7f857a4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt @@ -1,13 +1,13 @@ == Physical Plan == -TakeOrderedAndProject (143) -+- * HashAggregate (142) - +- Exchange (141) - +- * HashAggregate (140) - +- Union (139) - :- * HashAggregate (74) - : +- Exchange (73) - : +- * HashAggregate (72) - : +- Union (71) +TakeOrderedAndProject (83) ++- * HashAggregate (82) + +- Exchange (81) + +- * HashAggregate (80) + +- Union (79) + :- * HashAggregate (68) + : +- Exchange (67) + : +- * HashAggregate (66) + : +- Union (65) : :- * HashAggregate (22) : : +- Exchange (21) : : +- * ColumnarToRow (20) @@ -30,14 +30,14 @@ TakeOrderedAndProject (143) : : +- CometBroadcastExchange (16) : : +- CometFilter (15) : : +- CometScan parquet spark_catalog.default.store (14) - : :- * HashAggregate (44) - : : +- Exchange (43) - : : +- * ColumnarToRow (42) - : : +- CometHashAggregate (41) - : : +- CometProject (40) - : : +- CometBroadcastHashJoin (39) - : : :- CometProject (35) - : : : +- CometBroadcastHashJoin (34) + : :- * HashAggregate (41) + : : +- Exchange (40) + : : +- * ColumnarToRow (39) + : : +- CometHashAggregate (38) + : : +- CometProject (37) + : : +- CometBroadcastHashJoin (36) + : : :- CometProject (32) + : : : +- CometBroadcastHashJoin (31) : : : :- CometUnion (29) : : : : :- CometProject (25) : : : : : +- CometFilter (24) @@ -45,103 +45,43 @@ TakeOrderedAndProject (143) : : : : +- CometProject (28) : : : : +- CometFilter (27) : : : : +- CometScan parquet spark_catalog.default.catalog_returns (26) - : : : +- CometBroadcastExchange (33) - : : : +- CometProject (32) - : : : +- CometFilter (31) - : : : +- CometScan parquet spark_catalog.default.date_dim (30) - : : +- CometBroadcastExchange (38) - : : +- CometFilter (37) - : : +- CometScan parquet spark_catalog.default.catalog_page (36) - : +- * HashAggregate (70) - : +- Exchange (69) - : +- * ColumnarToRow (68) - : +- CometHashAggregate (67) - : +- CometProject (66) - : +- CometBroadcastHashJoin (65) - : :- CometProject (61) - : : +- CometBroadcastHashJoin (60) - : : :- CometUnion (55) - : : : :- CometProject (47) - : : : : +- CometFilter (46) - : : : : +- CometScan parquet spark_catalog.default.web_sales (45) - : : : +- CometProject (54) - : : : +- CometBroadcastHashJoin (53) - : : : :- CometBroadcastExchange (49) - : : : : +- CometScan parquet spark_catalog.default.web_returns (48) - : : : +- CometProject (52) - : : : +- CometFilter (51) - : : : +- CometScan parquet spark_catalog.default.web_sales (50) - : : +- CometBroadcastExchange (59) - : : +- CometProject (58) - : : +- CometFilter (57) - : : +- CometScan parquet spark_catalog.default.date_dim (56) - : +- CometBroadcastExchange (64) - : +- CometFilter (63) - : +- CometScan parquet spark_catalog.default.web_site (62) - :- * HashAggregate (106) - : +- Exchange (105) - : +- * HashAggregate (104) - : +- * HashAggregate (103) - : +- Exchange (102) - : +- * HashAggregate (101) - : +- Union (100) - : :- * HashAggregate (76) - : : +- ReusedExchange (75) - : :- * HashAggregate (78) - : : +- ReusedExchange (77) - : +- * HashAggregate (99) - : +- Exchange (98) - : +- * ColumnarToRow (97) - : +- CometHashAggregate (96) - : +- CometProject (95) - : +- CometBroadcastHashJoin (94) - : :- CometProject (92) - : : +- CometBroadcastHashJoin (91) - : : :- CometUnion (89) - : : : :- CometProject (81) - : : : : +- CometFilter (80) - : : : : +- CometScan parquet spark_catalog.default.web_sales (79) - : : : +- CometProject (88) - : : : +- CometBroadcastHashJoin (87) - : : : :- CometBroadcastExchange (83) - : : : : +- CometScan parquet spark_catalog.default.web_returns (82) - : : : +- CometProject (86) - : : : +- CometFilter (85) - : : : +- CometScan parquet spark_catalog.default.web_sales (84) - : : +- ReusedExchange (90) - : +- ReusedExchange (93) - +- * HashAggregate (138) - +- Exchange (137) - +- * HashAggregate (136) - +- * HashAggregate (135) - +- Exchange (134) - +- * HashAggregate (133) - +- Union (132) - :- * HashAggregate (108) - : +- ReusedExchange (107) - :- * HashAggregate (110) - : +- ReusedExchange (109) - +- * HashAggregate (131) - +- Exchange (130) - +- * ColumnarToRow (129) - +- CometHashAggregate (128) - +- CometProject (127) - +- CometBroadcastHashJoin (126) - :- CometProject (124) - : +- CometBroadcastHashJoin (123) - : :- CometUnion (121) - : : :- CometProject (113) - : : : +- CometFilter (112) - : : : +- CometScan parquet spark_catalog.default.web_sales (111) - : : +- CometProject (120) - : : +- CometBroadcastHashJoin (119) - : : :- CometBroadcastExchange (115) - : : : +- CometScan parquet spark_catalog.default.web_returns (114) - : : +- CometProject (118) - : : +- CometFilter (117) - : : +- CometScan parquet spark_catalog.default.web_sales (116) - : +- ReusedExchange (122) - +- ReusedExchange (125) + : : : +- ReusedExchange (30) + : : +- CometBroadcastExchange (35) + : : +- CometFilter (34) + : : +- CometScan parquet spark_catalog.default.catalog_page (33) + : +- * HashAggregate (64) + : +- Exchange (63) + : +- * ColumnarToRow (62) + : +- CometHashAggregate (61) + : +- CometProject (60) + : +- CometBroadcastHashJoin (59) + : :- CometProject (55) + : : +- CometBroadcastHashJoin (54) + : : :- CometUnion (52) + : : : :- CometProject (44) + : : : : +- CometFilter (43) + : : : : +- CometScan parquet spark_catalog.default.web_sales (42) + : : : +- CometProject (51) + : : : +- CometBroadcastHashJoin (50) + : : : :- CometBroadcastExchange (46) + : : : : +- CometScan parquet spark_catalog.default.web_returns (45) + : : : +- CometProject (49) + : : : +- CometFilter (48) + : : : +- CometScan parquet spark_catalog.default.web_sales (47) + : : +- ReusedExchange (53) + : +- CometBroadcastExchange (58) + : +- CometFilter (57) + : +- CometScan parquet spark_catalog.default.web_site (56) + :- * HashAggregate (73) + : +- Exchange (72) + : +- * HashAggregate (71) + : +- * HashAggregate (70) + : +- ReusedExchange (69) + +- * HashAggregate (78) + +- Exchange (77) + +- * HashAggregate (76) + +- * HashAggregate (75) + +- ReusedExchange (74) (1) Scan parquet spark_catalog.default.store_sales @@ -287,590 +227,302 @@ Arguments: [page_sk#54, date_sk#55, sales_price#56, profit#57, return_amt#58, ne Child 0 Input [6]: [page_sk#44, date_sk#45, sales_price#46, profit#47, return_amt#48, net_loss#49] Child 1 Input [6]: [page_sk#54, date_sk#55, sales_price#56, profit#57, return_amt#58, net_loss#59] -(30) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#60, d_date#61] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [2]: [d_date_sk#60, d_date#61] -Condition : (((isnotnull(d_date#61) AND (d_date#61 >= 1998-08-04)) AND (d_date#61 <= 1998-08-18)) AND isnotnull(d_date_sk#60)) - -(32) CometProject -Input [2]: [d_date_sk#60, d_date#61] -Arguments: [d_date_sk#60], [d_date_sk#60] - -(33) CometBroadcastExchange -Input [1]: [d_date_sk#60] -Arguments: [d_date_sk#60] +(30) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#60] -(34) CometBroadcastHashJoin +(31) CometBroadcastHashJoin Left output [6]: [page_sk#44, date_sk#45, sales_price#46, profit#47, return_amt#48, net_loss#49] Right output [1]: [d_date_sk#60] Arguments: [date_sk#45], [d_date_sk#60], Inner -(35) CometProject +(32) CometProject Input [7]: [page_sk#44, date_sk#45, sales_price#46, profit#47, return_amt#48, net_loss#49, d_date_sk#60] Arguments: [page_sk#44, sales_price#46, profit#47, return_amt#48, net_loss#49], [page_sk#44, sales_price#46, profit#47, return_amt#48, net_loss#49] -(36) Scan parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] +(33) Scan parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(37) CometFilter -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] -Condition : isnotnull(cp_catalog_page_sk#62) +(34) CometFilter +Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Condition : isnotnull(cp_catalog_page_sk#61) -(38) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] -Arguments: [cp_catalog_page_sk#62, cp_catalog_page_id#63] +(35) CometBroadcastExchange +Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Arguments: [cp_catalog_page_sk#61, cp_catalog_page_id#62] -(39) CometBroadcastHashJoin +(36) CometBroadcastHashJoin Left output [5]: [page_sk#44, sales_price#46, profit#47, return_amt#48, net_loss#49] -Right output [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] -Arguments: [page_sk#44], [cp_catalog_page_sk#62], Inner +Right output [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Arguments: [page_sk#44], [cp_catalog_page_sk#61], Inner -(40) CometProject -Input [7]: [page_sk#44, sales_price#46, profit#47, return_amt#48, net_loss#49, cp_catalog_page_sk#62, cp_catalog_page_id#63] -Arguments: [sales_price#46, profit#47, return_amt#48, net_loss#49, cp_catalog_page_id#63], [sales_price#46, profit#47, return_amt#48, net_loss#49, cp_catalog_page_id#63] +(37) CometProject +Input [7]: [page_sk#44, sales_price#46, profit#47, return_amt#48, net_loss#49, cp_catalog_page_sk#61, cp_catalog_page_id#62] +Arguments: [sales_price#46, profit#47, return_amt#48, net_loss#49, cp_catalog_page_id#62], [sales_price#46, profit#47, return_amt#48, net_loss#49, cp_catalog_page_id#62] -(41) CometHashAggregate -Input [5]: [sales_price#46, profit#47, return_amt#48, net_loss#49, cp_catalog_page_id#63] -Keys [1]: [cp_catalog_page_id#63] +(38) CometHashAggregate +Input [5]: [sales_price#46, profit#47, return_amt#48, net_loss#49, cp_catalog_page_id#62] +Keys [1]: [cp_catalog_page_id#62] Functions [4]: [partial_sum(UnscaledValue(sales_price#46)), partial_sum(UnscaledValue(return_amt#48)), partial_sum(UnscaledValue(profit#47)), partial_sum(UnscaledValue(net_loss#49))] -(42) ColumnarToRow [codegen id : 3] -Input [5]: [cp_catalog_page_id#63, sum#64, sum#65, sum#66, sum#67] +(39) ColumnarToRow [codegen id : 3] +Input [5]: [cp_catalog_page_id#62, sum#63, sum#64, sum#65, sum#66] -(43) Exchange -Input [5]: [cp_catalog_page_id#63, sum#64, sum#65, sum#66, sum#67] -Arguments: hashpartitioning(cp_catalog_page_id#63, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(40) Exchange +Input [5]: [cp_catalog_page_id#62, sum#63, sum#64, sum#65, sum#66] +Arguments: hashpartitioning(cp_catalog_page_id#62, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(44) HashAggregate [codegen id : 4] -Input [5]: [cp_catalog_page_id#63, sum#64, sum#65, sum#66, sum#67] -Keys [1]: [cp_catalog_page_id#63] +(41) HashAggregate [codegen id : 4] +Input [5]: [cp_catalog_page_id#62, sum#63, sum#64, sum#65, sum#66] +Keys [1]: [cp_catalog_page_id#62] Functions [4]: [sum(UnscaledValue(sales_price#46)), sum(UnscaledValue(return_amt#48)), sum(UnscaledValue(profit#47)), sum(UnscaledValue(net_loss#49))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#46))#68, sum(UnscaledValue(return_amt#48))#69, sum(UnscaledValue(profit#47))#70, sum(UnscaledValue(net_loss#49))#71] -Results [5]: [catalog channel AS channel#72, concat(catalog_page, cp_catalog_page_id#63) AS id#73, MakeDecimal(sum(UnscaledValue(sales_price#46))#68,17,2) AS sales#74, MakeDecimal(sum(UnscaledValue(return_amt#48))#69,17,2) AS returns#75, (MakeDecimal(sum(UnscaledValue(profit#47))#70,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#49))#71,17,2)) AS profit#76] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#46))#67, sum(UnscaledValue(return_amt#48))#68, sum(UnscaledValue(profit#47))#69, sum(UnscaledValue(net_loss#49))#70] +Results [5]: [catalog channel AS channel#71, concat(catalog_page, cp_catalog_page_id#62) AS id#72, MakeDecimal(sum(UnscaledValue(sales_price#46))#67,17,2) AS sales#73, MakeDecimal(sum(UnscaledValue(return_amt#48))#68,17,2) AS returns#74, (MakeDecimal(sum(UnscaledValue(profit#47))#69,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#49))#70,17,2)) AS profit#75] -(45) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#77, ws_ext_sales_price#78, ws_net_profit#79, ws_sold_date_sk#80] +(42) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_web_site_sk#76, ws_ext_sales_price#77, ws_net_profit#78, ws_sold_date_sk#79] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#80), dynamicpruningexpression(ws_sold_date_sk#80 IN dynamicpruning#81)] +PartitionFilters: [isnotnull(ws_sold_date_sk#79), dynamicpruningexpression(ws_sold_date_sk#79 IN dynamicpruning#80)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(46) CometFilter -Input [4]: [ws_web_site_sk#77, ws_ext_sales_price#78, ws_net_profit#79, ws_sold_date_sk#80] -Condition : isnotnull(ws_web_site_sk#77) +(43) CometFilter +Input [4]: [ws_web_site_sk#76, ws_ext_sales_price#77, ws_net_profit#78, ws_sold_date_sk#79] +Condition : isnotnull(ws_web_site_sk#76) -(47) CometProject -Input [4]: [ws_web_site_sk#77, ws_ext_sales_price#78, ws_net_profit#79, ws_sold_date_sk#80] -Arguments: [wsr_web_site_sk#82, date_sk#83, sales_price#84, profit#85, return_amt#86, net_loss#87], [ws_web_site_sk#77 AS wsr_web_site_sk#82, ws_sold_date_sk#80 AS date_sk#83, ws_ext_sales_price#78 AS sales_price#84, ws_net_profit#79 AS profit#85, 0.00 AS return_amt#86, 0.00 AS net_loss#87] +(44) CometProject +Input [4]: [ws_web_site_sk#76, ws_ext_sales_price#77, ws_net_profit#78, ws_sold_date_sk#79] +Arguments: [wsr_web_site_sk#81, date_sk#82, sales_price#83, profit#84, return_amt#85, net_loss#86], [ws_web_site_sk#76 AS wsr_web_site_sk#81, ws_sold_date_sk#79 AS date_sk#82, ws_ext_sales_price#77 AS sales_price#83, ws_net_profit#78 AS profit#84, 0.00 AS return_amt#85, 0.00 AS net_loss#86] -(48) Scan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#88, wr_order_number#89, wr_return_amt#90, wr_net_loss#91, wr_returned_date_sk#92] +(45) Scan parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#87, wr_order_number#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#92), dynamicpruningexpression(wr_returned_date_sk#92 IN dynamicpruning#81)] +PartitionFilters: [isnotnull(wr_returned_date_sk#91), dynamicpruningexpression(wr_returned_date_sk#91 IN dynamicpruning#80)] ReadSchema: struct -(49) CometBroadcastExchange -Input [5]: [wr_item_sk#88, wr_order_number#89, wr_return_amt#90, wr_net_loss#91, wr_returned_date_sk#92] -Arguments: [wr_item_sk#88, wr_order_number#89, wr_return_amt#90, wr_net_loss#91, wr_returned_date_sk#92] +(46) CometBroadcastExchange +Input [5]: [wr_item_sk#87, wr_order_number#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Arguments: [wr_item_sk#87, wr_order_number#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] -(50) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#93, ws_web_site_sk#94, ws_order_number#95, ws_sold_date_sk#96] +(47) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#92, ws_web_site_sk#93, ws_order_number#94, ws_sold_date_sk#95] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(51) CometFilter -Input [4]: [ws_item_sk#93, ws_web_site_sk#94, ws_order_number#95, ws_sold_date_sk#96] -Condition : ((isnotnull(ws_item_sk#93) AND isnotnull(ws_order_number#95)) AND isnotnull(ws_web_site_sk#94)) - -(52) CometProject -Input [4]: [ws_item_sk#93, ws_web_site_sk#94, ws_order_number#95, ws_sold_date_sk#96] -Arguments: [ws_item_sk#93, ws_web_site_sk#94, ws_order_number#95], [ws_item_sk#93, ws_web_site_sk#94, ws_order_number#95] - -(53) CometBroadcastHashJoin -Left output [5]: [wr_item_sk#88, wr_order_number#89, wr_return_amt#90, wr_net_loss#91, wr_returned_date_sk#92] -Right output [3]: [ws_item_sk#93, ws_web_site_sk#94, ws_order_number#95] -Arguments: [wr_item_sk#88, wr_order_number#89], [ws_item_sk#93, ws_order_number#95], Inner - -(54) CometProject -Input [8]: [wr_item_sk#88, wr_order_number#89, wr_return_amt#90, wr_net_loss#91, wr_returned_date_sk#92, ws_item_sk#93, ws_web_site_sk#94, ws_order_number#95] -Arguments: [wsr_web_site_sk#97, date_sk#98, sales_price#99, profit#100, return_amt#101, net_loss#102], [ws_web_site_sk#94 AS wsr_web_site_sk#97, wr_returned_date_sk#92 AS date_sk#98, 0.00 AS sales_price#99, 0.00 AS profit#100, wr_return_amt#90 AS return_amt#101, wr_net_loss#91 AS net_loss#102] +(48) CometFilter +Input [4]: [ws_item_sk#92, ws_web_site_sk#93, ws_order_number#94, ws_sold_date_sk#95] +Condition : ((isnotnull(ws_item_sk#92) AND isnotnull(ws_order_number#94)) AND isnotnull(ws_web_site_sk#93)) -(55) CometUnion -Child 0 Input [6]: [wsr_web_site_sk#82, date_sk#83, sales_price#84, profit#85, return_amt#86, net_loss#87] -Child 1 Input [6]: [wsr_web_site_sk#97, date_sk#98, sales_price#99, profit#100, return_amt#101, net_loss#102] +(49) CometProject +Input [4]: [ws_item_sk#92, ws_web_site_sk#93, ws_order_number#94, ws_sold_date_sk#95] +Arguments: [ws_item_sk#92, ws_web_site_sk#93, ws_order_number#94], [ws_item_sk#92, ws_web_site_sk#93, ws_order_number#94] -(56) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#103, d_date#104] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] -ReadSchema: struct +(50) CometBroadcastHashJoin +Left output [5]: [wr_item_sk#87, wr_order_number#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Right output [3]: [ws_item_sk#92, ws_web_site_sk#93, ws_order_number#94] +Arguments: [wr_item_sk#87, wr_order_number#88], [ws_item_sk#92, ws_order_number#94], Inner -(57) CometFilter -Input [2]: [d_date_sk#103, d_date#104] -Condition : (((isnotnull(d_date#104) AND (d_date#104 >= 1998-08-04)) AND (d_date#104 <= 1998-08-18)) AND isnotnull(d_date_sk#103)) +(51) CometProject +Input [8]: [wr_item_sk#87, wr_order_number#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, ws_item_sk#92, ws_web_site_sk#93, ws_order_number#94] +Arguments: [wsr_web_site_sk#96, date_sk#97, sales_price#98, profit#99, return_amt#100, net_loss#101], [ws_web_site_sk#93 AS wsr_web_site_sk#96, wr_returned_date_sk#91 AS date_sk#97, 0.00 AS sales_price#98, 0.00 AS profit#99, wr_return_amt#89 AS return_amt#100, wr_net_loss#90 AS net_loss#101] -(58) CometProject -Input [2]: [d_date_sk#103, d_date#104] -Arguments: [d_date_sk#103], [d_date_sk#103] +(52) CometUnion +Child 0 Input [6]: [wsr_web_site_sk#81, date_sk#82, sales_price#83, profit#84, return_amt#85, net_loss#86] +Child 1 Input [6]: [wsr_web_site_sk#96, date_sk#97, sales_price#98, profit#99, return_amt#100, net_loss#101] -(59) CometBroadcastExchange -Input [1]: [d_date_sk#103] -Arguments: [d_date_sk#103] +(53) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#102] -(60) CometBroadcastHashJoin -Left output [6]: [wsr_web_site_sk#82, date_sk#83, sales_price#84, profit#85, return_amt#86, net_loss#87] -Right output [1]: [d_date_sk#103] -Arguments: [date_sk#83], [d_date_sk#103], Inner +(54) CometBroadcastHashJoin +Left output [6]: [wsr_web_site_sk#81, date_sk#82, sales_price#83, profit#84, return_amt#85, net_loss#86] +Right output [1]: [d_date_sk#102] +Arguments: [date_sk#82], [d_date_sk#102], Inner -(61) CometProject -Input [7]: [wsr_web_site_sk#82, date_sk#83, sales_price#84, profit#85, return_amt#86, net_loss#87, d_date_sk#103] -Arguments: [wsr_web_site_sk#82, sales_price#84, profit#85, return_amt#86, net_loss#87], [wsr_web_site_sk#82, sales_price#84, profit#85, return_amt#86, net_loss#87] +(55) CometProject +Input [7]: [wsr_web_site_sk#81, date_sk#82, sales_price#83, profit#84, return_amt#85, net_loss#86, d_date_sk#102] +Arguments: [wsr_web_site_sk#81, sales_price#83, profit#84, return_amt#85, net_loss#86], [wsr_web_site_sk#81, sales_price#83, profit#84, return_amt#85, net_loss#86] -(62) Scan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#105, web_site_id#106] +(56) Scan parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#103, web_site_id#104] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(63) CometFilter -Input [2]: [web_site_sk#105, web_site_id#106] -Condition : isnotnull(web_site_sk#105) +(57) CometFilter +Input [2]: [web_site_sk#103, web_site_id#104] +Condition : isnotnull(web_site_sk#103) -(64) CometBroadcastExchange -Input [2]: [web_site_sk#105, web_site_id#106] -Arguments: [web_site_sk#105, web_site_id#106] +(58) CometBroadcastExchange +Input [2]: [web_site_sk#103, web_site_id#104] +Arguments: [web_site_sk#103, web_site_id#104] -(65) CometBroadcastHashJoin -Left output [5]: [wsr_web_site_sk#82, sales_price#84, profit#85, return_amt#86, net_loss#87] -Right output [2]: [web_site_sk#105, web_site_id#106] -Arguments: [wsr_web_site_sk#82], [web_site_sk#105], Inner +(59) CometBroadcastHashJoin +Left output [5]: [wsr_web_site_sk#81, sales_price#83, profit#84, return_amt#85, net_loss#86] +Right output [2]: [web_site_sk#103, web_site_id#104] +Arguments: [wsr_web_site_sk#81], [web_site_sk#103], Inner -(66) CometProject -Input [7]: [wsr_web_site_sk#82, sales_price#84, profit#85, return_amt#86, net_loss#87, web_site_sk#105, web_site_id#106] -Arguments: [sales_price#84, profit#85, return_amt#86, net_loss#87, web_site_id#106], [sales_price#84, profit#85, return_amt#86, net_loss#87, web_site_id#106] +(60) CometProject +Input [7]: [wsr_web_site_sk#81, sales_price#83, profit#84, return_amt#85, net_loss#86, web_site_sk#103, web_site_id#104] +Arguments: [sales_price#83, profit#84, return_amt#85, net_loss#86, web_site_id#104], [sales_price#83, profit#84, return_amt#85, net_loss#86, web_site_id#104] -(67) CometHashAggregate -Input [5]: [sales_price#84, profit#85, return_amt#86, net_loss#87, web_site_id#106] -Keys [1]: [web_site_id#106] -Functions [4]: [partial_sum(UnscaledValue(sales_price#84)), partial_sum(UnscaledValue(return_amt#86)), partial_sum(UnscaledValue(profit#85)), partial_sum(UnscaledValue(net_loss#87))] +(61) CometHashAggregate +Input [5]: [sales_price#83, profit#84, return_amt#85, net_loss#86, web_site_id#104] +Keys [1]: [web_site_id#104] +Functions [4]: [partial_sum(UnscaledValue(sales_price#83)), partial_sum(UnscaledValue(return_amt#85)), partial_sum(UnscaledValue(profit#84)), partial_sum(UnscaledValue(net_loss#86))] -(68) ColumnarToRow [codegen id : 5] -Input [5]: [web_site_id#106, sum#107, sum#108, sum#109, sum#110] +(62) ColumnarToRow [codegen id : 5] +Input [5]: [web_site_id#104, sum#105, sum#106, sum#107, sum#108] -(69) Exchange -Input [5]: [web_site_id#106, sum#107, sum#108, sum#109, sum#110] -Arguments: hashpartitioning(web_site_id#106, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(63) Exchange +Input [5]: [web_site_id#104, sum#105, sum#106, sum#107, sum#108] +Arguments: hashpartitioning(web_site_id#104, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(70) HashAggregate [codegen id : 6] -Input [5]: [web_site_id#106, sum#107, sum#108, sum#109, sum#110] -Keys [1]: [web_site_id#106] -Functions [4]: [sum(UnscaledValue(sales_price#84)), sum(UnscaledValue(return_amt#86)), sum(UnscaledValue(profit#85)), sum(UnscaledValue(net_loss#87))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#84))#111, sum(UnscaledValue(return_amt#86))#112, sum(UnscaledValue(profit#85))#113, sum(UnscaledValue(net_loss#87))#114] -Results [5]: [web channel AS channel#115, concat(web_site, web_site_id#106) AS id#116, MakeDecimal(sum(UnscaledValue(sales_price#84))#111,17,2) AS sales#117, MakeDecimal(sum(UnscaledValue(return_amt#86))#112,17,2) AS returns#118, (MakeDecimal(sum(UnscaledValue(profit#85))#113,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#87))#114,17,2)) AS profit#119] +(64) HashAggregate [codegen id : 6] +Input [5]: [web_site_id#104, sum#105, sum#106, sum#107, sum#108] +Keys [1]: [web_site_id#104] +Functions [4]: [sum(UnscaledValue(sales_price#83)), sum(UnscaledValue(return_amt#85)), sum(UnscaledValue(profit#84)), sum(UnscaledValue(net_loss#86))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#83))#109, sum(UnscaledValue(return_amt#85))#110, sum(UnscaledValue(profit#84))#111, sum(UnscaledValue(net_loss#86))#112] +Results [5]: [web channel AS channel#113, concat(web_site, web_site_id#104) AS id#114, MakeDecimal(sum(UnscaledValue(sales_price#83))#109,17,2) AS sales#115, MakeDecimal(sum(UnscaledValue(return_amt#85))#110,17,2) AS returns#116, (MakeDecimal(sum(UnscaledValue(profit#84))#111,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#86))#112,17,2)) AS profit#117] -(71) Union +(65) Union -(72) HashAggregate [codegen id : 7] +(66) HashAggregate [codegen id : 7] 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#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] -Results [8]: [channel#34, id#35, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] +Aggregate Attributes [6]: [sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123] +Results [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] -(73) Exchange -Input [8]: [channel#34, id#35, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] +(67) Exchange +Input [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] Arguments: hashpartitioning(channel#34, id#35, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(74) HashAggregate [codegen id : 8] -Input [8]: [channel#34, id#35, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] +(68) HashAggregate [codegen id : 8] +Input [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] Keys [2]: [channel#34, id#35] Functions [3]: [sum(sales#36), sum(returns#37), sum(profit#38)] -Aggregate Attributes [3]: [sum(sales#36)#132, sum(returns#37)#133, sum(profit#38)#134] -Results [5]: [channel#34, id#35, cast(sum(sales#36)#132 as decimal(37,2)) AS sales#135, cast(sum(returns#37)#133 as decimal(37,2)) AS returns#136, cast(sum(profit#38)#134 as decimal(38,2)) AS profit#137] - -(75) ReusedExchange [Reuses operator id: 21] -Output [5]: [s_store_id#25, sum#138, sum#139, sum#140, sum#141] - -(76) HashAggregate [codegen id : 10] -Input [5]: [s_store_id#25, sum#138, sum#139, sum#140, sum#141] -Keys [1]: [s_store_id#25] -Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#30, sum(UnscaledValue(return_amt#10))#31, sum(UnscaledValue(profit#9))#32, sum(UnscaledValue(net_loss#11))#33] -Results [5]: [store channel AS channel#34, concat(store, s_store_id#25) AS id#35, MakeDecimal(sum(UnscaledValue(sales_price#8))#30,17,2) AS sales#36, MakeDecimal(sum(UnscaledValue(return_amt#10))#31,17,2) AS returns#37, (MakeDecimal(sum(UnscaledValue(profit#9))#32,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#33,17,2)) AS profit#38] - -(77) ReusedExchange [Reuses operator id: 43] -Output [5]: [cp_catalog_page_id#63, sum#142, sum#143, sum#144, sum#145] - -(78) HashAggregate [codegen id : 12] -Input [5]: [cp_catalog_page_id#63, sum#142, sum#143, sum#144, sum#145] -Keys [1]: [cp_catalog_page_id#63] -Functions [4]: [sum(UnscaledValue(sales_price#46)), sum(UnscaledValue(return_amt#48)), sum(UnscaledValue(profit#47)), sum(UnscaledValue(net_loss#49))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#46))#68, sum(UnscaledValue(return_amt#48))#69, sum(UnscaledValue(profit#47))#70, sum(UnscaledValue(net_loss#49))#71] -Results [5]: [catalog channel AS channel#72, concat(catalog_page, cp_catalog_page_id#63) AS id#73, MakeDecimal(sum(UnscaledValue(sales_price#46))#68,17,2) AS sales#74, MakeDecimal(sum(UnscaledValue(return_amt#48))#69,17,2) AS returns#75, (MakeDecimal(sum(UnscaledValue(profit#47))#70,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#49))#71,17,2)) AS profit#76] - -(79) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#77, ws_ext_sales_price#78, ws_net_profit#79, ws_sold_date_sk#80] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#80), dynamicpruningexpression(ws_sold_date_sk#80 IN dynamicpruning#146)] -PushedFilters: [IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(80) CometFilter -Input [4]: [ws_web_site_sk#77, ws_ext_sales_price#78, ws_net_profit#79, ws_sold_date_sk#80] -Condition : isnotnull(ws_web_site_sk#77) - -(81) CometProject -Input [4]: [ws_web_site_sk#77, ws_ext_sales_price#78, ws_net_profit#79, ws_sold_date_sk#80] -Arguments: [wsr_web_site_sk#82, date_sk#83, sales_price#84, profit#85, return_amt#86, net_loss#87], [ws_web_site_sk#77 AS wsr_web_site_sk#82, ws_sold_date_sk#80 AS date_sk#83, ws_ext_sales_price#78 AS sales_price#84, ws_net_profit#79 AS profit#85, 0.00 AS return_amt#86, 0.00 AS net_loss#87] - -(82) Scan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#88, wr_order_number#89, wr_return_amt#90, wr_net_loss#91, wr_returned_date_sk#92] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#92), dynamicpruningexpression(wr_returned_date_sk#92 IN dynamicpruning#146)] -ReadSchema: struct - -(83) CometBroadcastExchange -Input [5]: [wr_item_sk#88, wr_order_number#89, wr_return_amt#90, wr_net_loss#91, wr_returned_date_sk#92] -Arguments: [wr_item_sk#88, wr_order_number#89, wr_return_amt#90, wr_net_loss#91, wr_returned_date_sk#92] - -(84) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#93, ws_web_site_sk#94, ws_order_number#95, ws_sold_date_sk#96] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(85) CometFilter -Input [4]: [ws_item_sk#93, ws_web_site_sk#94, ws_order_number#95, ws_sold_date_sk#96] -Condition : ((isnotnull(ws_item_sk#93) AND isnotnull(ws_order_number#95)) AND isnotnull(ws_web_site_sk#94)) - -(86) CometProject -Input [4]: [ws_item_sk#93, ws_web_site_sk#94, ws_order_number#95, ws_sold_date_sk#96] -Arguments: [ws_item_sk#93, ws_web_site_sk#94, ws_order_number#95], [ws_item_sk#93, ws_web_site_sk#94, ws_order_number#95] - -(87) CometBroadcastHashJoin -Left output [5]: [wr_item_sk#88, wr_order_number#89, wr_return_amt#90, wr_net_loss#91, wr_returned_date_sk#92] -Right output [3]: [ws_item_sk#93, ws_web_site_sk#94, ws_order_number#95] -Arguments: [wr_item_sk#88, wr_order_number#89], [ws_item_sk#93, ws_order_number#95], Inner - -(88) CometProject -Input [8]: [wr_item_sk#88, wr_order_number#89, wr_return_amt#90, wr_net_loss#91, wr_returned_date_sk#92, ws_item_sk#93, ws_web_site_sk#94, ws_order_number#95] -Arguments: [wsr_web_site_sk#97, date_sk#98, sales_price#99, profit#100, return_amt#101, net_loss#102], [ws_web_site_sk#94 AS wsr_web_site_sk#97, wr_returned_date_sk#92 AS date_sk#98, 0.00 AS sales_price#99, 0.00 AS profit#100, wr_return_amt#90 AS return_amt#101, wr_net_loss#91 AS net_loss#102] - -(89) CometUnion -Child 0 Input [6]: [wsr_web_site_sk#82, date_sk#83, sales_price#84, profit#85, return_amt#86, net_loss#87] -Child 1 Input [6]: [wsr_web_site_sk#97, date_sk#98, sales_price#99, profit#100, return_amt#101, net_loss#102] - -(90) ReusedExchange [Reuses operator id: 59] -Output [1]: [d_date_sk#103] - -(91) CometBroadcastHashJoin -Left output [6]: [wsr_web_site_sk#82, date_sk#83, sales_price#84, profit#85, return_amt#86, net_loss#87] -Right output [1]: [d_date_sk#103] -Arguments: [date_sk#83], [d_date_sk#103], Inner - -(92) CometProject -Input [7]: [wsr_web_site_sk#82, date_sk#83, sales_price#84, profit#85, return_amt#86, net_loss#87, d_date_sk#103] -Arguments: [wsr_web_site_sk#82, sales_price#84, profit#85, return_amt#86, net_loss#87], [wsr_web_site_sk#82, sales_price#84, profit#85, return_amt#86, net_loss#87] - -(93) ReusedExchange [Reuses operator id: 64] -Output [2]: [web_site_sk#105, web_site_id#106] - -(94) CometBroadcastHashJoin -Left output [5]: [wsr_web_site_sk#82, sales_price#84, profit#85, return_amt#86, net_loss#87] -Right output [2]: [web_site_sk#105, web_site_id#106] -Arguments: [wsr_web_site_sk#82], [web_site_sk#105], Inner - -(95) CometProject -Input [7]: [wsr_web_site_sk#82, sales_price#84, profit#85, return_amt#86, net_loss#87, web_site_sk#105, web_site_id#106] -Arguments: [sales_price#84, profit#85, return_amt#86, net_loss#87, web_site_id#106], [sales_price#84, profit#85, return_amt#86, net_loss#87, web_site_id#106] - -(96) CometHashAggregate -Input [5]: [sales_price#84, profit#85, return_amt#86, net_loss#87, web_site_id#106] -Keys [1]: [web_site_id#106] -Functions [4]: [partial_sum(UnscaledValue(sales_price#84)), partial_sum(UnscaledValue(return_amt#86)), partial_sum(UnscaledValue(profit#85)), partial_sum(UnscaledValue(net_loss#87))] - -(97) ColumnarToRow [codegen id : 13] -Input [5]: [web_site_id#106, sum#147, sum#148, sum#149, sum#150] +Aggregate Attributes [3]: [sum(sales#36)#130, sum(returns#37)#131, sum(profit#38)#132] +Results [5]: [channel#34, id#35, cast(sum(sales#36)#130 as decimal(37,2)) AS sales#133, cast(sum(returns#37)#131 as decimal(37,2)) AS returns#134, cast(sum(profit#38)#132 as decimal(38,2)) AS profit#135] -(98) Exchange -Input [5]: [web_site_id#106, sum#147, sum#148, sum#149, sum#150] -Arguments: hashpartitioning(web_site_id#106, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(69) ReusedExchange [Reuses operator id: 67] +Output [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] -(99) HashAggregate [codegen id : 14] -Input [5]: [web_site_id#106, sum#147, sum#148, sum#149, sum#150] -Keys [1]: [web_site_id#106] -Functions [4]: [sum(UnscaledValue(sales_price#84)), sum(UnscaledValue(return_amt#86)), sum(UnscaledValue(profit#85)), sum(UnscaledValue(net_loss#87))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#84))#111, sum(UnscaledValue(return_amt#86))#112, sum(UnscaledValue(profit#85))#113, sum(UnscaledValue(net_loss#87))#114] -Results [5]: [web channel AS channel#115, concat(web_site, web_site_id#106) AS id#116, MakeDecimal(sum(UnscaledValue(sales_price#84))#111,17,2) AS sales#117, MakeDecimal(sum(UnscaledValue(return_amt#86))#112,17,2) AS returns#118, (MakeDecimal(sum(UnscaledValue(profit#85))#113,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#87))#114,17,2)) AS profit#119] - -(100) Union - -(101) HashAggregate [codegen id : 15] -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#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] -Results [8]: [channel#34, id#35, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] - -(102) Exchange -Input [8]: [channel#34, id#35, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] -Arguments: hashpartitioning(channel#34, id#35, 5), ENSURE_REQUIREMENTS, [plan_id=6] - -(103) HashAggregate [codegen id : 16] -Input [8]: [channel#34, id#35, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] +(70) HashAggregate [codegen id : 16] +Input [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] Keys [2]: [channel#34, id#35] Functions [3]: [sum(sales#36), sum(returns#37), sum(profit#38)] -Aggregate Attributes [3]: [sum(sales#36)#132, sum(returns#37)#133, sum(profit#38)#134] -Results [4]: [channel#34, sum(sales#36)#132 AS sales#151, sum(returns#37)#133 AS returns#152, sum(profit#38)#134 AS profit#153] +Aggregate Attributes [3]: [sum(sales#36)#130, sum(returns#37)#131, sum(profit#38)#132] +Results [4]: [channel#34, sum(sales#36)#130 AS sales#136, sum(returns#37)#131 AS returns#137, sum(profit#38)#132 AS profit#138] -(104) HashAggregate [codegen id : 16] -Input [4]: [channel#34, sales#151, returns#152, profit#153] +(71) HashAggregate [codegen id : 16] +Input [4]: [channel#34, sales#136, returns#137, profit#138] Keys [1]: [channel#34] -Functions [3]: [partial_sum(sales#151), partial_sum(returns#152), partial_sum(profit#153)] -Aggregate Attributes [6]: [sum#154, isEmpty#155, sum#156, isEmpty#157, sum#158, isEmpty#159] -Results [7]: [channel#34, sum#160, isEmpty#161, sum#162, isEmpty#163, sum#164, isEmpty#165] +Functions [3]: [partial_sum(sales#136), partial_sum(returns#137), partial_sum(profit#138)] +Aggregate Attributes [6]: [sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] +Results [7]: [channel#34, sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150] -(105) Exchange -Input [7]: [channel#34, sum#160, isEmpty#161, sum#162, isEmpty#163, sum#164, isEmpty#165] -Arguments: hashpartitioning(channel#34, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(72) Exchange +Input [7]: [channel#34, sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150] +Arguments: hashpartitioning(channel#34, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(106) HashAggregate [codegen id : 17] -Input [7]: [channel#34, sum#160, isEmpty#161, sum#162, isEmpty#163, sum#164, isEmpty#165] +(73) HashAggregate [codegen id : 17] +Input [7]: [channel#34, sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150] Keys [1]: [channel#34] -Functions [3]: [sum(sales#151), sum(returns#152), sum(profit#153)] -Aggregate Attributes [3]: [sum(sales#151)#166, sum(returns#152)#167, sum(profit#153)#168] -Results [5]: [channel#34, null AS id#169, sum(sales#151)#166 AS sum(sales)#170, sum(returns#152)#167 AS sum(returns)#171, sum(profit#153)#168 AS sum(profit)#172] - -(107) ReusedExchange [Reuses operator id: 21] -Output [5]: [s_store_id#25, sum#173, sum#174, sum#175, sum#176] - -(108) HashAggregate [codegen id : 19] -Input [5]: [s_store_id#25, sum#173, sum#174, sum#175, sum#176] -Keys [1]: [s_store_id#25] -Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#30, sum(UnscaledValue(return_amt#10))#31, sum(UnscaledValue(profit#9))#32, sum(UnscaledValue(net_loss#11))#33] -Results [5]: [store channel AS channel#34, concat(store, s_store_id#25) AS id#35, MakeDecimal(sum(UnscaledValue(sales_price#8))#30,17,2) AS sales#36, MakeDecimal(sum(UnscaledValue(return_amt#10))#31,17,2) AS returns#37, (MakeDecimal(sum(UnscaledValue(profit#9))#32,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#33,17,2)) AS profit#38] - -(109) ReusedExchange [Reuses operator id: 43] -Output [5]: [cp_catalog_page_id#63, sum#177, sum#178, sum#179, sum#180] - -(110) HashAggregate [codegen id : 21] -Input [5]: [cp_catalog_page_id#63, sum#177, sum#178, sum#179, sum#180] -Keys [1]: [cp_catalog_page_id#63] -Functions [4]: [sum(UnscaledValue(sales_price#46)), sum(UnscaledValue(return_amt#48)), sum(UnscaledValue(profit#47)), sum(UnscaledValue(net_loss#49))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#46))#68, sum(UnscaledValue(return_amt#48))#69, sum(UnscaledValue(profit#47))#70, sum(UnscaledValue(net_loss#49))#71] -Results [5]: [catalog channel AS channel#72, concat(catalog_page, cp_catalog_page_id#63) AS id#73, MakeDecimal(sum(UnscaledValue(sales_price#46))#68,17,2) AS sales#74, MakeDecimal(sum(UnscaledValue(return_amt#48))#69,17,2) AS returns#75, (MakeDecimal(sum(UnscaledValue(profit#47))#70,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#49))#71,17,2)) AS profit#76] - -(111) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#77, ws_ext_sales_price#78, ws_net_profit#79, ws_sold_date_sk#80] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#80), dynamicpruningexpression(ws_sold_date_sk#80 IN dynamicpruning#181)] -PushedFilters: [IsNotNull(ws_web_site_sk)] -ReadSchema: struct +Functions [3]: [sum(sales#136), sum(returns#137), sum(profit#138)] +Aggregate Attributes [3]: [sum(sales#136)#151, sum(returns#137)#152, sum(profit#138)#153] +Results [5]: [channel#34, null AS id#154, sum(sales#136)#151 AS sum(sales)#155, sum(returns#137)#152 AS sum(returns)#156, sum(profit#138)#153 AS sum(profit)#157] -(112) CometFilter -Input [4]: [ws_web_site_sk#77, ws_ext_sales_price#78, ws_net_profit#79, ws_sold_date_sk#80] -Condition : isnotnull(ws_web_site_sk#77) +(74) ReusedExchange [Reuses operator id: 67] +Output [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] -(113) CometProject -Input [4]: [ws_web_site_sk#77, ws_ext_sales_price#78, ws_net_profit#79, ws_sold_date_sk#80] -Arguments: [wsr_web_site_sk#82, date_sk#83, sales_price#84, profit#85, return_amt#86, net_loss#87], [ws_web_site_sk#77 AS wsr_web_site_sk#82, ws_sold_date_sk#80 AS date_sk#83, ws_ext_sales_price#78 AS sales_price#84, ws_net_profit#79 AS profit#85, 0.00 AS return_amt#86, 0.00 AS net_loss#87] - -(114) Scan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#88, wr_order_number#89, wr_return_amt#90, wr_net_loss#91, wr_returned_date_sk#92] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#92), dynamicpruningexpression(wr_returned_date_sk#92 IN dynamicpruning#181)] -ReadSchema: struct - -(115) CometBroadcastExchange -Input [5]: [wr_item_sk#88, wr_order_number#89, wr_return_amt#90, wr_net_loss#91, wr_returned_date_sk#92] -Arguments: [wr_item_sk#88, wr_order_number#89, wr_return_amt#90, wr_net_loss#91, wr_returned_date_sk#92] - -(116) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#93, ws_web_site_sk#94, ws_order_number#95, ws_sold_date_sk#96] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(117) CometFilter -Input [4]: [ws_item_sk#93, ws_web_site_sk#94, ws_order_number#95, ws_sold_date_sk#96] -Condition : ((isnotnull(ws_item_sk#93) AND isnotnull(ws_order_number#95)) AND isnotnull(ws_web_site_sk#94)) - -(118) CometProject -Input [4]: [ws_item_sk#93, ws_web_site_sk#94, ws_order_number#95, ws_sold_date_sk#96] -Arguments: [ws_item_sk#93, ws_web_site_sk#94, ws_order_number#95], [ws_item_sk#93, ws_web_site_sk#94, ws_order_number#95] - -(119) CometBroadcastHashJoin -Left output [5]: [wr_item_sk#88, wr_order_number#89, wr_return_amt#90, wr_net_loss#91, wr_returned_date_sk#92] -Right output [3]: [ws_item_sk#93, ws_web_site_sk#94, ws_order_number#95] -Arguments: [wr_item_sk#88, wr_order_number#89], [ws_item_sk#93, ws_order_number#95], Inner - -(120) CometProject -Input [8]: [wr_item_sk#88, wr_order_number#89, wr_return_amt#90, wr_net_loss#91, wr_returned_date_sk#92, ws_item_sk#93, ws_web_site_sk#94, ws_order_number#95] -Arguments: [wsr_web_site_sk#97, date_sk#98, sales_price#99, profit#100, return_amt#101, net_loss#102], [ws_web_site_sk#94 AS wsr_web_site_sk#97, wr_returned_date_sk#92 AS date_sk#98, 0.00 AS sales_price#99, 0.00 AS profit#100, wr_return_amt#90 AS return_amt#101, wr_net_loss#91 AS net_loss#102] - -(121) CometUnion -Child 0 Input [6]: [wsr_web_site_sk#82, date_sk#83, sales_price#84, profit#85, return_amt#86, net_loss#87] -Child 1 Input [6]: [wsr_web_site_sk#97, date_sk#98, sales_price#99, profit#100, return_amt#101, net_loss#102] - -(122) ReusedExchange [Reuses operator id: 59] -Output [1]: [d_date_sk#103] - -(123) CometBroadcastHashJoin -Left output [6]: [wsr_web_site_sk#82, date_sk#83, sales_price#84, profit#85, return_amt#86, net_loss#87] -Right output [1]: [d_date_sk#103] -Arguments: [date_sk#83], [d_date_sk#103], Inner - -(124) CometProject -Input [7]: [wsr_web_site_sk#82, date_sk#83, sales_price#84, profit#85, return_amt#86, net_loss#87, d_date_sk#103] -Arguments: [wsr_web_site_sk#82, sales_price#84, profit#85, return_amt#86, net_loss#87], [wsr_web_site_sk#82, sales_price#84, profit#85, return_amt#86, net_loss#87] - -(125) ReusedExchange [Reuses operator id: 64] -Output [2]: [web_site_sk#105, web_site_id#106] - -(126) CometBroadcastHashJoin -Left output [5]: [wsr_web_site_sk#82, sales_price#84, profit#85, return_amt#86, net_loss#87] -Right output [2]: [web_site_sk#105, web_site_id#106] -Arguments: [wsr_web_site_sk#82], [web_site_sk#105], Inner - -(127) CometProject -Input [7]: [wsr_web_site_sk#82, sales_price#84, profit#85, return_amt#86, net_loss#87, web_site_sk#105, web_site_id#106] -Arguments: [sales_price#84, profit#85, return_amt#86, net_loss#87, web_site_id#106], [sales_price#84, profit#85, return_amt#86, net_loss#87, web_site_id#106] - -(128) CometHashAggregate -Input [5]: [sales_price#84, profit#85, return_amt#86, net_loss#87, web_site_id#106] -Keys [1]: [web_site_id#106] -Functions [4]: [partial_sum(UnscaledValue(sales_price#84)), partial_sum(UnscaledValue(return_amt#86)), partial_sum(UnscaledValue(profit#85)), partial_sum(UnscaledValue(net_loss#87))] - -(129) ColumnarToRow [codegen id : 22] -Input [5]: [web_site_id#106, sum#182, sum#183, sum#184, sum#185] - -(130) Exchange -Input [5]: [web_site_id#106, sum#182, sum#183, sum#184, sum#185] -Arguments: hashpartitioning(web_site_id#106, 5), ENSURE_REQUIREMENTS, [plan_id=8] - -(131) HashAggregate [codegen id : 23] -Input [5]: [web_site_id#106, sum#182, sum#183, sum#184, sum#185] -Keys [1]: [web_site_id#106] -Functions [4]: [sum(UnscaledValue(sales_price#84)), sum(UnscaledValue(return_amt#86)), sum(UnscaledValue(profit#85)), sum(UnscaledValue(net_loss#87))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#84))#111, sum(UnscaledValue(return_amt#86))#112, sum(UnscaledValue(profit#85))#113, sum(UnscaledValue(net_loss#87))#114] -Results [5]: [web channel AS channel#115, concat(web_site, web_site_id#106) AS id#116, MakeDecimal(sum(UnscaledValue(sales_price#84))#111,17,2) AS sales#117, MakeDecimal(sum(UnscaledValue(return_amt#86))#112,17,2) AS returns#118, (MakeDecimal(sum(UnscaledValue(profit#85))#113,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#87))#114,17,2)) AS profit#119] - -(132) Union - -(133) HashAggregate [codegen id : 24] -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#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] -Results [8]: [channel#34, id#35, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] - -(134) Exchange -Input [8]: [channel#34, id#35, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] -Arguments: hashpartitioning(channel#34, id#35, 5), ENSURE_REQUIREMENTS, [plan_id=9] - -(135) HashAggregate [codegen id : 25] -Input [8]: [channel#34, id#35, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] +(75) HashAggregate [codegen id : 25] +Input [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] Keys [2]: [channel#34, id#35] Functions [3]: [sum(sales#36), sum(returns#37), sum(profit#38)] -Aggregate Attributes [3]: [sum(sales#36)#132, sum(returns#37)#133, sum(profit#38)#134] -Results [3]: [sum(sales#36)#132 AS sales#151, sum(returns#37)#133 AS returns#152, sum(profit#38)#134 AS profit#153] +Aggregate Attributes [3]: [sum(sales#36)#130, sum(returns#37)#131, sum(profit#38)#132] +Results [3]: [sum(sales#36)#130 AS sales#136, sum(returns#37)#131 AS returns#137, sum(profit#38)#132 AS profit#138] -(136) HashAggregate [codegen id : 25] -Input [3]: [sales#151, returns#152, profit#153] +(76) HashAggregate [codegen id : 25] +Input [3]: [sales#136, returns#137, profit#138] Keys: [] -Functions [3]: [partial_sum(sales#151), partial_sum(returns#152), partial_sum(profit#153)] -Aggregate Attributes [6]: [sum#186, isEmpty#187, sum#188, isEmpty#189, sum#190, isEmpty#191] -Results [6]: [sum#192, isEmpty#193, sum#194, isEmpty#195, sum#196, isEmpty#197] +Functions [3]: [partial_sum(sales#136), partial_sum(returns#137), partial_sum(profit#138)] +Aggregate Attributes [6]: [sum#158, isEmpty#159, sum#160, isEmpty#161, sum#162, isEmpty#163] +Results [6]: [sum#164, isEmpty#165, sum#166, isEmpty#167, sum#168, isEmpty#169] -(137) Exchange -Input [6]: [sum#192, isEmpty#193, sum#194, isEmpty#195, sum#196, isEmpty#197] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] +(77) Exchange +Input [6]: [sum#164, isEmpty#165, sum#166, isEmpty#167, sum#168, isEmpty#169] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(138) HashAggregate [codegen id : 26] -Input [6]: [sum#192, isEmpty#193, sum#194, isEmpty#195, sum#196, isEmpty#197] +(78) HashAggregate [codegen id : 26] +Input [6]: [sum#164, isEmpty#165, sum#166, isEmpty#167, sum#168, isEmpty#169] Keys: [] -Functions [3]: [sum(sales#151), sum(returns#152), sum(profit#153)] -Aggregate Attributes [3]: [sum(sales#151)#198, sum(returns#152)#199, sum(profit#153)#200] -Results [5]: [null AS channel#201, null AS id#202, sum(sales#151)#198 AS sum(sales)#203, sum(returns#152)#199 AS sum(returns)#204, sum(profit#153)#200 AS sum(profit)#205] +Functions [3]: [sum(sales#136), sum(returns#137), sum(profit#138)] +Aggregate Attributes [3]: [sum(sales#136)#170, sum(returns#137)#171, sum(profit#138)#172] +Results [5]: [null AS channel#173, null AS id#174, sum(sales#136)#170 AS sum(sales)#175, sum(returns#137)#171 AS sum(returns)#176, sum(profit#138)#172 AS sum(profit)#177] -(139) Union +(79) Union -(140) HashAggregate [codegen id : 27] -Input [5]: [channel#34, id#35, sales#135, returns#136, profit#137] -Keys [5]: [channel#34, id#35, sales#135, returns#136, profit#137] +(80) HashAggregate [codegen id : 27] +Input [5]: [channel#34, id#35, sales#133, returns#134, profit#135] +Keys [5]: [channel#34, id#35, sales#133, returns#134, profit#135] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#34, id#35, sales#135, returns#136, profit#137] +Results [5]: [channel#34, id#35, sales#133, returns#134, profit#135] -(141) Exchange -Input [5]: [channel#34, id#35, sales#135, returns#136, profit#137] -Arguments: hashpartitioning(channel#34, id#35, sales#135, returns#136, profit#137, 5), ENSURE_REQUIREMENTS, [plan_id=11] +(81) Exchange +Input [5]: [channel#34, id#35, sales#133, returns#134, profit#135] +Arguments: hashpartitioning(channel#34, id#35, sales#133, returns#134, profit#135, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(142) HashAggregate [codegen id : 28] -Input [5]: [channel#34, id#35, sales#135, returns#136, profit#137] -Keys [5]: [channel#34, id#35, sales#135, returns#136, profit#137] +(82) HashAggregate [codegen id : 28] +Input [5]: [channel#34, id#35, sales#133, returns#134, profit#135] +Keys [5]: [channel#34, id#35, sales#133, returns#134, profit#135] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#34, id#35, sales#135, returns#136, profit#137] +Results [5]: [channel#34, id#35, sales#133, returns#134, profit#135] -(143) TakeOrderedAndProject -Input [5]: [channel#34, id#35, sales#135, returns#136, profit#137] -Arguments: 100, [channel#34 ASC NULLS FIRST, id#35 ASC NULLS FIRST], [channel#34, id#35, sales#135, returns#136, profit#137] +(83) TakeOrderedAndProject +Input [5]: [channel#34, id#35, sales#133, returns#134, profit#135] +Arguments: 100, [channel#34 ASC NULLS FIRST, id#35 ASC NULLS FIRST], [channel#34, id#35, sales#133, returns#134, profit#135] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (148) -+- * ColumnarToRow (147) - +- CometProject (146) - +- CometFilter (145) - +- CometScan parquet spark_catalog.default.date_dim (144) +BroadcastExchange (88) ++- * ColumnarToRow (87) + +- CometProject (86) + +- CometFilter (85) + +- CometScan parquet spark_catalog.default.date_dim (84) -(144) Scan parquet spark_catalog.default.date_dim +(84) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#22, d_date#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] ReadSchema: struct -(145) CometFilter +(85) CometFilter Input [2]: [d_date_sk#22, d_date#23] Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) -(146) CometProject +(86) CometProject Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(147) ColumnarToRow [codegen id : 1] +(87) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(148) BroadcastExchange +(88) BroadcastExchange Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 @@ -878,16 +530,8 @@ Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#42 IN d Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#53 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 45 Hosting Expression = ws_sold_date_sk#80 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 48 Hosting Expression = wr_returned_date_sk#92 IN dynamicpruning#5 - -Subquery:7 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#80 IN dynamicpruning#5 - -Subquery:8 Hosting operator id = 82 Hosting Expression = wr_returned_date_sk#92 IN dynamicpruning#5 - -Subquery:9 Hosting operator id = 111 Hosting Expression = ws_sold_date_sk#80 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#79 IN dynamicpruning#5 -Subquery:10 Hosting operator id = 114 Hosting Expression = wr_returned_date_sk#92 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 45 Hosting Expression = wr_returned_date_sk#91 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt index 8c523ae02..dd66c9582 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt @@ -71,17 +71,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometFilter [cr_catalog_page_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #8 - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #9 + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange #8 CometFilter [cp_catalog_page_sk] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (6) HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] InputAdapter - Exchange [web_site_id] #10 + Exchange [web_site_id] #9 WholeStageCodegen (5) ColumnarToRow InputAdapter @@ -97,112 +94,31 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedSubquery [d_date_sk] #1 CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - CometBroadcastExchange #11 + CometBroadcastExchange #10 CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - CometBroadcastExchange #12 - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #13 + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange #11 CometFilter [web_site_sk] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] WholeStageCodegen (17) HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange [channel] #14 + Exchange [channel] #12 WholeStageCodegen (16) 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 - Exchange [channel,id] #15 - WholeStageCodegen (15) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (10) - HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - InputAdapter - ReusedExchange [s_store_id,sum,sum,sum,sum] #3 - WholeStageCodegen (12) - HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - InputAdapter - ReusedExchange [cp_catalog_page_id,sum,sum,sum,sum] #7 - WholeStageCodegen (14) - HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - InputAdapter - Exchange [web_site_id] #16 - WholeStageCodegen (13) - ColumnarToRow - InputAdapter - CometHashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] - CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,web_site_sk] - CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion - CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ws_web_site_sk] - CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - CometBroadcastExchange #17 - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - ReusedExchange [d_date_sk] #12 - ReusedExchange [web_site_sk,web_site_id] #13 + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 WholeStageCodegen (26) HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange #18 + Exchange #13 WholeStageCodegen (25) 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 - Exchange [channel,id] #19 - WholeStageCodegen (24) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (19) - HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - InputAdapter - ReusedExchange [s_store_id,sum,sum,sum,sum] #3 - WholeStageCodegen (21) - HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - InputAdapter - ReusedExchange [cp_catalog_page_id,sum,sum,sum,sum] #7 - WholeStageCodegen (23) - HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - InputAdapter - Exchange [web_site_id] #20 - WholeStageCodegen (22) - ColumnarToRow - InputAdapter - CometHashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] - CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,web_site_sk] - CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion - CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ws_web_site_sk] - CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - CometBroadcastExchange #21 - CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - ReusedExchange [d_date_sk] #12 - ReusedExchange [web_site_sk,web_site_id] #13 + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt index 88ee6fd1f..a1c93644a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt @@ -1,18 +1,18 @@ == Physical Plan == -TakeOrderedAndProject (60) -+- * Project (59) - +- Window (58) - +- * Sort (57) - +- Exchange (56) - +- * HashAggregate (55) - +- Exchange (54) - +- * HashAggregate (53) - +- Union (52) - :- * HashAggregate (41) - : +- Exchange (40) - : +- * HashAggregate (39) - : +- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) +TakeOrderedAndProject (57) ++- * Project (56) + +- Window (55) + +- * Sort (54) + +- Exchange (53) + +- * HashAggregate (52) + +- Exchange (51) + +- * HashAggregate (50) + +- Union (49) + :- * HashAggregate (38) + : +- Exchange (37) + : +- * HashAggregate (36) + : +- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) : :- * ColumnarToRow (9) : : +- CometProject (8) : : +- CometBroadcastHashJoin (7) @@ -22,22 +22,22 @@ TakeOrderedAndProject (60) : : +- CometProject (5) : : +- CometFilter (4) : : +- CometScan parquet spark_catalog.default.date_dim (3) - : +- BroadcastExchange (36) - : +- * BroadcastHashJoin LeftSemi BuildRight (35) + : +- BroadcastExchange (33) + : +- * BroadcastHashJoin LeftSemi BuildRight (32) : :- * ColumnarToRow (12) : : +- CometFilter (11) : : +- CometScan parquet spark_catalog.default.store (10) - : +- BroadcastExchange (34) - : +- * Project (33) - : +- * Filter (32) - : +- Window (31) - : +- * Sort (30) - : +- * HashAggregate (29) - : +- Exchange (28) - : +- * ColumnarToRow (27) - : +- CometHashAggregate (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) + : +- BroadcastExchange (31) + : +- * Project (30) + : +- * Filter (29) + : +- Window (28) + : +- * Sort (27) + : +- * HashAggregate (26) + : +- Exchange (25) + : +- * ColumnarToRow (24) + : +- CometHashAggregate (23) + : +- CometProject (22) + : +- CometBroadcastHashJoin (21) : :- CometProject (19) : : +- CometBroadcastHashJoin (18) : : :- CometFilter (14) @@ -45,20 +45,17 @@ TakeOrderedAndProject (60) : : +- CometBroadcastExchange (17) : : +- CometFilter (16) : : +- CometScan parquet spark_catalog.default.store (15) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan parquet spark_catalog.default.date_dim (20) - :- * HashAggregate (46) - : +- Exchange (45) - : +- * HashAggregate (44) - : +- * HashAggregate (43) - : +- ReusedExchange (42) - +- * HashAggregate (51) - +- Exchange (50) - +- * HashAggregate (49) - +- * HashAggregate (48) - +- ReusedExchange (47) + : +- ReusedExchange (20) + :- * HashAggregate (43) + : +- Exchange (42) + : +- * HashAggregate (41) + : +- * HashAggregate (40) + : +- ReusedExchange (39) + +- * HashAggregate (48) + +- Exchange (47) + +- * HashAggregate (46) + +- * HashAggregate (45) + +- ReusedExchange (44) (1) Scan parquet spark_catalog.default.store_sales @@ -154,236 +151,220 @@ Arguments: [ss_store_sk#10], [s_store_sk#14], Inner Input [5]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12, s_store_sk#14, s_state#15] Arguments: [ss_net_profit#11, ss_sold_date_sk#12, s_state#15], [ss_net_profit#11, ss_sold_date_sk#12, s_state#15] -(20) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#16, d_month_seq#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [d_date_sk#16, d_month_seq#17] -Condition : (((isnotnull(d_month_seq#17) AND (d_month_seq#17 >= 1212)) AND (d_month_seq#17 <= 1223)) AND isnotnull(d_date_sk#16)) +(20) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#16] -(22) CometProject -Input [2]: [d_date_sk#16, d_month_seq#17] -Arguments: [d_date_sk#16], [d_date_sk#16] - -(23) CometBroadcastExchange -Input [1]: [d_date_sk#16] -Arguments: [d_date_sk#16] - -(24) CometBroadcastHashJoin +(21) CometBroadcastHashJoin Left output [3]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#15] Right output [1]: [d_date_sk#16] Arguments: [ss_sold_date_sk#12], [d_date_sk#16], Inner -(25) CometProject +(22) CometProject Input [4]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#15, d_date_sk#16] Arguments: [ss_net_profit#11, s_state#15], [ss_net_profit#11, s_state#15] -(26) CometHashAggregate +(23) CometHashAggregate Input [2]: [ss_net_profit#11, s_state#15] Keys [1]: [s_state#15] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] -(27) ColumnarToRow [codegen id : 1] -Input [2]: [s_state#15, sum#18] +(24) ColumnarToRow [codegen id : 1] +Input [2]: [s_state#15, sum#17] -(28) Exchange -Input [2]: [s_state#15, sum#18] +(25) Exchange +Input [2]: [s_state#15, sum#17] Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(29) HashAggregate [codegen id : 2] -Input [2]: [s_state#15, sum#18] +(26) HashAggregate [codegen id : 2] +Input [2]: [s_state#15, sum#17] Keys [1]: [s_state#15] Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#11))#19] -Results [3]: [s_state#15, MakeDecimal(sum(UnscaledValue(ss_net_profit#11))#19,17,2) AS _w0#20, s_state#15] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#11))#18] +Results [3]: [s_state#15, MakeDecimal(sum(UnscaledValue(ss_net_profit#11))#18,17,2) AS _w0#19, s_state#15] -(30) Sort [codegen id : 2] -Input [3]: [s_state#15, _w0#20, s_state#15] -Arguments: [s_state#15 ASC NULLS FIRST, _w0#20 DESC NULLS LAST], false, 0 +(27) Sort [codegen id : 2] +Input [3]: [s_state#15, _w0#19, s_state#15] +Arguments: [s_state#15 ASC NULLS FIRST, _w0#19 DESC NULLS LAST], false, 0 -(31) Window -Input [3]: [s_state#15, _w0#20, s_state#15] -Arguments: [rank(_w0#20) windowspecdefinition(s_state#15, _w0#20 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#21], [s_state#15], [_w0#20 DESC NULLS LAST] +(28) Window +Input [3]: [s_state#15, _w0#19, s_state#15] +Arguments: [rank(_w0#19) windowspecdefinition(s_state#15, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#15], [_w0#19 DESC NULLS LAST] -(32) Filter [codegen id : 3] -Input [4]: [s_state#15, _w0#20, s_state#15, ranking#21] -Condition : (ranking#21 <= 5) +(29) Filter [codegen id : 3] +Input [4]: [s_state#15, _w0#19, s_state#15, ranking#20] +Condition : (ranking#20 <= 5) -(33) Project [codegen id : 3] +(30) Project [codegen id : 3] Output [1]: [s_state#15] -Input [4]: [s_state#15, _w0#20, s_state#15, ranking#21] +Input [4]: [s_state#15, _w0#19, s_state#15, ranking#20] -(34) BroadcastExchange +(31) BroadcastExchange Input [1]: [s_state#15] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] -(35) BroadcastHashJoin [codegen id : 4] +(32) BroadcastHashJoin [codegen id : 4] Left keys [1]: [s_state#9] Right keys [1]: [s_state#15] Join type: LeftSemi Join condition: None -(36) BroadcastExchange +(33) BroadcastExchange Input [3]: [s_store_sk#7, s_county#8, s_state#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(37) BroadcastHashJoin [codegen id : 5] +(34) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_store_sk#1] Right keys [1]: [s_store_sk#7] Join type: Inner Join condition: None -(38) Project [codegen id : 5] +(35) Project [codegen id : 5] Output [3]: [ss_net_profit#2, s_county#8, s_state#9] Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#9] -(39) HashAggregate [codegen id : 5] +(36) HashAggregate [codegen id : 5] Input [3]: [ss_net_profit#2, s_county#8, s_state#9] Keys [2]: [s_state#9, s_county#8] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#22] -Results [3]: [s_state#9, s_county#8, sum#23] +Aggregate Attributes [1]: [sum#21] +Results [3]: [s_state#9, s_county#8, sum#22] -(40) Exchange -Input [3]: [s_state#9, s_county#8, sum#23] +(37) Exchange +Input [3]: [s_state#9, s_county#8, sum#22] Arguments: hashpartitioning(s_state#9, s_county#8, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(41) HashAggregate [codegen id : 6] -Input [3]: [s_state#9, s_county#8, sum#23] +(38) HashAggregate [codegen id : 6] +Input [3]: [s_state#9, s_county#8, sum#22] Keys [2]: [s_state#9, s_county#8] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#24] -Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) as decimal(27,2)) AS total_sum#25, s_state#9, s_county#8, 0 AS g_state#26, 0 AS g_county#27, 0 AS lochierarchy#28] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#23] +Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#23,17,2) as decimal(27,2)) AS total_sum#24, s_state#9, s_county#8, 0 AS g_state#25, 0 AS g_county#26, 0 AS lochierarchy#27] -(42) ReusedExchange [Reuses operator id: 40] -Output [3]: [s_state#9, s_county#8, sum#29] +(39) ReusedExchange [Reuses operator id: 37] +Output [3]: [s_state#9, s_county#8, sum#28] -(43) HashAggregate [codegen id : 12] -Input [3]: [s_state#9, s_county#8, sum#29] +(40) HashAggregate [codegen id : 12] +Input [3]: [s_state#9, s_county#8, sum#28] Keys [2]: [s_state#9, s_county#8] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#24] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) AS total_sum#30, s_state#9] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#23] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#23,17,2) AS total_sum#29, s_state#9] -(44) HashAggregate [codegen id : 12] -Input [2]: [total_sum#30, s_state#9] +(41) HashAggregate [codegen id : 12] +Input [2]: [total_sum#29, s_state#9] Keys [1]: [s_state#9] -Functions [1]: [partial_sum(total_sum#30)] -Aggregate Attributes [2]: [sum#31, isEmpty#32] -Results [3]: [s_state#9, sum#33, isEmpty#34] +Functions [1]: [partial_sum(total_sum#29)] +Aggregate Attributes [2]: [sum#30, isEmpty#31] +Results [3]: [s_state#9, sum#32, isEmpty#33] -(45) Exchange -Input [3]: [s_state#9, sum#33, isEmpty#34] +(42) Exchange +Input [3]: [s_state#9, sum#32, isEmpty#33] Arguments: hashpartitioning(s_state#9, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(46) HashAggregate [codegen id : 13] -Input [3]: [s_state#9, sum#33, isEmpty#34] +(43) HashAggregate [codegen id : 13] +Input [3]: [s_state#9, sum#32, isEmpty#33] Keys [1]: [s_state#9] -Functions [1]: [sum(total_sum#30)] -Aggregate Attributes [1]: [sum(total_sum#30)#35] -Results [6]: [sum(total_sum#30)#35 AS total_sum#36, s_state#9, null AS s_county#37, 0 AS g_state#38, 1 AS g_county#39, 1 AS lochierarchy#40] +Functions [1]: [sum(total_sum#29)] +Aggregate Attributes [1]: [sum(total_sum#29)#34] +Results [6]: [sum(total_sum#29)#34 AS total_sum#35, s_state#9, null AS s_county#36, 0 AS g_state#37, 1 AS g_county#38, 1 AS lochierarchy#39] -(47) ReusedExchange [Reuses operator id: 40] -Output [3]: [s_state#9, s_county#8, sum#41] +(44) ReusedExchange [Reuses operator id: 37] +Output [3]: [s_state#9, s_county#8, sum#40] -(48) HashAggregate [codegen id : 19] -Input [3]: [s_state#9, s_county#8, sum#41] +(45) HashAggregate [codegen id : 19] +Input [3]: [s_state#9, s_county#8, sum#40] Keys [2]: [s_state#9, s_county#8] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#24] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) AS total_sum#30] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#23] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#23,17,2) AS total_sum#29] -(49) HashAggregate [codegen id : 19] -Input [1]: [total_sum#30] +(46) HashAggregate [codegen id : 19] +Input [1]: [total_sum#29] Keys: [] -Functions [1]: [partial_sum(total_sum#30)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [2]: [sum#44, isEmpty#45] +Functions [1]: [partial_sum(total_sum#29)] +Aggregate Attributes [2]: [sum#41, isEmpty#42] +Results [2]: [sum#43, isEmpty#44] -(50) Exchange -Input [2]: [sum#44, isEmpty#45] +(47) Exchange +Input [2]: [sum#43, isEmpty#44] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(51) HashAggregate [codegen id : 20] -Input [2]: [sum#44, isEmpty#45] +(48) HashAggregate [codegen id : 20] +Input [2]: [sum#43, isEmpty#44] Keys: [] -Functions [1]: [sum(total_sum#30)] -Aggregate Attributes [1]: [sum(total_sum#30)#46] -Results [6]: [sum(total_sum#30)#46 AS total_sum#47, null AS s_state#48, null AS s_county#49, 1 AS g_state#50, 1 AS g_county#51, 2 AS lochierarchy#52] +Functions [1]: [sum(total_sum#29)] +Aggregate Attributes [1]: [sum(total_sum#29)#45] +Results [6]: [sum(total_sum#29)#45 AS total_sum#46, null AS s_state#47, null AS s_county#48, 1 AS g_state#49, 1 AS g_county#50, 2 AS lochierarchy#51] -(52) Union +(49) Union -(53) HashAggregate [codegen id : 21] -Input [6]: [total_sum#25, s_state#9, s_county#8, g_state#26, g_county#27, lochierarchy#28] -Keys [6]: [total_sum#25, s_state#9, s_county#8, g_state#26, g_county#27, lochierarchy#28] +(50) HashAggregate [codegen id : 21] +Input [6]: [total_sum#24, s_state#9, s_county#8, g_state#25, g_county#26, lochierarchy#27] +Keys [6]: [total_sum#24, s_state#9, s_county#8, g_state#25, g_county#26, lochierarchy#27] Functions: [] Aggregate Attributes: [] -Results [6]: [total_sum#25, s_state#9, s_county#8, g_state#26, g_county#27, lochierarchy#28] +Results [6]: [total_sum#24, s_state#9, s_county#8, g_state#25, g_county#26, lochierarchy#27] -(54) Exchange -Input [6]: [total_sum#25, s_state#9, s_county#8, g_state#26, g_county#27, lochierarchy#28] -Arguments: hashpartitioning(total_sum#25, s_state#9, s_county#8, g_state#26, g_county#27, lochierarchy#28, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(51) Exchange +Input [6]: [total_sum#24, s_state#9, s_county#8, g_state#25, g_county#26, lochierarchy#27] +Arguments: hashpartitioning(total_sum#24, s_state#9, s_county#8, g_state#25, g_county#26, lochierarchy#27, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(55) HashAggregate [codegen id : 22] -Input [6]: [total_sum#25, s_state#9, s_county#8, g_state#26, g_county#27, lochierarchy#28] -Keys [6]: [total_sum#25, s_state#9, s_county#8, g_state#26, g_county#27, lochierarchy#28] +(52) HashAggregate [codegen id : 22] +Input [6]: [total_sum#24, s_state#9, s_county#8, g_state#25, g_county#26, lochierarchy#27] +Keys [6]: [total_sum#24, s_state#9, s_county#8, g_state#25, g_county#26, lochierarchy#27] Functions: [] Aggregate Attributes: [] -Results [5]: [total_sum#25, s_state#9, s_county#8, lochierarchy#28, CASE WHEN (g_county#27 = 0) THEN s_state#9 END AS _w0#53] +Results [5]: [total_sum#24, s_state#9, s_county#8, lochierarchy#27, CASE WHEN (g_county#26 = 0) THEN s_state#9 END AS _w0#52] -(56) Exchange -Input [5]: [total_sum#25, s_state#9, s_county#8, lochierarchy#28, _w0#53] -Arguments: hashpartitioning(lochierarchy#28, _w0#53, 5), ENSURE_REQUIREMENTS, [plan_id=8] +(53) Exchange +Input [5]: [total_sum#24, s_state#9, s_county#8, lochierarchy#27, _w0#52] +Arguments: hashpartitioning(lochierarchy#27, _w0#52, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(57) Sort [codegen id : 23] -Input [5]: [total_sum#25, s_state#9, s_county#8, lochierarchy#28, _w0#53] -Arguments: [lochierarchy#28 ASC NULLS FIRST, _w0#53 ASC NULLS FIRST, total_sum#25 DESC NULLS LAST], false, 0 +(54) Sort [codegen id : 23] +Input [5]: [total_sum#24, s_state#9, s_county#8, lochierarchy#27, _w0#52] +Arguments: [lochierarchy#27 ASC NULLS FIRST, _w0#52 ASC NULLS FIRST, total_sum#24 DESC NULLS LAST], false, 0 -(58) Window -Input [5]: [total_sum#25, s_state#9, s_county#8, lochierarchy#28, _w0#53] -Arguments: [rank(total_sum#25) windowspecdefinition(lochierarchy#28, _w0#53, total_sum#25 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#54], [lochierarchy#28, _w0#53], [total_sum#25 DESC NULLS LAST] +(55) Window +Input [5]: [total_sum#24, s_state#9, s_county#8, lochierarchy#27, _w0#52] +Arguments: [rank(total_sum#24) windowspecdefinition(lochierarchy#27, _w0#52, total_sum#24 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#53], [lochierarchy#27, _w0#52], [total_sum#24 DESC NULLS LAST] -(59) Project [codegen id : 24] -Output [5]: [total_sum#25, s_state#9, s_county#8, lochierarchy#28, rank_within_parent#54] -Input [6]: [total_sum#25, s_state#9, s_county#8, lochierarchy#28, _w0#53, rank_within_parent#54] +(56) Project [codegen id : 24] +Output [5]: [total_sum#24, s_state#9, s_county#8, lochierarchy#27, rank_within_parent#53] +Input [6]: [total_sum#24, s_state#9, s_county#8, lochierarchy#27, _w0#52, rank_within_parent#53] -(60) TakeOrderedAndProject -Input [5]: [total_sum#25, s_state#9, s_county#8, lochierarchy#28, rank_within_parent#54] -Arguments: 100, [lochierarchy#28 DESC NULLS LAST, CASE WHEN (lochierarchy#28 = 0) THEN s_state#9 END ASC NULLS FIRST, rank_within_parent#54 ASC NULLS FIRST], [total_sum#25, s_state#9, s_county#8, lochierarchy#28, rank_within_parent#54] +(57) TakeOrderedAndProject +Input [5]: [total_sum#24, s_state#9, s_county#8, lochierarchy#27, rank_within_parent#53] +Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN s_state#9 END ASC NULLS FIRST, rank_within_parent#53 ASC NULLS FIRST], [total_sum#24, s_state#9, s_county#8, lochierarchy#27, rank_within_parent#53] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (65) -+- * ColumnarToRow (64) - +- CometProject (63) - +- CometFilter (62) - +- CometScan parquet spark_catalog.default.date_dim (61) +BroadcastExchange (62) ++- * ColumnarToRow (61) + +- CometProject (60) + +- CometFilter (59) + +- CometScan parquet spark_catalog.default.date_dim (58) -(61) Scan parquet spark_catalog.default.date_dim +(58) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#5, d_month_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(62) CometFilter +(59) CometFilter Input [2]: [d_date_sk#5, d_month_seq#6] Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) -(63) CometProject +(60) CometProject Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(64) ColumnarToRow [codegen id : 1] +(61) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(65) BroadcastExchange +(62) BroadcastExchange Input [1]: [d_date_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt index da7ab7e0e..9e72edbb4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt @@ -75,14 +75,11 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count CometBroadcastExchange #9 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange #10 - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + ReusedExchange [d_date_sk] #5 WholeStageCodegen (13) HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] InputAdapter - Exchange [s_state] #11 + Exchange [s_state] #10 WholeStageCodegen (12) HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] @@ -91,7 +88,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count WholeStageCodegen (20) HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] InputAdapter - Exchange #12 + Exchange #11 WholeStageCodegen (19) HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt index b092e0cbc..c8a2d3a23 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (75) -+- * Project (74) - +- * BroadcastHashJoin Inner BuildRight (73) - :- * Project (55) - : +- * BroadcastHashJoin Inner BuildRight (54) +TakeOrderedAndProject (71) ++- * Project (70) + +- * BroadcastHashJoin Inner BuildRight (69) + :- * Project (53) + : +- * BroadcastHashJoin Inner BuildRight (52) : :- * BroadcastHashJoin Inner BuildRight (35) : : :- * Filter (17) : : : +- * HashAggregate (16) @@ -39,14 +39,14 @@ TakeOrderedAndProject (75) : : +- CometBroadcastExchange (27) : : +- CometFilter (26) : : +- CometScan parquet spark_catalog.default.date_dim (25) - : +- BroadcastExchange (53) - : +- * Filter (52) - : +- * HashAggregate (51) - : +- Exchange (50) - : +- * ColumnarToRow (49) - : +- CometHashAggregate (48) - : +- CometProject (47) - : +- CometBroadcastHashJoin (46) + : +- BroadcastExchange (51) + : +- * Filter (50) + : +- * HashAggregate (49) + : +- Exchange (48) + : +- * ColumnarToRow (47) + : +- CometHashAggregate (46) + : +- CometProject (45) + : +- CometBroadcastHashJoin (44) : :- CometProject (42) : : +- CometBroadcastHashJoin (41) : : :- CometFilter (37) @@ -54,26 +54,22 @@ TakeOrderedAndProject (75) : : +- CometBroadcastExchange (40) : : +- CometFilter (39) : : +- CometScan parquet spark_catalog.default.web_sales (38) - : +- CometBroadcastExchange (45) - : +- CometFilter (44) - : +- CometScan parquet spark_catalog.default.date_dim (43) - +- BroadcastExchange (72) - +- * HashAggregate (71) - +- Exchange (70) - +- * ColumnarToRow (69) - +- CometHashAggregate (68) - +- CometProject (67) - +- CometBroadcastHashJoin (66) - :- CometProject (62) - : +- CometBroadcastHashJoin (61) - : :- CometFilter (57) - : : +- CometScan parquet spark_catalog.default.customer (56) - : +- CometBroadcastExchange (60) - : +- CometFilter (59) - : +- CometScan parquet spark_catalog.default.web_sales (58) - +- CometBroadcastExchange (65) - +- CometFilter (64) - +- CometScan parquet spark_catalog.default.date_dim (63) + : +- ReusedExchange (43) + +- BroadcastExchange (68) + +- * HashAggregate (67) + +- Exchange (66) + +- * ColumnarToRow (65) + +- CometHashAggregate (64) + +- CometProject (63) + +- CometBroadcastHashJoin (62) + :- CometProject (60) + : +- CometBroadcastHashJoin (59) + : :- CometFilter (55) + : : +- CometScan parquet spark_catalog.default.customer (54) + : +- CometBroadcastExchange (58) + : +- CometFilter (57) + : +- CometScan parquet spark_catalog.default.web_sales (56) + +- ReusedExchange (61) (1) Scan parquet spark_catalog.default.customer @@ -284,79 +280,67 @@ Arguments: [c_customer_sk#30], [ws_bill_customer_sk#34], Inner Input [7]: [c_customer_sk#30, c_customer_id#31, c_first_name#32, c_last_name#33, ws_bill_customer_sk#34, ws_net_paid#35, ws_sold_date_sk#36] Arguments: [c_customer_id#31, c_first_name#32, c_last_name#33, ws_net_paid#35, ws_sold_date_sk#36], [c_customer_id#31, c_first_name#32, c_last_name#33, ws_net_paid#35, ws_sold_date_sk#36] -(43) Scan parquet spark_catalog.default.date_dim +(43) ReusedExchange [Reuses operator id: 10] Output [2]: [d_date_sk#38, d_year#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct -(44) CometFilter -Input [2]: [d_date_sk#38, d_year#39] -Condition : (((isnotnull(d_year#39) AND (d_year#39 = 2001)) AND d_year#39 IN (2001,2002)) AND isnotnull(d_date_sk#38)) - -(45) CometBroadcastExchange -Input [2]: [d_date_sk#38, d_year#39] -Arguments: [d_date_sk#38, d_year#39] - -(46) CometBroadcastHashJoin +(44) CometBroadcastHashJoin Left output [5]: [c_customer_id#31, c_first_name#32, c_last_name#33, ws_net_paid#35, ws_sold_date_sk#36] Right output [2]: [d_date_sk#38, d_year#39] Arguments: [ws_sold_date_sk#36], [d_date_sk#38], Inner -(47) CometProject +(45) CometProject Input [7]: [c_customer_id#31, c_first_name#32, c_last_name#33, ws_net_paid#35, ws_sold_date_sk#36, d_date_sk#38, d_year#39] Arguments: [c_customer_id#31, c_first_name#32, c_last_name#33, ws_net_paid#35, d_year#39], [c_customer_id#31, c_first_name#32, c_last_name#33, ws_net_paid#35, d_year#39] -(48) CometHashAggregate +(46) CometHashAggregate Input [5]: [c_customer_id#31, c_first_name#32, c_last_name#33, ws_net_paid#35, d_year#39] Keys [4]: [c_customer_id#31, c_first_name#32, c_last_name#33, d_year#39] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#35))] -(49) ColumnarToRow [codegen id : 4] +(47) ColumnarToRow [codegen id : 4] Input [5]: [c_customer_id#31, c_first_name#32, c_last_name#33, d_year#39, sum#40] -(50) Exchange +(48) Exchange Input [5]: [c_customer_id#31, c_first_name#32, c_last_name#33, d_year#39, sum#40] Arguments: hashpartitioning(c_customer_id#31, c_first_name#32, c_last_name#33, d_year#39, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(51) HashAggregate [codegen id : 5] +(49) HashAggregate [codegen id : 5] Input [5]: [c_customer_id#31, c_first_name#32, c_last_name#33, d_year#39, sum#40] Keys [4]: [c_customer_id#31, c_first_name#32, c_last_name#33, d_year#39] Functions [1]: [sum(UnscaledValue(ws_net_paid#35))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#35))#41] Results [2]: [c_customer_id#31 AS customer_id#42, MakeDecimal(sum(UnscaledValue(ws_net_paid#35))#41,17,2) AS year_total#43] -(52) Filter [codegen id : 5] +(50) Filter [codegen id : 5] Input [2]: [customer_id#42, year_total#43] Condition : (isnotnull(year_total#43) AND (year_total#43 > 0.00)) -(53) BroadcastExchange +(51) BroadcastExchange Input [2]: [customer_id#42, year_total#43] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(54) BroadcastHashJoin [codegen id : 8] +(52) BroadcastHashJoin [codegen id : 8] Left keys [1]: [customer_id#13] Right keys [1]: [customer_id#42] Join type: Inner Join condition: None -(55) Project [codegen id : 8] +(53) Project [codegen id : 8] Output [7]: [customer_id#13, year_total#14, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, year_total#43] Input [8]: [customer_id#13, year_total#14, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, customer_id#42, year_total#43] -(56) Scan parquet spark_catalog.default.customer +(54) Scan parquet spark_catalog.default.customer Output [4]: [c_customer_sk#44, c_customer_id#45, c_first_name#46, c_last_name#47] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(57) CometFilter +(55) CometFilter Input [4]: [c_customer_sk#44, c_customer_id#45, c_first_name#46, c_last_name#47] Condition : (isnotnull(c_customer_sk#44) AND isnotnull(c_customer_id#45)) -(58) Scan parquet spark_catalog.default.web_sales +(56) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] Batched: true Location: InMemoryFileIndex [] @@ -364,138 +348,126 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#50), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(59) CometFilter +(57) CometFilter Input [3]: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] Condition : isnotnull(ws_bill_customer_sk#48) -(60) CometBroadcastExchange +(58) CometBroadcastExchange Input [3]: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] Arguments: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] -(61) CometBroadcastHashJoin +(59) CometBroadcastHashJoin Left output [4]: [c_customer_sk#44, c_customer_id#45, c_first_name#46, c_last_name#47] Right output [3]: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] Arguments: [c_customer_sk#44], [ws_bill_customer_sk#48], Inner -(62) CometProject +(60) CometProject Input [7]: [c_customer_sk#44, c_customer_id#45, c_first_name#46, c_last_name#47, ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] Arguments: [c_customer_id#45, c_first_name#46, c_last_name#47, ws_net_paid#49, ws_sold_date_sk#50], [c_customer_id#45, c_first_name#46, c_last_name#47, ws_net_paid#49, ws_sold_date_sk#50] -(63) Scan parquet spark_catalog.default.date_dim +(61) ReusedExchange [Reuses operator id: 27] Output [2]: [d_date_sk#52, d_year#53] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct -(64) CometFilter -Input [2]: [d_date_sk#52, d_year#53] -Condition : (((isnotnull(d_year#53) AND (d_year#53 = 2002)) AND d_year#53 IN (2001,2002)) AND isnotnull(d_date_sk#52)) - -(65) CometBroadcastExchange -Input [2]: [d_date_sk#52, d_year#53] -Arguments: [d_date_sk#52, d_year#53] - -(66) CometBroadcastHashJoin +(62) CometBroadcastHashJoin Left output [5]: [c_customer_id#45, c_first_name#46, c_last_name#47, ws_net_paid#49, ws_sold_date_sk#50] Right output [2]: [d_date_sk#52, d_year#53] Arguments: [ws_sold_date_sk#50], [d_date_sk#52], Inner -(67) CometProject +(63) CometProject Input [7]: [c_customer_id#45, c_first_name#46, c_last_name#47, ws_net_paid#49, ws_sold_date_sk#50, d_date_sk#52, d_year#53] Arguments: [c_customer_id#45, c_first_name#46, c_last_name#47, ws_net_paid#49, d_year#53], [c_customer_id#45, c_first_name#46, c_last_name#47, ws_net_paid#49, d_year#53] -(68) CometHashAggregate +(64) CometHashAggregate Input [5]: [c_customer_id#45, c_first_name#46, c_last_name#47, ws_net_paid#49, d_year#53] Keys [4]: [c_customer_id#45, c_first_name#46, c_last_name#47, d_year#53] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#49))] -(69) ColumnarToRow [codegen id : 6] +(65) ColumnarToRow [codegen id : 6] Input [5]: [c_customer_id#45, c_first_name#46, c_last_name#47, d_year#53, sum#54] -(70) Exchange +(66) Exchange Input [5]: [c_customer_id#45, c_first_name#46, c_last_name#47, d_year#53, sum#54] Arguments: hashpartitioning(c_customer_id#45, c_first_name#46, c_last_name#47, d_year#53, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(71) HashAggregate [codegen id : 7] +(67) HashAggregate [codegen id : 7] Input [5]: [c_customer_id#45, c_first_name#46, c_last_name#47, d_year#53, sum#54] Keys [4]: [c_customer_id#45, c_first_name#46, c_last_name#47, d_year#53] Functions [1]: [sum(UnscaledValue(ws_net_paid#49))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#49))#41] Results [2]: [c_customer_id#45 AS customer_id#55, MakeDecimal(sum(UnscaledValue(ws_net_paid#49))#41,17,2) AS year_total#56] -(72) BroadcastExchange +(68) BroadcastExchange Input [2]: [customer_id#55, year_total#56] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] -(73) BroadcastHashJoin [codegen id : 8] +(69) BroadcastHashJoin [codegen id : 8] Left keys [1]: [customer_id#13] Right keys [1]: [customer_id#55] Join type: Inner Join condition: (CASE WHEN (year_total#43 > 0.00) THEN (year_total#56 / year_total#43) END > CASE WHEN (year_total#14 > 0.00) THEN (year_total#29 / year_total#14) END) -(74) Project [codegen id : 8] +(70) Project [codegen id : 8] Output [3]: [customer_id#26, customer_first_name#27, customer_last_name#28] Input [9]: [customer_id#13, year_total#14, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, year_total#43, customer_id#55, year_total#56] -(75) TakeOrderedAndProject +(71) TakeOrderedAndProject Input [3]: [customer_id#26, customer_first_name#27, customer_last_name#28] Arguments: 100, [customer_first_name#27 ASC NULLS FIRST, customer_id#26 ASC NULLS FIRST, customer_last_name#28 ASC NULLS FIRST], [customer_id#26, customer_first_name#27, customer_last_name#28] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (79) -+- * ColumnarToRow (78) - +- CometFilter (77) - +- CometScan parquet spark_catalog.default.date_dim (76) +BroadcastExchange (75) ++- * ColumnarToRow (74) + +- CometFilter (73) + +- CometScan parquet spark_catalog.default.date_dim (72) -(76) Scan parquet spark_catalog.default.date_dim +(72) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#9, d_year#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct -(77) CometFilter +(73) CometFilter Input [2]: [d_date_sk#9, d_year#10] Condition : (((isnotnull(d_year#10) AND (d_year#10 = 2001)) AND d_year#10 IN (2001,2002)) AND isnotnull(d_date_sk#9)) -(78) ColumnarToRow [codegen id : 1] +(74) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_year#10] -(79) BroadcastExchange +(75) BroadcastExchange Input [2]: [d_date_sk#9, d_year#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#21 IN dynamicpruning#22 -BroadcastExchange (83) -+- * ColumnarToRow (82) - +- CometFilter (81) - +- CometScan parquet spark_catalog.default.date_dim (80) +BroadcastExchange (79) ++- * ColumnarToRow (78) + +- CometFilter (77) + +- CometScan parquet spark_catalog.default.date_dim (76) -(80) Scan parquet spark_catalog.default.date_dim +(76) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#23, d_year#24] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct -(81) CometFilter +(77) CometFilter Input [2]: [d_date_sk#23, d_year#24] Condition : (((isnotnull(d_year#24) AND (d_year#24 = 2002)) AND d_year#24 IN (2001,2002)) AND isnotnull(d_date_sk#23)) -(82) ColumnarToRow [codegen id : 1] +(78) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#23, d_year#24] -(83) BroadcastExchange +(79) BroadcastExchange Input [2]: [d_date_sk#23, d_year#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] Subquery:3 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#8 -Subquery:4 Hosting operator id = 58 Hosting Expression = ws_sold_date_sk#50 IN dynamicpruning#22 +Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#50 IN dynamicpruning#22 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt index 4437e831e..283fe5bde 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt @@ -82,15 +82,13 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] CometFilter [ws_bill_customer_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #13 - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk,d_year] #4 InputAdapter - BroadcastExchange #14 + BroadcastExchange #13 WholeStageCodegen (7) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year] #15 + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #14 WholeStageCodegen (6) ColumnarToRow InputAdapter @@ -101,10 +99,8 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #16 + CometBroadcastExchange #15 CometFilter [ws_bill_customer_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #17 - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk,d_year] #9 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 1334d004a..baf8516e1 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,17 +1,17 @@ == Physical Plan == -TakeOrderedAndProject (155) -+- * Project (154) - +- * SortMergeJoin Inner (153) - :- * Sort (82) - : +- Exchange (81) - : +- * Filter (80) - : +- * HashAggregate (79) - : +- Exchange (78) - : +- * HashAggregate (77) - : +- * HashAggregate (76) - : +- Exchange (75) - : +- * HashAggregate (74) - : +- Union (73) +TakeOrderedAndProject (132) ++- * Project (131) + +- * SortMergeJoin Inner (130) + :- * Sort (72) + : +- Exchange (71) + : +- * Filter (70) + : +- * HashAggregate (69) + : +- Exchange (68) + : +- * HashAggregate (67) + : +- * HashAggregate (66) + : +- Exchange (65) + : +- * HashAggregate (64) + : +- Union (63) : :- * Project (24) : : +- * SortMergeJoin LeftOuter (23) : : :- * Sort (16) @@ -36,124 +36,101 @@ TakeOrderedAndProject (155) : : +- CometProject (19) : : +- CometFilter (18) : : +- CometScan parquet spark_catalog.default.catalog_returns (17) - : :- * Project (48) - : : +- * SortMergeJoin LeftOuter (47) - : : :- * Sort (40) - : : : +- Exchange (39) - : : : +- * ColumnarToRow (38) - : : : +- CometProject (37) - : : : +- CometBroadcastHashJoin (36) - : : : :- CometProject (32) - : : : : +- CometBroadcastHashJoin (31) + : :- * Project (43) + : : +- * SortMergeJoin LeftOuter (42) + : : :- * Sort (35) + : : : +- Exchange (34) + : : : +- * ColumnarToRow (33) + : : : +- CometProject (32) + : : : +- CometBroadcastHashJoin (31) + : : : :- CometProject (29) + : : : : +- CometBroadcastHashJoin (28) : : : : :- CometFilter (26) : : : : : +- CometScan parquet spark_catalog.default.store_sales (25) - : : : : +- CometBroadcastExchange (30) - : : : : +- CometProject (29) - : : : : +- CometFilter (28) - : : : : +- CometScan parquet spark_catalog.default.item (27) - : : : +- CometBroadcastExchange (35) - : : : +- CometFilter (34) - : : : +- CometScan parquet spark_catalog.default.date_dim (33) - : : +- * Sort (46) - : : +- Exchange (45) - : : +- * ColumnarToRow (44) - : : +- CometProject (43) - : : +- CometFilter (42) - : : +- CometScan parquet spark_catalog.default.store_returns (41) - : +- * Project (72) - : +- * SortMergeJoin LeftOuter (71) - : :- * Sort (64) - : : +- Exchange (63) - : : +- * ColumnarToRow (62) - : : +- CometProject (61) - : : +- CometBroadcastHashJoin (60) - : : :- CometProject (56) - : : : +- CometBroadcastHashJoin (55) - : : : :- CometFilter (50) - : : : : +- CometScan parquet spark_catalog.default.web_sales (49) - : : : +- CometBroadcastExchange (54) - : : : +- CometProject (53) - : : : +- CometFilter (52) - : : : +- CometScan parquet spark_catalog.default.item (51) - : : +- CometBroadcastExchange (59) - : : +- CometFilter (58) - : : +- CometScan parquet spark_catalog.default.date_dim (57) - : +- * Sort (70) - : +- Exchange (69) - : +- * ColumnarToRow (68) - : +- CometProject (67) - : +- CometFilter (66) - : +- CometScan parquet spark_catalog.default.web_returns (65) - +- * Sort (152) - +- Exchange (151) - +- * Filter (150) - +- * HashAggregate (149) - +- Exchange (148) - +- * HashAggregate (147) - +- * HashAggregate (146) - +- Exchange (145) - +- * HashAggregate (144) - +- Union (143) - :- * Project (102) - : +- * SortMergeJoin LeftOuter (101) - : :- * Sort (98) - : : +- Exchange (97) - : : +- * ColumnarToRow (96) - : : +- CometProject (95) - : : +- CometBroadcastHashJoin (94) - : : :- CometProject (90) - : : : +- CometBroadcastHashJoin (89) - : : : :- CometFilter (84) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (83) - : : : +- CometBroadcastExchange (88) - : : : +- CometProject (87) - : : : +- CometFilter (86) - : : : +- CometScan parquet spark_catalog.default.item (85) - : : +- CometBroadcastExchange (93) - : : +- CometFilter (92) - : : +- CometScan parquet spark_catalog.default.date_dim (91) - : +- * Sort (100) - : +- ReusedExchange (99) - :- * Project (122) - : +- * SortMergeJoin LeftOuter (121) - : :- * Sort (118) - : : +- Exchange (117) - : : +- * ColumnarToRow (116) - : : +- CometProject (115) - : : +- CometBroadcastHashJoin (114) - : : :- CometProject (110) - : : : +- CometBroadcastHashJoin (109) - : : : :- CometFilter (104) - : : : : +- CometScan parquet spark_catalog.default.store_sales (103) - : : : +- CometBroadcastExchange (108) - : : : +- CometProject (107) - : : : +- CometFilter (106) - : : : +- CometScan parquet spark_catalog.default.item (105) - : : +- CometBroadcastExchange (113) - : : +- CometFilter (112) - : : +- CometScan parquet spark_catalog.default.date_dim (111) - : +- * Sort (120) - : +- ReusedExchange (119) - +- * Project (142) - +- * SortMergeJoin LeftOuter (141) - :- * Sort (138) - : +- Exchange (137) - : +- * ColumnarToRow (136) - : +- CometProject (135) - : +- CometBroadcastHashJoin (134) - : :- CometProject (130) - : : +- CometBroadcastHashJoin (129) - : : :- CometFilter (124) - : : : +- CometScan parquet spark_catalog.default.web_sales (123) - : : +- CometBroadcastExchange (128) - : : +- CometProject (127) - : : +- CometFilter (126) - : : +- CometScan parquet spark_catalog.default.item (125) - : +- CometBroadcastExchange (133) - : +- CometFilter (132) - : +- CometScan parquet spark_catalog.default.date_dim (131) - +- * Sort (140) - +- ReusedExchange (139) + : : : : +- ReusedExchange (27) + : : : +- ReusedExchange (30) + : : +- * Sort (41) + : : +- Exchange (40) + : : +- * ColumnarToRow (39) + : : +- CometProject (38) + : : +- CometFilter (37) + : : +- CometScan parquet spark_catalog.default.store_returns (36) + : +- * Project (62) + : +- * SortMergeJoin LeftOuter (61) + : :- * Sort (54) + : : +- Exchange (53) + : : +- * ColumnarToRow (52) + : : +- CometProject (51) + : : +- CometBroadcastHashJoin (50) + : : :- CometProject (48) + : : : +- CometBroadcastHashJoin (47) + : : : :- CometFilter (45) + : : : : +- CometScan parquet spark_catalog.default.web_sales (44) + : : : +- ReusedExchange (46) + : : +- ReusedExchange (49) + : +- * Sort (60) + : +- Exchange (59) + : +- * ColumnarToRow (58) + : +- CometProject (57) + : +- CometFilter (56) + : +- CometScan parquet spark_catalog.default.web_returns (55) + +- * Sort (129) + +- Exchange (128) + +- * Filter (127) + +- * HashAggregate (126) + +- Exchange (125) + +- * HashAggregate (124) + +- * HashAggregate (123) + +- Exchange (122) + +- * HashAggregate (121) + +- Union (120) + :- * Project (89) + : +- * SortMergeJoin LeftOuter (88) + : :- * Sort (85) + : : +- Exchange (84) + : : +- * ColumnarToRow (83) + : : +- CometProject (82) + : : +- CometBroadcastHashJoin (81) + : : :- CometProject (77) + : : : +- CometBroadcastHashJoin (76) + : : : :- CometFilter (74) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (73) + : : : +- ReusedExchange (75) + : : +- CometBroadcastExchange (80) + : : +- CometFilter (79) + : : +- CometScan parquet spark_catalog.default.date_dim (78) + : +- * Sort (87) + : +- ReusedExchange (86) + :- * Project (104) + : +- * SortMergeJoin LeftOuter (103) + : :- * Sort (100) + : : +- Exchange (99) + : : +- * ColumnarToRow (98) + : : +- CometProject (97) + : : +- CometBroadcastHashJoin (96) + : : :- CometProject (94) + : : : +- CometBroadcastHashJoin (93) + : : : :- CometFilter (91) + : : : : +- CometScan parquet spark_catalog.default.store_sales (90) + : : : +- ReusedExchange (92) + : : +- ReusedExchange (95) + : +- * Sort (102) + : +- ReusedExchange (101) + +- * Project (119) + +- * SortMergeJoin LeftOuter (118) + :- * Sort (115) + : +- Exchange (114) + : +- * ColumnarToRow (113) + : +- CometProject (112) + : +- CometBroadcastHashJoin (111) + : :- CometProject (109) + : : +- CometBroadcastHashJoin (108) + : : :- CometFilter (106) + : : : +- CometScan parquet spark_catalog.default.web_sales (105) + : : +- ReusedExchange (107) + : +- ReusedExchange (110) + +- * Sort (117) + +- ReusedExchange (116) (1) Scan parquet spark_catalog.default.catalog_sales @@ -279,664 +256,536 @@ ReadSchema: struct +(27) ReusedExchange [Reuses operator id: 6] +Output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -(28) CometFilter -Input [6]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_category#32, i_manufact_id#33] -Condition : ((((((isnotnull(i_category#32) AND (i_category#32 = Books )) AND isnotnull(i_item_sk#28)) AND isnotnull(i_brand_id#29)) AND isnotnull(i_class_id#30)) AND isnotnull(i_category_id#31)) AND isnotnull(i_manufact_id#33)) +(28) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] +Right output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] +Arguments: [ss_item_sk#22], [i_item_sk#28], Inner (29) CometProject -Input [6]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_category#32, i_manufact_id#33] -Arguments: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#33], [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#33] +Input [10]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] +Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -(30) CometBroadcastExchange -Input [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#33] -Arguments: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#33] +(30) ReusedExchange [Reuses operator id: 11] +Output [2]: [d_date_sk#33, d_year#34] (31) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Right output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#33] -Arguments: [ss_item_sk#22], [i_item_sk#28], Inner +Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] +Right output [2]: [d_date_sk#33, d_year#34] +Arguments: [ss_sold_date_sk#26], [d_date_sk#33], Inner (32) CometProject -Input [10]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#33] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#33], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#33] - -(33) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#34, d_year#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [d_date_sk#34, d_year#35] -Condition : ((isnotnull(d_year#35) AND (d_year#35 = 2002)) AND isnotnull(d_date_sk#34)) - -(35) CometBroadcastExchange -Input [2]: [d_date_sk#34, d_year#35] -Arguments: [d_date_sk#34, d_year#35] - -(36) CometBroadcastHashJoin -Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#33] -Right output [2]: [d_date_sk#34, d_year#35] -Arguments: [ss_sold_date_sk#26], [d_date_sk#34], Inner - -(37) CometProject -Input [11]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#33, d_date_sk#34, d_year#35] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#33, d_year#35], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#33, d_year#35] +Input [11]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_date_sk#33, d_year#34] +Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -(38) ColumnarToRow [codegen id : 6] -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#33, d_year#35] +(33) ColumnarToRow [codegen id : 6] +Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -(39) Exchange -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#33, d_year#35] +(34) Exchange +Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] Arguments: hashpartitioning(ss_ticket_number#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(40) Sort [codegen id : 7] -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#33, d_year#35] +(35) Sort [codegen id : 7] +Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] Arguments: [ss_ticket_number#23 ASC NULLS FIRST, ss_item_sk#22 ASC NULLS FIRST], false, 0 -(41) Scan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#36, sr_ticket_number#37, sr_return_quantity#38, sr_return_amt#39, sr_returned_date_sk#40] +(36) Scan parquet spark_catalog.default.store_returns +Output [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(42) CometFilter -Input [5]: [sr_item_sk#36, sr_ticket_number#37, sr_return_quantity#38, sr_return_amt#39, sr_returned_date_sk#40] -Condition : (isnotnull(sr_ticket_number#37) AND isnotnull(sr_item_sk#36)) +(37) CometFilter +Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] +Condition : (isnotnull(sr_ticket_number#36) AND isnotnull(sr_item_sk#35)) -(43) CometProject -Input [5]: [sr_item_sk#36, sr_ticket_number#37, sr_return_quantity#38, sr_return_amt#39, sr_returned_date_sk#40] -Arguments: [sr_item_sk#36, sr_ticket_number#37, sr_return_quantity#38, sr_return_amt#39], [sr_item_sk#36, sr_ticket_number#37, sr_return_quantity#38, sr_return_amt#39] +(38) CometProject +Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] +Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -(44) ColumnarToRow [codegen id : 8] -Input [4]: [sr_item_sk#36, sr_ticket_number#37, sr_return_quantity#38, sr_return_amt#39] +(39) ColumnarToRow [codegen id : 8] +Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -(45) Exchange -Input [4]: [sr_item_sk#36, sr_ticket_number#37, sr_return_quantity#38, sr_return_amt#39] -Arguments: hashpartitioning(sr_ticket_number#37, sr_item_sk#36, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(40) Exchange +Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] +Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(46) Sort [codegen id : 9] -Input [4]: [sr_item_sk#36, sr_ticket_number#37, sr_return_quantity#38, sr_return_amt#39] -Arguments: [sr_ticket_number#37 ASC NULLS FIRST, sr_item_sk#36 ASC NULLS FIRST], false, 0 +(41) Sort [codegen id : 9] +Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] +Arguments: [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST], false, 0 -(47) SortMergeJoin [codegen id : 10] +(42) SortMergeJoin [codegen id : 10] Left keys [2]: [ss_ticket_number#23, ss_item_sk#22] -Right keys [2]: [sr_ticket_number#37, sr_item_sk#36] +Right keys [2]: [sr_ticket_number#36, sr_item_sk#35] Join type: LeftOuter Join condition: None -(48) Project [codegen id : 10] -Output [7]: [d_year#35, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#33, (ss_quantity#24 - coalesce(sr_return_quantity#38, 0)) AS sales_cnt#41, (ss_ext_sales_price#25 - coalesce(sr_return_amt#39, 0.00)) AS sales_amt#42] -Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#33, d_year#35, sr_item_sk#36, sr_ticket_number#37, sr_return_quantity#38, sr_return_amt#39] +(43) Project [codegen id : 10] +Output [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, (ss_quantity#24 - coalesce(sr_return_quantity#37, 0)) AS sales_cnt#40, (ss_ext_sales_price#25 - coalesce(sr_return_amt#38, 0.00)) AS sales_amt#41] +Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34, sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -(49) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#43, ws_order_number#44, ws_quantity#45, ws_ext_sales_price#46, ws_sold_date_sk#47] +(44) Scan parquet spark_catalog.default.web_sales +Output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#48)] +PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_sold_date_sk#46 IN dynamicpruning#47)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(50) CometFilter -Input [5]: [ws_item_sk#43, ws_order_number#44, ws_quantity#45, ws_ext_sales_price#46, ws_sold_date_sk#47] -Condition : isnotnull(ws_item_sk#43) +(45) CometFilter +Input [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] +Condition : isnotnull(ws_item_sk#42) -(51) Scan parquet spark_catalog.default.item -Output [6]: [i_item_sk#49, i_brand_id#50, i_class_id#51, i_category_id#52, i_category#53, i_manufact_id#54] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books ), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] -ReadSchema: struct - -(52) CometFilter -Input [6]: [i_item_sk#49, i_brand_id#50, i_class_id#51, i_category_id#52, i_category#53, i_manufact_id#54] -Condition : ((((((isnotnull(i_category#53) AND (i_category#53 = Books )) AND isnotnull(i_item_sk#49)) AND isnotnull(i_brand_id#50)) AND isnotnull(i_class_id#51)) AND isnotnull(i_category_id#52)) AND isnotnull(i_manufact_id#54)) - -(53) CometProject -Input [6]: [i_item_sk#49, i_brand_id#50, i_class_id#51, i_category_id#52, i_category#53, i_manufact_id#54] -Arguments: [i_item_sk#49, i_brand_id#50, i_class_id#51, i_category_id#52, i_manufact_id#54], [i_item_sk#49, i_brand_id#50, i_class_id#51, i_category_id#52, i_manufact_id#54] - -(54) CometBroadcastExchange -Input [5]: [i_item_sk#49, i_brand_id#50, i_class_id#51, i_category_id#52, i_manufact_id#54] -Arguments: [i_item_sk#49, i_brand_id#50, i_class_id#51, i_category_id#52, i_manufact_id#54] - -(55) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#43, ws_order_number#44, ws_quantity#45, ws_ext_sales_price#46, ws_sold_date_sk#47] -Right output [5]: [i_item_sk#49, i_brand_id#50, i_class_id#51, i_category_id#52, i_manufact_id#54] -Arguments: [ws_item_sk#43], [i_item_sk#49], Inner +(46) ReusedExchange [Reuses operator id: 6] +Output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -(56) CometProject -Input [10]: [ws_item_sk#43, ws_order_number#44, ws_quantity#45, ws_ext_sales_price#46, ws_sold_date_sk#47, i_item_sk#49, i_brand_id#50, i_class_id#51, i_category_id#52, i_manufact_id#54] -Arguments: [ws_item_sk#43, ws_order_number#44, ws_quantity#45, ws_ext_sales_price#46, ws_sold_date_sk#47, i_brand_id#50, i_class_id#51, i_category_id#52, i_manufact_id#54], [ws_item_sk#43, ws_order_number#44, ws_quantity#45, ws_ext_sales_price#46, ws_sold_date_sk#47, i_brand_id#50, i_class_id#51, i_category_id#52, i_manufact_id#54] +(47) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] +Right output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] +Arguments: [ws_item_sk#42], [i_item_sk#48], Inner -(57) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#55, d_year#56] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct +(48) CometProject +Input [10]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] +Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -(58) CometFilter -Input [2]: [d_date_sk#55, d_year#56] -Condition : ((isnotnull(d_year#56) AND (d_year#56 = 2002)) AND isnotnull(d_date_sk#55)) +(49) ReusedExchange [Reuses operator id: 11] +Output [2]: [d_date_sk#53, d_year#54] -(59) CometBroadcastExchange -Input [2]: [d_date_sk#55, d_year#56] -Arguments: [d_date_sk#55, d_year#56] +(50) CometBroadcastHashJoin +Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] +Right output [2]: [d_date_sk#53, d_year#54] +Arguments: [ws_sold_date_sk#46], [d_date_sk#53], Inner -(60) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#43, ws_order_number#44, ws_quantity#45, ws_ext_sales_price#46, ws_sold_date_sk#47, i_brand_id#50, i_class_id#51, i_category_id#52, i_manufact_id#54] -Right output [2]: [d_date_sk#55, d_year#56] -Arguments: [ws_sold_date_sk#47], [d_date_sk#55], Inner +(51) CometProject +Input [11]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_date_sk#53, d_year#54] +Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -(61) CometProject -Input [11]: [ws_item_sk#43, ws_order_number#44, ws_quantity#45, ws_ext_sales_price#46, ws_sold_date_sk#47, i_brand_id#50, i_class_id#51, i_category_id#52, i_manufact_id#54, d_date_sk#55, d_year#56] -Arguments: [ws_item_sk#43, ws_order_number#44, ws_quantity#45, ws_ext_sales_price#46, i_brand_id#50, i_class_id#51, i_category_id#52, i_manufact_id#54, d_year#56], [ws_item_sk#43, ws_order_number#44, ws_quantity#45, ws_ext_sales_price#46, i_brand_id#50, i_class_id#51, i_category_id#52, i_manufact_id#54, d_year#56] +(52) ColumnarToRow [codegen id : 11] +Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -(62) ColumnarToRow [codegen id : 11] -Input [9]: [ws_item_sk#43, ws_order_number#44, ws_quantity#45, ws_ext_sales_price#46, i_brand_id#50, i_class_id#51, i_category_id#52, i_manufact_id#54, d_year#56] +(53) Exchange +Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] +Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(63) Exchange -Input [9]: [ws_item_sk#43, ws_order_number#44, ws_quantity#45, ws_ext_sales_price#46, i_brand_id#50, i_class_id#51, i_category_id#52, i_manufact_id#54, d_year#56] -Arguments: hashpartitioning(ws_order_number#44, ws_item_sk#43, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(54) Sort [codegen id : 12] +Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] +Arguments: [ws_order_number#43 ASC NULLS FIRST, ws_item_sk#42 ASC NULLS FIRST], false, 0 -(64) Sort [codegen id : 12] -Input [9]: [ws_item_sk#43, ws_order_number#44, ws_quantity#45, ws_ext_sales_price#46, i_brand_id#50, i_class_id#51, i_category_id#52, i_manufact_id#54, d_year#56] -Arguments: [ws_order_number#44 ASC NULLS FIRST, ws_item_sk#43 ASC NULLS FIRST], false, 0 - -(65) Scan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#57, wr_order_number#58, wr_return_quantity#59, wr_return_amt#60, wr_returned_date_sk#61] +(55) Scan parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct -(66) CometFilter -Input [5]: [wr_item_sk#57, wr_order_number#58, wr_return_quantity#59, wr_return_amt#60, wr_returned_date_sk#61] -Condition : (isnotnull(wr_order_number#58) AND isnotnull(wr_item_sk#57)) +(56) CometFilter +Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] +Condition : (isnotnull(wr_order_number#56) AND isnotnull(wr_item_sk#55)) -(67) CometProject -Input [5]: [wr_item_sk#57, wr_order_number#58, wr_return_quantity#59, wr_return_amt#60, wr_returned_date_sk#61] -Arguments: [wr_item_sk#57, wr_order_number#58, wr_return_quantity#59, wr_return_amt#60], [wr_item_sk#57, wr_order_number#58, wr_return_quantity#59, wr_return_amt#60] +(57) CometProject +Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] +Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -(68) ColumnarToRow [codegen id : 13] -Input [4]: [wr_item_sk#57, wr_order_number#58, wr_return_quantity#59, wr_return_amt#60] +(58) ColumnarToRow [codegen id : 13] +Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -(69) Exchange -Input [4]: [wr_item_sk#57, wr_order_number#58, wr_return_quantity#59, wr_return_amt#60] -Arguments: hashpartitioning(wr_order_number#58, wr_item_sk#57, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(59) Exchange +Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] +Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(70) Sort [codegen id : 14] -Input [4]: [wr_item_sk#57, wr_order_number#58, wr_return_quantity#59, wr_return_amt#60] -Arguments: [wr_order_number#58 ASC NULLS FIRST, wr_item_sk#57 ASC NULLS FIRST], false, 0 +(60) Sort [codegen id : 14] +Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] +Arguments: [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST], false, 0 -(71) SortMergeJoin [codegen id : 15] -Left keys [2]: [ws_order_number#44, ws_item_sk#43] -Right keys [2]: [wr_order_number#58, wr_item_sk#57] +(61) SortMergeJoin [codegen id : 15] +Left keys [2]: [ws_order_number#43, ws_item_sk#42] +Right keys [2]: [wr_order_number#56, wr_item_sk#55] Join type: LeftOuter Join condition: None -(72) Project [codegen id : 15] -Output [7]: [d_year#56, i_brand_id#50, i_class_id#51, i_category_id#52, i_manufact_id#54, (ws_quantity#45 - coalesce(wr_return_quantity#59, 0)) AS sales_cnt#62, (ws_ext_sales_price#46 - coalesce(wr_return_amt#60, 0.00)) AS sales_amt#63] -Input [13]: [ws_item_sk#43, ws_order_number#44, ws_quantity#45, ws_ext_sales_price#46, i_brand_id#50, i_class_id#51, i_category_id#52, i_manufact_id#54, d_year#56, wr_item_sk#57, wr_order_number#58, wr_return_quantity#59, wr_return_amt#60] +(62) Project [codegen id : 15] +Output [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, (ws_quantity#44 - coalesce(wr_return_quantity#57, 0)) AS sales_cnt#60, (ws_ext_sales_price#45 - coalesce(wr_return_amt#58, 0.00)) AS sales_amt#61] +Input [13]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54, wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -(73) Union +(63) Union -(74) HashAggregate [codegen id : 16] +(64) HashAggregate [codegen id : 16] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Functions: [] Aggregate Attributes: [] Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -(75) Exchange +(65) Exchange Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(76) HashAggregate [codegen id : 17] +(66) HashAggregate [codegen id : 17] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Functions: [] Aggregate Attributes: [] Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -(77) HashAggregate [codegen id : 17] +(67) HashAggregate [codegen id : 17] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] -Aggregate Attributes [2]: [sum#64, sum#65] -Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#66, sum#67] +Aggregate Attributes [2]: [sum#62, sum#63] +Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] -(78) Exchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#66, sum#67] +(68) Exchange +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(79) HashAggregate [codegen id : 18] -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#66, sum#67] +(69) HashAggregate [codegen id : 18] +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] -Aggregate Attributes [2]: [sum(sales_cnt#20)#68, sum(UnscaledValue(sales_amt#21))#69] -Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#20)#68 AS sales_cnt#70, MakeDecimal(sum(UnscaledValue(sales_amt#21))#69,18,2) AS sales_amt#71] +Aggregate Attributes [2]: [sum(sales_cnt#20)#66, sum(UnscaledValue(sales_amt#21))#67] +Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#20)#66 AS sales_cnt#68, MakeDecimal(sum(UnscaledValue(sales_amt#21))#67,18,2) AS sales_amt#69] -(80) Filter [codegen id : 18] -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#70, sales_amt#71] -Condition : isnotnull(sales_cnt#70) +(70) Filter [codegen id : 18] +Input [7]: [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] +Condition : isnotnull(sales_cnt#68) -(81) Exchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#70, sales_amt#71] +(71) Exchange +Input [7]: [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] Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(82) Sort [codegen id : 19] -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#70, sales_amt#71] +(72) Sort [codegen id : 19] +Input [7]: [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] Arguments: [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST], false, 0 -(83) Scan parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#72, cs_order_number#73, cs_quantity#74, cs_ext_sales_price#75, cs_sold_date_sk#76] +(73) Scan parquet spark_catalog.default.catalog_sales +Output [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#76), dynamicpruningexpression(cs_sold_date_sk#76 IN dynamicpruning#77)] +PartitionFilters: [isnotnull(cs_sold_date_sk#74), dynamicpruningexpression(cs_sold_date_sk#74 IN dynamicpruning#75)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(84) CometFilter -Input [5]: [cs_item_sk#72, cs_order_number#73, cs_quantity#74, cs_ext_sales_price#75, cs_sold_date_sk#76] -Condition : isnotnull(cs_item_sk#72) +(74) CometFilter +Input [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] +Condition : isnotnull(cs_item_sk#70) -(85) Scan parquet spark_catalog.default.item -Output [6]: [i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81, i_category#82, i_manufact_id#83] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books ), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] -ReadSchema: struct - -(86) CometFilter -Input [6]: [i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81, i_category#82, i_manufact_id#83] -Condition : ((((((isnotnull(i_category#82) AND (i_category#82 = Books )) AND isnotnull(i_item_sk#78)) AND isnotnull(i_brand_id#79)) AND isnotnull(i_class_id#80)) AND isnotnull(i_category_id#81)) AND isnotnull(i_manufact_id#83)) - -(87) CometProject -Input [6]: [i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81, i_category#82, i_manufact_id#83] -Arguments: [i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83], [i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83] - -(88) CometBroadcastExchange -Input [5]: [i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83] -Arguments: [i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83] +(75) ReusedExchange [Reuses operator id: 6] +Output [5]: [i_item_sk#76, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] -(89) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#72, cs_order_number#73, cs_quantity#74, cs_ext_sales_price#75, cs_sold_date_sk#76] -Right output [5]: [i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83] -Arguments: [cs_item_sk#72], [i_item_sk#78], Inner +(76) CometBroadcastHashJoin +Left output [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] +Right output [5]: [i_item_sk#76, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +Arguments: [cs_item_sk#70], [i_item_sk#76], Inner -(90) CometProject -Input [10]: [cs_item_sk#72, cs_order_number#73, cs_quantity#74, cs_ext_sales_price#75, cs_sold_date_sk#76, i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83] -Arguments: [cs_item_sk#72, cs_order_number#73, cs_quantity#74, cs_ext_sales_price#75, cs_sold_date_sk#76, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83], [cs_item_sk#72, cs_order_number#73, cs_quantity#74, cs_ext_sales_price#75, cs_sold_date_sk#76, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83] +(77) CometProject +Input [10]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_item_sk#76, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +Arguments: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80], [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] -(91) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#84, d_year#85] +(78) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#81, d_year#82] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(92) CometFilter -Input [2]: [d_date_sk#84, d_year#85] -Condition : ((isnotnull(d_year#85) AND (d_year#85 = 2001)) AND isnotnull(d_date_sk#84)) +(79) 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)) -(93) CometBroadcastExchange -Input [2]: [d_date_sk#84, d_year#85] -Arguments: [d_date_sk#84, d_year#85] +(80) CometBroadcastExchange +Input [2]: [d_date_sk#81, d_year#82] +Arguments: [d_date_sk#81, d_year#82] -(94) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#72, cs_order_number#73, cs_quantity#74, cs_ext_sales_price#75, cs_sold_date_sk#76, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83] -Right output [2]: [d_date_sk#84, d_year#85] -Arguments: [cs_sold_date_sk#76], [d_date_sk#84], Inner +(81) CometBroadcastHashJoin +Left output [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +Right output [2]: [d_date_sk#81, d_year#82] +Arguments: [cs_sold_date_sk#74], [d_date_sk#81], Inner -(95) CometProject -Input [11]: [cs_item_sk#72, cs_order_number#73, cs_quantity#74, cs_ext_sales_price#75, cs_sold_date_sk#76, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, d_date_sk#84, d_year#85] -Arguments: [cs_item_sk#72, cs_order_number#73, cs_quantity#74, cs_ext_sales_price#75, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, d_year#85], [cs_item_sk#72, cs_order_number#73, cs_quantity#74, cs_ext_sales_price#75, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, d_year#85] +(82) CometProject +Input [11]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_date_sk#81, d_year#82] +Arguments: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82], [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] -(96) ColumnarToRow [codegen id : 20] -Input [9]: [cs_item_sk#72, cs_order_number#73, cs_quantity#74, cs_ext_sales_price#75, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, d_year#85] +(83) ColumnarToRow [codegen id : 20] +Input [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] -(97) Exchange -Input [9]: [cs_item_sk#72, cs_order_number#73, cs_quantity#74, cs_ext_sales_price#75, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, d_year#85] -Arguments: hashpartitioning(cs_order_number#73, cs_item_sk#72, 5), ENSURE_REQUIREMENTS, [plan_id=10] +(84) Exchange +Input [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] +Arguments: hashpartitioning(cs_order_number#71, cs_item_sk#70, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(98) Sort [codegen id : 21] -Input [9]: [cs_item_sk#72, cs_order_number#73, cs_quantity#74, cs_ext_sales_price#75, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, d_year#85] -Arguments: [cs_order_number#73 ASC NULLS FIRST, cs_item_sk#72 ASC NULLS FIRST], false, 0 +(85) Sort [codegen id : 21] +Input [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] +Arguments: [cs_order_number#71 ASC NULLS FIRST, cs_item_sk#70 ASC NULLS FIRST], false, 0 -(99) ReusedExchange [Reuses operator id: 21] -Output [4]: [cr_item_sk#86, cr_order_number#87, cr_return_quantity#88, cr_return_amount#89] +(86) ReusedExchange [Reuses operator id: 21] +Output [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] -(100) Sort [codegen id : 23] -Input [4]: [cr_item_sk#86, cr_order_number#87, cr_return_quantity#88, cr_return_amount#89] -Arguments: [cr_order_number#87 ASC NULLS FIRST, cr_item_sk#86 ASC NULLS FIRST], false, 0 +(87) Sort [codegen id : 23] +Input [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] +Arguments: [cr_order_number#84 ASC NULLS FIRST, cr_item_sk#83 ASC NULLS FIRST], false, 0 -(101) SortMergeJoin [codegen id : 24] -Left keys [2]: [cs_order_number#73, cs_item_sk#72] -Right keys [2]: [cr_order_number#87, cr_item_sk#86] +(88) SortMergeJoin [codegen id : 24] +Left keys [2]: [cs_order_number#71, cs_item_sk#70] +Right keys [2]: [cr_order_number#84, cr_item_sk#83] Join type: LeftOuter Join condition: None -(102) Project [codegen id : 24] -Output [7]: [d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, (cs_quantity#74 - coalesce(cr_return_quantity#88, 0)) AS sales_cnt#20, (cs_ext_sales_price#75 - coalesce(cr_return_amount#89, 0.00)) AS sales_amt#21] -Input [13]: [cs_item_sk#72, cs_order_number#73, cs_quantity#74, cs_ext_sales_price#75, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, d_year#85, cr_item_sk#86, cr_order_number#87, cr_return_quantity#88, cr_return_amount#89] +(89) Project [codegen id : 24] +Output [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, (cs_quantity#72 - coalesce(cr_return_quantity#85, 0)) AS sales_cnt#20, (cs_ext_sales_price#73 - coalesce(cr_return_amount#86, 0.00)) AS sales_amt#21] +Input [13]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82, cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] -(103) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#90, ss_ticket_number#91, ss_quantity#92, ss_ext_sales_price#93, ss_sold_date_sk#94] +(90) Scan parquet spark_catalog.default.store_sales +Output [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#94), dynamicpruningexpression(ss_sold_date_sk#94 IN dynamicpruning#95)] +PartitionFilters: [isnotnull(ss_sold_date_sk#91), dynamicpruningexpression(ss_sold_date_sk#91 IN dynamicpruning#92)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(104) CometFilter -Input [5]: [ss_item_sk#90, ss_ticket_number#91, ss_quantity#92, ss_ext_sales_price#93, ss_sold_date_sk#94] -Condition : isnotnull(ss_item_sk#90) - -(105) Scan parquet spark_catalog.default.item -Output [6]: [i_item_sk#96, i_brand_id#97, i_class_id#98, i_category_id#99, i_category#100, i_manufact_id#101] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books ), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] -ReadSchema: struct - -(106) CometFilter -Input [6]: [i_item_sk#96, i_brand_id#97, i_class_id#98, i_category_id#99, i_category#100, i_manufact_id#101] -Condition : ((((((isnotnull(i_category#100) AND (i_category#100 = Books )) AND isnotnull(i_item_sk#96)) AND isnotnull(i_brand_id#97)) AND isnotnull(i_class_id#98)) AND isnotnull(i_category_id#99)) AND isnotnull(i_manufact_id#101)) - -(107) CometProject -Input [6]: [i_item_sk#96, i_brand_id#97, i_class_id#98, i_category_id#99, i_category#100, i_manufact_id#101] -Arguments: [i_item_sk#96, i_brand_id#97, i_class_id#98, i_category_id#99, i_manufact_id#101], [i_item_sk#96, i_brand_id#97, i_class_id#98, i_category_id#99, i_manufact_id#101] +(91) 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) -(108) CometBroadcastExchange -Input [5]: [i_item_sk#96, i_brand_id#97, i_class_id#98, i_category_id#99, i_manufact_id#101] -Arguments: [i_item_sk#96, i_brand_id#97, i_class_id#98, i_category_id#99, i_manufact_id#101] +(92) ReusedExchange [Reuses operator id: 6] +Output [5]: [i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] -(109) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#90, ss_ticket_number#91, ss_quantity#92, ss_ext_sales_price#93, ss_sold_date_sk#94] -Right output [5]: [i_item_sk#96, i_brand_id#97, i_class_id#98, i_category_id#99, i_manufact_id#101] -Arguments: [ss_item_sk#90], [i_item_sk#96], Inner - -(110) CometProject -Input [10]: [ss_item_sk#90, ss_ticket_number#91, ss_quantity#92, ss_ext_sales_price#93, ss_sold_date_sk#94, i_item_sk#96, i_brand_id#97, i_class_id#98, i_category_id#99, i_manufact_id#101] -Arguments: [ss_item_sk#90, ss_ticket_number#91, ss_quantity#92, ss_ext_sales_price#93, ss_sold_date_sk#94, i_brand_id#97, i_class_id#98, i_category_id#99, i_manufact_id#101], [ss_item_sk#90, ss_ticket_number#91, ss_quantity#92, ss_ext_sales_price#93, ss_sold_date_sk#94, i_brand_id#97, i_class_id#98, i_category_id#99, i_manufact_id#101] - -(111) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#102, d_year#103] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct +(93) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] +Right output [5]: [i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] +Arguments: [ss_item_sk#87], [i_item_sk#93], Inner -(112) CometFilter -Input [2]: [d_date_sk#102, d_year#103] -Condition : ((isnotnull(d_year#103) AND (d_year#103 = 2001)) AND isnotnull(d_date_sk#102)) +(94) CometProject +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] +Arguments: [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], [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] -(113) CometBroadcastExchange -Input [2]: [d_date_sk#102, d_year#103] -Arguments: [d_date_sk#102, d_year#103] +(95) ReusedExchange [Reuses operator id: 80] +Output [2]: [d_date_sk#98, d_year#99] -(114) CometBroadcastHashJoin -Left output [9]: [ss_item_sk#90, ss_ticket_number#91, ss_quantity#92, ss_ext_sales_price#93, ss_sold_date_sk#94, i_brand_id#97, i_class_id#98, i_category_id#99, i_manufact_id#101] -Right output [2]: [d_date_sk#102, d_year#103] -Arguments: [ss_sold_date_sk#94], [d_date_sk#102], Inner +(96) CometBroadcastHashJoin +Left 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] +Right output [2]: [d_date_sk#98, d_year#99] +Arguments: [ss_sold_date_sk#91], [d_date_sk#98], Inner -(115) CometProject -Input [11]: [ss_item_sk#90, ss_ticket_number#91, ss_quantity#92, ss_ext_sales_price#93, ss_sold_date_sk#94, i_brand_id#97, i_class_id#98, i_category_id#99, i_manufact_id#101, d_date_sk#102, d_year#103] -Arguments: [ss_item_sk#90, ss_ticket_number#91, ss_quantity#92, ss_ext_sales_price#93, i_brand_id#97, i_class_id#98, i_category_id#99, i_manufact_id#101, d_year#103], [ss_item_sk#90, ss_ticket_number#91, ss_quantity#92, ss_ext_sales_price#93, i_brand_id#97, i_class_id#98, i_category_id#99, i_manufact_id#101, d_year#103] +(97) CometProject +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] +Arguments: [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], [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] -(116) ColumnarToRow [codegen id : 25] -Input [9]: [ss_item_sk#90, ss_ticket_number#91, ss_quantity#92, ss_ext_sales_price#93, i_brand_id#97, i_class_id#98, i_category_id#99, i_manufact_id#101, d_year#103] +(98) ColumnarToRow [codegen id : 25] +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] -(117) Exchange -Input [9]: [ss_item_sk#90, ss_ticket_number#91, ss_quantity#92, ss_ext_sales_price#93, i_brand_id#97, i_class_id#98, i_category_id#99, i_manufact_id#101, d_year#103] -Arguments: hashpartitioning(ss_ticket_number#91, ss_item_sk#90, 5), ENSURE_REQUIREMENTS, [plan_id=11] +(99) 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=11] -(118) Sort [codegen id : 26] -Input [9]: [ss_item_sk#90, ss_ticket_number#91, ss_quantity#92, ss_ext_sales_price#93, i_brand_id#97, i_class_id#98, i_category_id#99, i_manufact_id#101, d_year#103] -Arguments: [ss_ticket_number#91 ASC NULLS FIRST, ss_item_sk#90 ASC NULLS FIRST], false, 0 +(100) Sort [codegen id : 26] +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 -(119) ReusedExchange [Reuses operator id: 45] -Output [4]: [sr_item_sk#104, sr_ticket_number#105, sr_return_quantity#106, sr_return_amt#107] +(101) ReusedExchange [Reuses operator id: 40] +Output [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] -(120) Sort [codegen id : 28] -Input [4]: [sr_item_sk#104, sr_ticket_number#105, sr_return_quantity#106, sr_return_amt#107] -Arguments: [sr_ticket_number#105 ASC NULLS FIRST, sr_item_sk#104 ASC NULLS FIRST], false, 0 +(102) Sort [codegen id : 28] +Input [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] +Arguments: [sr_ticket_number#101 ASC NULLS FIRST, sr_item_sk#100 ASC NULLS FIRST], false, 0 -(121) SortMergeJoin [codegen id : 29] -Left keys [2]: [ss_ticket_number#91, ss_item_sk#90] -Right keys [2]: [sr_ticket_number#105, sr_item_sk#104] +(103) SortMergeJoin [codegen id : 29] +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 -(122) Project [codegen id : 29] -Output [7]: [d_year#103, i_brand_id#97, i_class_id#98, i_category_id#99, i_manufact_id#101, (ss_quantity#92 - coalesce(sr_return_quantity#106, 0)) AS sales_cnt#41, (ss_ext_sales_price#93 - coalesce(sr_return_amt#107, 0.00)) AS sales_amt#42] -Input [13]: [ss_item_sk#90, ss_ticket_number#91, ss_quantity#92, ss_ext_sales_price#93, i_brand_id#97, i_class_id#98, i_category_id#99, i_manufact_id#101, d_year#103, sr_item_sk#104, sr_ticket_number#105, sr_return_quantity#106, sr_return_amt#107] +(104) Project [codegen id : 29] +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] -(123) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112] +(105) Scan parquet spark_catalog.default.web_sales +Output [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#112), dynamicpruningexpression(ws_sold_date_sk#112 IN dynamicpruning#113)] +PartitionFilters: [isnotnull(ws_sold_date_sk#108), dynamicpruningexpression(ws_sold_date_sk#108 IN dynamicpruning#109)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(124) CometFilter -Input [5]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112] -Condition : isnotnull(ws_item_sk#108) - -(125) Scan parquet spark_catalog.default.item -Output [6]: [i_item_sk#114, i_brand_id#115, i_class_id#116, i_category_id#117, i_category#118, i_manufact_id#119] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books ), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] -ReadSchema: struct - -(126) CometFilter -Input [6]: [i_item_sk#114, i_brand_id#115, i_class_id#116, i_category_id#117, i_category#118, i_manufact_id#119] -Condition : ((((((isnotnull(i_category#118) AND (i_category#118 = Books )) AND isnotnull(i_item_sk#114)) AND isnotnull(i_brand_id#115)) AND isnotnull(i_class_id#116)) AND isnotnull(i_category_id#117)) AND isnotnull(i_manufact_id#119)) - -(127) CometProject -Input [6]: [i_item_sk#114, i_brand_id#115, i_class_id#116, i_category_id#117, i_category#118, i_manufact_id#119] -Arguments: [i_item_sk#114, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#119], [i_item_sk#114, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#119] - -(128) CometBroadcastExchange -Input [5]: [i_item_sk#114, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#119] -Arguments: [i_item_sk#114, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#119] - -(129) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112] -Right output [5]: [i_item_sk#114, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#119] -Arguments: [ws_item_sk#108], [i_item_sk#114], Inner +(106) 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) -(130) CometProject -Input [10]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112, i_item_sk#114, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#119] -Arguments: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#119], [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#119] +(107) ReusedExchange [Reuses operator id: 6] +Output [5]: [i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -(131) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#120, d_year#121] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct +(108) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] +Right output [5]: [i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] +Arguments: [ws_item_sk#104], [i_item_sk#110], Inner -(132) CometFilter -Input [2]: [d_date_sk#120, d_year#121] -Condition : ((isnotnull(d_year#121) AND (d_year#121 = 2001)) AND isnotnull(d_date_sk#120)) +(109) CometProject +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] +Arguments: [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], [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] -(133) CometBroadcastExchange -Input [2]: [d_date_sk#120, d_year#121] -Arguments: [d_date_sk#120, d_year#121] +(110) ReusedExchange [Reuses operator id: 80] +Output [2]: [d_date_sk#115, d_year#116] -(134) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#119] -Right output [2]: [d_date_sk#120, d_year#121] -Arguments: [ws_sold_date_sk#112], [d_date_sk#120], Inner +(111) CometBroadcastHashJoin +Left 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] +Right output [2]: [d_date_sk#115, d_year#116] +Arguments: [ws_sold_date_sk#108], [d_date_sk#115], Inner -(135) CometProject -Input [11]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#119, d_date_sk#120, d_year#121] -Arguments: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#119, d_year#121], [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#119, d_year#121] +(112) CometProject +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] +Arguments: [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], [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] -(136) ColumnarToRow [codegen id : 30] -Input [9]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#119, d_year#121] +(113) ColumnarToRow [codegen id : 30] +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] -(137) Exchange -Input [9]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#119, d_year#121] -Arguments: hashpartitioning(ws_order_number#109, ws_item_sk#108, 5), ENSURE_REQUIREMENTS, [plan_id=12] +(114) 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=12] -(138) Sort [codegen id : 31] -Input [9]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#119, d_year#121] -Arguments: [ws_order_number#109 ASC NULLS FIRST, ws_item_sk#108 ASC NULLS FIRST], false, 0 +(115) Sort [codegen id : 31] +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 -(139) ReusedExchange [Reuses operator id: 69] -Output [4]: [wr_item_sk#122, wr_order_number#123, wr_return_quantity#124, wr_return_amt#125] +(116) ReusedExchange [Reuses operator id: 59] +Output [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] -(140) Sort [codegen id : 33] -Input [4]: [wr_item_sk#122, wr_order_number#123, wr_return_quantity#124, wr_return_amt#125] -Arguments: [wr_order_number#123 ASC NULLS FIRST, wr_item_sk#122 ASC NULLS FIRST], false, 0 +(117) Sort [codegen id : 33] +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 -(141) SortMergeJoin [codegen id : 34] -Left keys [2]: [ws_order_number#109, ws_item_sk#108] -Right keys [2]: [wr_order_number#123, wr_item_sk#122] +(118) SortMergeJoin [codegen id : 34] +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 -(142) Project [codegen id : 34] -Output [7]: [d_year#121, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#119, (ws_quantity#110 - coalesce(wr_return_quantity#124, 0)) AS sales_cnt#62, (ws_ext_sales_price#111 - coalesce(wr_return_amt#125, 0.00)) AS sales_amt#63] -Input [13]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#119, d_year#121, wr_item_sk#122, wr_order_number#123, wr_return_quantity#124, wr_return_amt#125] +(119) Project [codegen id : 34] +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] -(143) Union +(120) Union -(144) HashAggregate [codegen id : 35] -Input [7]: [d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, sales_cnt#20, sales_amt#21] +(121) HashAggregate [codegen id : 35] +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#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, sales_cnt#20, sales_amt#21] +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] -(145) Exchange -Input [7]: [d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, [plan_id=13] +(122) 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=13] -(146) HashAggregate [codegen id : 36] -Input [7]: [d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, sales_cnt#20, sales_amt#21] +(123) HashAggregate [codegen id : 36] +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#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, sales_cnt#20, sales_amt#21] +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] -(147) HashAggregate [codegen id : 36] -Input [7]: [d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83] +(124) HashAggregate [codegen id : 36] +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#64, sum#126] -Results [7]: [d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, sum#66, sum#127] +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] -(148) Exchange -Input [7]: [d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, sum#66, sum#127] -Arguments: hashpartitioning(d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, 5), ENSURE_REQUIREMENTS, [plan_id=14] +(125) 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=14] -(149) HashAggregate [codegen id : 37] -Input [7]: [d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, sum#66, sum#127] -Keys [5]: [d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83] +(126) HashAggregate [codegen id : 37] +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)#68, sum(UnscaledValue(sales_amt#21))#69] -Results [7]: [d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, sum(sales_cnt#20)#68 AS sales_cnt#128, MakeDecimal(sum(UnscaledValue(sales_amt#21))#69,18,2) AS sales_amt#129] +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] -(150) Filter [codegen id : 37] -Input [7]: [d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, sales_cnt#128, sales_amt#129] -Condition : isnotnull(sales_cnt#128) +(127) Filter [codegen id : 37] +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) -(151) Exchange -Input [7]: [d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, sales_cnt#128, sales_amt#129] -Arguments: hashpartitioning(i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, 5), ENSURE_REQUIREMENTS, [plan_id=15] +(128) 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=15] -(152) Sort [codegen id : 38] -Input [7]: [d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, sales_cnt#128, sales_amt#129] -Arguments: [i_brand_id#79 ASC NULLS FIRST, i_class_id#80 ASC NULLS FIRST, i_category_id#81 ASC NULLS FIRST, i_manufact_id#83 ASC NULLS FIRST], false, 0 +(129) Sort [codegen id : 38] +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 -(153) SortMergeJoin [codegen id : 39] +(130) SortMergeJoin [codegen id : 39] 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#79, i_class_id#80, i_category_id#81, i_manufact_id#83] +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#70 as decimal(17,2)) / cast(sales_cnt#128 as decimal(17,2))) < 0.90000000000000000000) +Join condition: ((cast(sales_cnt#68 as decimal(17,2)) / cast(sales_cnt#123 as decimal(17,2))) < 0.90000000000000000000) -(154) Project [codegen id : 39] -Output [10]: [d_year#85 AS prev_year#130, d_year#14 AS year#131, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#128 AS prev_yr_cnt#132, sales_cnt#70 AS curr_yr_cnt#133, (sales_cnt#70 - sales_cnt#128) AS sales_cnt_diff#134, (sales_amt#71 - sales_amt#129) AS sales_amt_diff#135] -Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#70, sales_amt#71, d_year#85, i_brand_id#79, i_class_id#80, i_category_id#81, i_manufact_id#83, sales_cnt#128, sales_amt#129] +(131) Project [codegen id : 39] +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] -(155) TakeOrderedAndProject -Input [10]: [prev_year#130, year#131, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#132, curr_yr_cnt#133, sales_cnt_diff#134, sales_amt_diff#135] -Arguments: 100, [sales_cnt_diff#134 ASC NULLS FIRST, sales_amt_diff#135 ASC NULLS FIRST], [prev_year#130, year#131, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#132, curr_yr_cnt#133, sales_cnt_diff#134, sales_amt_diff#135] +(132) 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 (159) -+- * ColumnarToRow (158) - +- CometFilter (157) - +- CometScan parquet spark_catalog.default.date_dim (156) +BroadcastExchange (136) ++- * ColumnarToRow (135) + +- CometFilter (134) + +- CometScan parquet spark_catalog.default.date_dim (133) -(156) Scan parquet spark_catalog.default.date_dim +(133) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#13, d_year#14] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(157) CometFilter +(134) 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)) -(158) ColumnarToRow [codegen id : 1] +(135) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] -(159) BroadcastExchange +(136) BroadcastExchange Input [2]: [d_date_sk#13, d_year#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 49 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 83 Hosting Expression = cs_sold_date_sk#76 IN dynamicpruning#77 -BroadcastExchange (163) -+- * ColumnarToRow (162) - +- CometFilter (161) - +- CometScan parquet spark_catalog.default.date_dim (160) +Subquery:4 Hosting operator id = 73 Hosting Expression = cs_sold_date_sk#74 IN dynamicpruning#75 +BroadcastExchange (140) ++- * ColumnarToRow (139) + +- CometFilter (138) + +- CometScan parquet spark_catalog.default.date_dim (137) -(160) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#84, d_year#85] +(137) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#81, d_year#82] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(161) CometFilter -Input [2]: [d_date_sk#84, d_year#85] -Condition : ((isnotnull(d_year#85) AND (d_year#85 = 2001)) AND isnotnull(d_date_sk#84)) +(138) 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)) -(162) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#84, d_year#85] +(139) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#81, d_year#82] -(163) BroadcastExchange -Input [2]: [d_date_sk#84, d_year#85] +(140) BroadcastExchange +Input [2]: [d_date_sk#81, d_year#82] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] -Subquery:5 Hosting operator id = 103 Hosting Expression = ss_sold_date_sk#94 IN dynamicpruning#77 +Subquery:5 Hosting operator id = 90 Hosting Expression = ss_sold_date_sk#91 IN dynamicpruning#75 -Subquery:6 Hosting operator id = 123 Hosting Expression = ws_sold_date_sk#112 IN dynamicpruning#77 +Subquery:6 Hosting operator id = 105 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 8ca23509e..fb78d64b1 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 @@ -81,18 +81,13 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #10 - CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange #11 - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #7 InputAdapter WholeStageCodegen (9) Sort [sr_ticket_number,sr_item_sk] InputAdapter - Exchange [sr_ticket_number,sr_item_sk] #12 + Exchange [sr_ticket_number,sr_item_sk] #10 WholeStageCodegen (8) ColumnarToRow InputAdapter @@ -106,7 +101,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i WholeStageCodegen (12) Sort [ws_order_number,ws_item_sk] InputAdapter - Exchange [ws_order_number,ws_item_sk] #13 + Exchange [ws_order_number,ws_item_sk] #11 WholeStageCodegen (11) ColumnarToRow InputAdapter @@ -117,18 +112,13 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #14 - CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange #15 - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #7 InputAdapter WholeStageCodegen (14) Sort [wr_order_number,wr_item_sk] InputAdapter - Exchange [wr_order_number,wr_item_sk] #16 + Exchange [wr_order_number,wr_item_sk] #12 WholeStageCodegen (13) ColumnarToRow InputAdapter @@ -139,17 +129,17 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i WholeStageCodegen (38) 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] #17 + Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 WholeStageCodegen (37) 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] #18 + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 WholeStageCodegen (36) 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] #19 + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 WholeStageCodegen (35) HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter @@ -161,7 +151,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i WholeStageCodegen (21) Sort [cs_order_number,cs_item_sk] InputAdapter - Exchange [cs_order_number,cs_item_sk] #20 + Exchange [cs_order_number,cs_item_sk] #16 WholeStageCodegen (20) ColumnarToRow InputAdapter @@ -172,17 +162,14 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #21 + BroadcastExchange #17 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #22 - CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange #23 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + CometBroadcastExchange #18 CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter @@ -197,7 +184,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i WholeStageCodegen (26) Sort [ss_ticket_number,ss_item_sk] InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #24 + Exchange [ss_ticket_number,ss_item_sk] #19 WholeStageCodegen (25) ColumnarToRow InputAdapter @@ -208,18 +195,13 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #25 - CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange #26 - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #18 InputAdapter WholeStageCodegen (28) Sort [sr_ticket_number,sr_item_sk] InputAdapter - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #12 + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #10 WholeStageCodegen (34) 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] @@ -227,7 +209,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i WholeStageCodegen (31) Sort [ws_order_number,ws_item_sk] InputAdapter - Exchange [ws_order_number,ws_item_sk] #27 + Exchange [ws_order_number,ws_item_sk] #20 WholeStageCodegen (30) ColumnarToRow InputAdapter @@ -238,15 +220,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #28 - CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange #29 - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #18 InputAdapter WholeStageCodegen (33) Sort [wr_order_number,wr_item_sk] InputAdapter - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #16 + ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt index a0b14de93..7943726ea 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt @@ -1,15 +1,15 @@ == Physical Plan == -TakeOrderedAndProject (118) -+- * HashAggregate (117) - +- Exchange (116) - +- * HashAggregate (115) - +- Union (114) - :- * HashAggregate (103) - : +- Exchange (102) - : +- * HashAggregate (101) - : +- Union (100) - : :- * Project (37) - : : +- * BroadcastHashJoin LeftOuter BuildRight (36) +TakeOrderedAndProject (99) ++- * HashAggregate (98) + +- Exchange (97) + +- * HashAggregate (96) + +- Union (95) + :- * HashAggregate (84) + : +- Exchange (83) + : +- * HashAggregate (82) + : +- Union (81) + : :- * Project (32) + : : +- * BroadcastHashJoin LeftOuter BuildRight (31) : : :- * HashAggregate (17) : : : +- Exchange (16) : : : +- * ColumnarToRow (15) @@ -27,96 +27,77 @@ TakeOrderedAndProject (118) : : : +- CometBroadcastExchange (11) : : : +- CometFilter (10) : : : +- CometScan parquet spark_catalog.default.store (9) - : : +- BroadcastExchange (35) - : : +- * HashAggregate (34) - : : +- Exchange (33) - : : +- * ColumnarToRow (32) - : : +- CometHashAggregate (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) + : : +- BroadcastExchange (30) + : : +- * HashAggregate (29) + : : +- Exchange (28) + : : +- * ColumnarToRow (27) + : : +- CometHashAggregate (26) + : : +- CometProject (25) + : : +- CometBroadcastHashJoin (24) + : : :- CometProject (22) + : : : +- CometBroadcastHashJoin (21) : : : :- CometFilter (19) : : : : +- CometScan parquet spark_catalog.default.store_returns (18) - : : : +- CometBroadcastExchange (23) - : : : +- CometProject (22) - : : : +- CometFilter (21) - : : : +- CometScan parquet spark_catalog.default.date_dim (20) - : : +- CometBroadcastExchange (28) - : : +- CometFilter (27) - : : +- CometScan parquet spark_catalog.default.store (26) - : :- * Project (62) - : : +- * BroadcastNestedLoopJoin Inner BuildLeft (61) - : : :- BroadcastExchange (49) - : : : +- * HashAggregate (48) - : : : +- Exchange (47) - : : : +- * ColumnarToRow (46) - : : : +- CometHashAggregate (45) - : : : +- CometProject (44) - : : : +- CometBroadcastHashJoin (43) - : : : :- CometScan parquet spark_catalog.default.catalog_sales (38) - : : : +- CometBroadcastExchange (42) - : : : +- CometProject (41) - : : : +- CometFilter (40) - : : : +- CometScan parquet spark_catalog.default.date_dim (39) - : : +- * HashAggregate (60) - : : +- Exchange (59) - : : +- * ColumnarToRow (58) - : : +- CometHashAggregate (57) - : : +- CometProject (56) - : : +- CometBroadcastHashJoin (55) - : : :- CometScan parquet spark_catalog.default.catalog_returns (50) - : : +- CometBroadcastExchange (54) - : : +- CometProject (53) - : : +- CometFilter (52) - : : +- CometScan parquet spark_catalog.default.date_dim (51) - : +- * Project (99) - : +- * BroadcastHashJoin LeftOuter BuildRight (98) - : :- * HashAggregate (79) - : : +- Exchange (78) - : : +- * ColumnarToRow (77) - : : +- CometHashAggregate (76) - : : +- CometProject (75) - : : +- CometBroadcastHashJoin (74) - : : :- CometProject (70) - : : : +- CometBroadcastHashJoin (69) - : : : :- CometFilter (64) - : : : : +- CometScan parquet spark_catalog.default.web_sales (63) - : : : +- CometBroadcastExchange (68) - : : : +- CometProject (67) - : : : +- CometFilter (66) - : : : +- CometScan parquet spark_catalog.default.date_dim (65) - : : +- CometBroadcastExchange (73) - : : +- CometFilter (72) - : : +- CometScan parquet spark_catalog.default.web_page (71) - : +- BroadcastExchange (97) - : +- * HashAggregate (96) - : +- Exchange (95) - : +- * ColumnarToRow (94) - : +- CometHashAggregate (93) - : +- CometProject (92) - : +- CometBroadcastHashJoin (91) - : :- CometProject (87) - : : +- CometBroadcastHashJoin (86) - : : :- CometFilter (81) - : : : +- CometScan parquet spark_catalog.default.web_returns (80) - : : +- CometBroadcastExchange (85) - : : +- CometProject (84) - : : +- CometFilter (83) - : : +- CometScan parquet spark_catalog.default.date_dim (82) - : +- CometBroadcastExchange (90) - : +- CometFilter (89) - : +- CometScan parquet spark_catalog.default.web_page (88) - :- * HashAggregate (108) - : +- Exchange (107) - : +- * HashAggregate (106) - : +- * HashAggregate (105) - : +- ReusedExchange (104) - +- * HashAggregate (113) - +- Exchange (112) - +- * HashAggregate (111) - +- * HashAggregate (110) - +- ReusedExchange (109) + : : : +- ReusedExchange (20) + : : +- ReusedExchange (23) + : :- * Project (51) + : : +- * BroadcastNestedLoopJoin Inner BuildLeft (50) + : : :- BroadcastExchange (41) + : : : +- * HashAggregate (40) + : : : +- Exchange (39) + : : : +- * ColumnarToRow (38) + : : : +- CometHashAggregate (37) + : : : +- CometProject (36) + : : : +- CometBroadcastHashJoin (35) + : : : :- CometScan parquet spark_catalog.default.catalog_sales (33) + : : : +- ReusedExchange (34) + : : +- * HashAggregate (49) + : : +- Exchange (48) + : : +- * ColumnarToRow (47) + : : +- CometHashAggregate (46) + : : +- CometProject (45) + : : +- CometBroadcastHashJoin (44) + : : :- CometScan parquet spark_catalog.default.catalog_returns (42) + : : +- ReusedExchange (43) + : +- * Project (80) + : +- * BroadcastHashJoin LeftOuter BuildRight (79) + : :- * HashAggregate (65) + : : +- Exchange (64) + : : +- * ColumnarToRow (63) + : : +- CometHashAggregate (62) + : : +- CometProject (61) + : : +- CometBroadcastHashJoin (60) + : : :- CometProject (56) + : : : +- CometBroadcastHashJoin (55) + : : : :- CometFilter (53) + : : : : +- CometScan parquet spark_catalog.default.web_sales (52) + : : : +- ReusedExchange (54) + : : +- CometBroadcastExchange (59) + : : +- CometFilter (58) + : : +- CometScan parquet spark_catalog.default.web_page (57) + : +- BroadcastExchange (78) + : +- * HashAggregate (77) + : +- Exchange (76) + : +- * ColumnarToRow (75) + : +- CometHashAggregate (74) + : +- CometProject (73) + : +- CometBroadcastHashJoin (72) + : :- CometProject (70) + : : +- CometBroadcastHashJoin (69) + : : :- CometFilter (67) + : : : +- CometScan parquet spark_catalog.default.web_returns (66) + : : +- ReusedExchange (68) + : +- ReusedExchange (71) + :- * HashAggregate (89) + : +- Exchange (88) + : +- * HashAggregate (87) + : +- * HashAggregate (86) + : +- ReusedExchange (85) + +- * HashAggregate (94) + +- Exchange (93) + +- * HashAggregate (92) + +- * HashAggregate (91) + +- ReusedExchange (90) (1) Scan parquet spark_catalog.default.store_sales @@ -214,531 +195,427 @@ ReadSchema: struct - -(21) CometFilter -Input [2]: [d_date_sk#20, d_date#21] -Condition : (((isnotnull(d_date#21) AND (d_date#21 >= 1998-08-04)) AND (d_date#21 <= 1998-09-03)) AND isnotnull(d_date_sk#20)) +(20) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#20] -(22) CometProject -Input [2]: [d_date_sk#20, d_date#21] -Arguments: [d_date_sk#20], [d_date_sk#20] - -(23) CometBroadcastExchange -Input [1]: [d_date_sk#20] -Arguments: [d_date_sk#20] - -(24) CometBroadcastHashJoin +(21) CometBroadcastHashJoin Left output [4]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, sr_returned_date_sk#18] Right output [1]: [d_date_sk#20] Arguments: [sr_returned_date_sk#18], [d_date_sk#20], Inner -(25) CometProject +(22) CometProject Input [5]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, sr_returned_date_sk#18, d_date_sk#20] Arguments: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17], [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17] -(26) Scan parquet spark_catalog.default.store -Output [1]: [s_store_sk#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(27) CometFilter -Input [1]: [s_store_sk#22] -Condition : isnotnull(s_store_sk#22) +(23) ReusedExchange [Reuses operator id: 11] +Output [1]: [s_store_sk#21] -(28) CometBroadcastExchange -Input [1]: [s_store_sk#22] -Arguments: [s_store_sk#22] - -(29) CometBroadcastHashJoin +(24) CometBroadcastHashJoin Left output [3]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17] -Right output [1]: [s_store_sk#22] -Arguments: [sr_store_sk#15], [s_store_sk#22], Inner +Right output [1]: [s_store_sk#21] +Arguments: [sr_store_sk#15], [s_store_sk#21], Inner -(30) CometProject -Input [4]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, s_store_sk#22] -Arguments: [sr_return_amt#16, sr_net_loss#17, s_store_sk#22], [sr_return_amt#16, sr_net_loss#17, s_store_sk#22] +(25) CometProject +Input [4]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, s_store_sk#21] +Arguments: [sr_return_amt#16, sr_net_loss#17, s_store_sk#21], [sr_return_amt#16, sr_net_loss#17, s_store_sk#21] -(31) CometHashAggregate -Input [3]: [sr_return_amt#16, sr_net_loss#17, s_store_sk#22] -Keys [1]: [s_store_sk#22] +(26) CometHashAggregate +Input [3]: [sr_return_amt#16, sr_net_loss#17, s_store_sk#21] +Keys [1]: [s_store_sk#21] Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#16)), partial_sum(UnscaledValue(sr_net_loss#17))] -(32) ColumnarToRow [codegen id : 2] -Input [3]: [s_store_sk#22, sum#23, sum#24] +(27) ColumnarToRow [codegen id : 2] +Input [3]: [s_store_sk#21, sum#22, sum#23] -(33) Exchange -Input [3]: [s_store_sk#22, sum#23, sum#24] -Arguments: hashpartitioning(s_store_sk#22, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(28) Exchange +Input [3]: [s_store_sk#21, sum#22, sum#23] +Arguments: hashpartitioning(s_store_sk#21, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(34) HashAggregate [codegen id : 3] -Input [3]: [s_store_sk#22, sum#23, sum#24] -Keys [1]: [s_store_sk#22] +(29) HashAggregate [codegen id : 3] +Input [3]: [s_store_sk#21, sum#22, sum#23] +Keys [1]: [s_store_sk#21] Functions [2]: [sum(UnscaledValue(sr_return_amt#16)), sum(UnscaledValue(sr_net_loss#17))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#16))#25, sum(UnscaledValue(sr_net_loss#17))#26] -Results [3]: [s_store_sk#22, MakeDecimal(sum(UnscaledValue(sr_return_amt#16))#25,17,2) AS returns#27, MakeDecimal(sum(UnscaledValue(sr_net_loss#17))#26,17,2) AS profit_loss#28] +Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#16))#24, sum(UnscaledValue(sr_net_loss#17))#25] +Results [3]: [s_store_sk#21, MakeDecimal(sum(UnscaledValue(sr_return_amt#16))#24,17,2) AS returns#26, MakeDecimal(sum(UnscaledValue(sr_net_loss#17))#25,17,2) AS profit_loss#27] -(35) BroadcastExchange -Input [3]: [s_store_sk#22, returns#27, profit_loss#28] +(30) BroadcastExchange +Input [3]: [s_store_sk#21, returns#26, profit_loss#27] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(36) BroadcastHashJoin [codegen id : 4] +(31) BroadcastHashJoin [codegen id : 4] Left keys [1]: [s_store_sk#8] -Right keys [1]: [s_store_sk#22] +Right keys [1]: [s_store_sk#21] Join type: LeftOuter Join condition: None -(37) Project [codegen id : 4] -Output [5]: [store channel AS channel#29, s_store_sk#8 AS id#30, sales#13, coalesce(returns#27, 0.00) AS returns#31, (profit#14 - coalesce(profit_loss#28, 0.00)) AS profit#32] -Input [6]: [s_store_sk#8, sales#13, profit#14, s_store_sk#22, returns#27, profit_loss#28] +(32) Project [codegen id : 4] +Output [5]: [store channel AS channel#28, s_store_sk#8 AS id#29, sales#13, coalesce(returns#26, 0.00) AS returns#30, (profit#14 - coalesce(profit_loss#27, 0.00)) AS profit#31] +Input [6]: [s_store_sk#8, sales#13, profit#14, s_store_sk#21, returns#26, profit_loss#27] -(38) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#33, cs_ext_sales_price#34, cs_net_profit#35, cs_sold_date_sk#36] +(33) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34, cs_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#36), dynamicpruningexpression(cs_sold_date_sk#36 IN dynamicpruning#37)] +PartitionFilters: [isnotnull(cs_sold_date_sk#35), dynamicpruningexpression(cs_sold_date_sk#35 IN dynamicpruning#36)] ReadSchema: struct -(39) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#38, d_date#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct +(34) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#37] -(40) CometFilter -Input [2]: [d_date_sk#38, d_date#39] -Condition : (((isnotnull(d_date#39) AND (d_date#39 >= 1998-08-04)) AND (d_date#39 <= 1998-09-03)) AND isnotnull(d_date_sk#38)) +(35) CometBroadcastHashJoin +Left output [4]: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34, cs_sold_date_sk#35] +Right output [1]: [d_date_sk#37] +Arguments: [cs_sold_date_sk#35], [d_date_sk#37], Inner -(41) CometProject -Input [2]: [d_date_sk#38, d_date#39] -Arguments: [d_date_sk#38], [d_date_sk#38] +(36) CometProject +Input [5]: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34, cs_sold_date_sk#35, d_date_sk#37] +Arguments: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34], [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34] -(42) CometBroadcastExchange -Input [1]: [d_date_sk#38] -Arguments: [d_date_sk#38] +(37) CometHashAggregate +Input [3]: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34] +Keys [1]: [cs_call_center_sk#32] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#33)), partial_sum(UnscaledValue(cs_net_profit#34))] -(43) CometBroadcastHashJoin -Left output [4]: [cs_call_center_sk#33, cs_ext_sales_price#34, cs_net_profit#35, cs_sold_date_sk#36] -Right output [1]: [d_date_sk#38] -Arguments: [cs_sold_date_sk#36], [d_date_sk#38], Inner +(38) ColumnarToRow [codegen id : 5] +Input [3]: [cs_call_center_sk#32, sum#38, sum#39] -(44) CometProject -Input [5]: [cs_call_center_sk#33, cs_ext_sales_price#34, cs_net_profit#35, cs_sold_date_sk#36, d_date_sk#38] -Arguments: [cs_call_center_sk#33, cs_ext_sales_price#34, cs_net_profit#35], [cs_call_center_sk#33, cs_ext_sales_price#34, cs_net_profit#35] +(39) Exchange +Input [3]: [cs_call_center_sk#32, sum#38, sum#39] +Arguments: hashpartitioning(cs_call_center_sk#32, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(45) CometHashAggregate -Input [3]: [cs_call_center_sk#33, cs_ext_sales_price#34, cs_net_profit#35] -Keys [1]: [cs_call_center_sk#33] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#34)), partial_sum(UnscaledValue(cs_net_profit#35))] +(40) HashAggregate [codegen id : 6] +Input [3]: [cs_call_center_sk#32, sum#38, sum#39] +Keys [1]: [cs_call_center_sk#32] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#33)), sum(UnscaledValue(cs_net_profit#34))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#33))#40, sum(UnscaledValue(cs_net_profit#34))#41] +Results [3]: [cs_call_center_sk#32, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#33))#40,17,2) AS sales#42, MakeDecimal(sum(UnscaledValue(cs_net_profit#34))#41,17,2) AS profit#43] -(46) ColumnarToRow [codegen id : 5] -Input [3]: [cs_call_center_sk#33, sum#40, sum#41] - -(47) Exchange -Input [3]: [cs_call_center_sk#33, sum#40, sum#41] -Arguments: hashpartitioning(cs_call_center_sk#33, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(48) HashAggregate [codegen id : 6] -Input [3]: [cs_call_center_sk#33, sum#40, sum#41] -Keys [1]: [cs_call_center_sk#33] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#34)), sum(UnscaledValue(cs_net_profit#35))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#34))#42, sum(UnscaledValue(cs_net_profit#35))#43] -Results [3]: [cs_call_center_sk#33, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#34))#42,17,2) AS sales#44, MakeDecimal(sum(UnscaledValue(cs_net_profit#35))#43,17,2) AS profit#45] - -(49) BroadcastExchange -Input [3]: [cs_call_center_sk#33, sales#44, profit#45] +(41) BroadcastExchange +Input [3]: [cs_call_center_sk#32, sales#42, profit#43] Arguments: IdentityBroadcastMode, [plan_id=5] -(50) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] +(42) Scan parquet spark_catalog.default.catalog_returns +Output [3]: [cr_return_amount#44, cr_net_loss#45, cr_returned_date_sk#46] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#48), dynamicpruningexpression(cr_returned_date_sk#48 IN dynamicpruning#49)] +PartitionFilters: [isnotnull(cr_returned_date_sk#46), dynamicpruningexpression(cr_returned_date_sk#46 IN dynamicpruning#47)] ReadSchema: struct -(51) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#50, d_date#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(52) CometFilter -Input [2]: [d_date_sk#50, d_date#51] -Condition : (((isnotnull(d_date#51) AND (d_date#51 >= 1998-08-04)) AND (d_date#51 <= 1998-09-03)) AND isnotnull(d_date_sk#50)) +(43) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#48] -(53) CometProject -Input [2]: [d_date_sk#50, d_date#51] -Arguments: [d_date_sk#50], [d_date_sk#50] +(44) CometBroadcastHashJoin +Left output [3]: [cr_return_amount#44, cr_net_loss#45, cr_returned_date_sk#46] +Right output [1]: [d_date_sk#48] +Arguments: [cr_returned_date_sk#46], [d_date_sk#48], Inner -(54) CometBroadcastExchange -Input [1]: [d_date_sk#50] -Arguments: [d_date_sk#50] - -(55) CometBroadcastHashJoin -Left output [3]: [cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] -Right output [1]: [d_date_sk#50] -Arguments: [cr_returned_date_sk#48], [d_date_sk#50], Inner - -(56) CometProject -Input [4]: [cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48, d_date_sk#50] -Arguments: [cr_return_amount#46, cr_net_loss#47], [cr_return_amount#46, cr_net_loss#47] +(45) CometProject +Input [4]: [cr_return_amount#44, cr_net_loss#45, cr_returned_date_sk#46, d_date_sk#48] +Arguments: [cr_return_amount#44, cr_net_loss#45], [cr_return_amount#44, cr_net_loss#45] -(57) CometHashAggregate -Input [2]: [cr_return_amount#46, cr_net_loss#47] +(46) CometHashAggregate +Input [2]: [cr_return_amount#44, cr_net_loss#45] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#46)), partial_sum(UnscaledValue(cr_net_loss#47))] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#44)), partial_sum(UnscaledValue(cr_net_loss#45))] -(58) ColumnarToRow [codegen id : 7] -Input [2]: [sum#52, sum#53] +(47) ColumnarToRow [codegen id : 7] +Input [2]: [sum#49, sum#50] -(59) Exchange -Input [2]: [sum#52, sum#53] +(48) Exchange +Input [2]: [sum#49, sum#50] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(60) HashAggregate -Input [2]: [sum#52, sum#53] +(49) HashAggregate +Input [2]: [sum#49, sum#50] Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#46)), sum(UnscaledValue(cr_net_loss#47))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#46))#54, sum(UnscaledValue(cr_net_loss#47))#55] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#46))#54,17,2) AS returns#56, MakeDecimal(sum(UnscaledValue(cr_net_loss#47))#55,17,2) AS profit_loss#57] +Functions [2]: [sum(UnscaledValue(cr_return_amount#44)), sum(UnscaledValue(cr_net_loss#45))] +Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#44))#51, sum(UnscaledValue(cr_net_loss#45))#52] +Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#44))#51,17,2) AS returns#53, MakeDecimal(sum(UnscaledValue(cr_net_loss#45))#52,17,2) AS profit_loss#54] -(61) BroadcastNestedLoopJoin [codegen id : 8] +(50) BroadcastNestedLoopJoin [codegen id : 8] Join type: Inner Join condition: None -(62) Project [codegen id : 8] -Output [5]: [catalog channel AS channel#58, cs_call_center_sk#33 AS id#59, sales#44, returns#56, (profit#45 - profit_loss#57) AS profit#60] -Input [5]: [cs_call_center_sk#33, sales#44, profit#45, returns#56, profit_loss#57] +(51) Project [codegen id : 8] +Output [5]: [catalog channel AS channel#55, cs_call_center_sk#32 AS id#56, sales#42, returns#53, (profit#43 - profit_loss#54) AS profit#57] +Input [5]: [cs_call_center_sk#32, sales#42, profit#43, returns#53, profit_loss#54] -(63) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#61, ws_ext_sales_price#62, ws_net_profit#63, ws_sold_date_sk#64] +(52) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, ws_sold_date_sk#61] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#64), dynamicpruningexpression(ws_sold_date_sk#64 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(ws_sold_date_sk#61), dynamicpruningexpression(ws_sold_date_sk#61 IN dynamicpruning#62)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct -(64) CometFilter -Input [4]: [ws_web_page_sk#61, ws_ext_sales_price#62, ws_net_profit#63, ws_sold_date_sk#64] -Condition : isnotnull(ws_web_page_sk#61) +(53) CometFilter +Input [4]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, ws_sold_date_sk#61] +Condition : isnotnull(ws_web_page_sk#58) -(65) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#66, d_date#67] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(66) CometFilter -Input [2]: [d_date_sk#66, d_date#67] -Condition : (((isnotnull(d_date#67) AND (d_date#67 >= 1998-08-04)) AND (d_date#67 <= 1998-09-03)) AND isnotnull(d_date_sk#66)) +(54) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#63] -(67) CometProject -Input [2]: [d_date_sk#66, d_date#67] -Arguments: [d_date_sk#66], [d_date_sk#66] - -(68) CometBroadcastExchange -Input [1]: [d_date_sk#66] -Arguments: [d_date_sk#66] - -(69) CometBroadcastHashJoin -Left output [4]: [ws_web_page_sk#61, ws_ext_sales_price#62, ws_net_profit#63, ws_sold_date_sk#64] -Right output [1]: [d_date_sk#66] -Arguments: [ws_sold_date_sk#64], [d_date_sk#66], Inner +(55) CometBroadcastHashJoin +Left output [4]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, ws_sold_date_sk#61] +Right output [1]: [d_date_sk#63] +Arguments: [ws_sold_date_sk#61], [d_date_sk#63], Inner -(70) CometProject -Input [5]: [ws_web_page_sk#61, ws_ext_sales_price#62, ws_net_profit#63, ws_sold_date_sk#64, d_date_sk#66] -Arguments: [ws_web_page_sk#61, ws_ext_sales_price#62, ws_net_profit#63], [ws_web_page_sk#61, ws_ext_sales_price#62, ws_net_profit#63] +(56) CometProject +Input [5]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, ws_sold_date_sk#61, d_date_sk#63] +Arguments: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60], [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60] -(71) Scan parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#68] +(57) Scan parquet spark_catalog.default.web_page +Output [1]: [wp_web_page_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct -(72) CometFilter -Input [1]: [wp_web_page_sk#68] -Condition : isnotnull(wp_web_page_sk#68) +(58) CometFilter +Input [1]: [wp_web_page_sk#64] +Condition : isnotnull(wp_web_page_sk#64) -(73) CometBroadcastExchange -Input [1]: [wp_web_page_sk#68] -Arguments: [wp_web_page_sk#68] +(59) CometBroadcastExchange +Input [1]: [wp_web_page_sk#64] +Arguments: [wp_web_page_sk#64] -(74) CometBroadcastHashJoin -Left output [3]: [ws_web_page_sk#61, ws_ext_sales_price#62, ws_net_profit#63] -Right output [1]: [wp_web_page_sk#68] -Arguments: [ws_web_page_sk#61], [wp_web_page_sk#68], Inner +(60) CometBroadcastHashJoin +Left output [3]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60] +Right output [1]: [wp_web_page_sk#64] +Arguments: [ws_web_page_sk#58], [wp_web_page_sk#64], Inner -(75) CometProject -Input [4]: [ws_web_page_sk#61, ws_ext_sales_price#62, ws_net_profit#63, wp_web_page_sk#68] -Arguments: [ws_ext_sales_price#62, ws_net_profit#63, wp_web_page_sk#68], [ws_ext_sales_price#62, ws_net_profit#63, wp_web_page_sk#68] +(61) CometProject +Input [4]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, wp_web_page_sk#64] +Arguments: [ws_ext_sales_price#59, ws_net_profit#60, wp_web_page_sk#64], [ws_ext_sales_price#59, ws_net_profit#60, wp_web_page_sk#64] -(76) CometHashAggregate -Input [3]: [ws_ext_sales_price#62, ws_net_profit#63, wp_web_page_sk#68] -Keys [1]: [wp_web_page_sk#68] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#62)), partial_sum(UnscaledValue(ws_net_profit#63))] +(62) CometHashAggregate +Input [3]: [ws_ext_sales_price#59, ws_net_profit#60, wp_web_page_sk#64] +Keys [1]: [wp_web_page_sk#64] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#59)), partial_sum(UnscaledValue(ws_net_profit#60))] -(77) ColumnarToRow [codegen id : 9] -Input [3]: [wp_web_page_sk#68, sum#69, sum#70] +(63) ColumnarToRow [codegen id : 9] +Input [3]: [wp_web_page_sk#64, sum#65, sum#66] -(78) Exchange -Input [3]: [wp_web_page_sk#68, sum#69, sum#70] -Arguments: hashpartitioning(wp_web_page_sk#68, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(64) Exchange +Input [3]: [wp_web_page_sk#64, sum#65, sum#66] +Arguments: hashpartitioning(wp_web_page_sk#64, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(79) HashAggregate [codegen id : 12] -Input [3]: [wp_web_page_sk#68, sum#69, sum#70] -Keys [1]: [wp_web_page_sk#68] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#62)), sum(UnscaledValue(ws_net_profit#63))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#62))#71, sum(UnscaledValue(ws_net_profit#63))#72] -Results [3]: [wp_web_page_sk#68, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#62))#71,17,2) AS sales#73, MakeDecimal(sum(UnscaledValue(ws_net_profit#63))#72,17,2) AS profit#74] +(65) HashAggregate [codegen id : 12] +Input [3]: [wp_web_page_sk#64, sum#65, sum#66] +Keys [1]: [wp_web_page_sk#64] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#59)), sum(UnscaledValue(ws_net_profit#60))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#59))#67, sum(UnscaledValue(ws_net_profit#60))#68] +Results [3]: [wp_web_page_sk#64, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#59))#67,17,2) AS sales#69, MakeDecimal(sum(UnscaledValue(ws_net_profit#60))#68,17,2) AS profit#70] -(80) Scan parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#75, wr_return_amt#76, wr_net_loss#77, wr_returned_date_sk#78] +(66) Scan parquet spark_catalog.default.web_returns +Output [4]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wr_returned_date_sk#74] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#78), dynamicpruningexpression(wr_returned_date_sk#78 IN dynamicpruning#79)] +PartitionFilters: [isnotnull(wr_returned_date_sk#74), dynamicpruningexpression(wr_returned_date_sk#74 IN dynamicpruning#75)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct -(81) CometFilter -Input [4]: [wr_web_page_sk#75, wr_return_amt#76, wr_net_loss#77, wr_returned_date_sk#78] -Condition : isnotnull(wr_web_page_sk#75) - -(82) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#80, d_date#81] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(83) CometFilter -Input [2]: [d_date_sk#80, d_date#81] -Condition : (((isnotnull(d_date#81) AND (d_date#81 >= 1998-08-04)) AND (d_date#81 <= 1998-09-03)) AND isnotnull(d_date_sk#80)) +(67) CometFilter +Input [4]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wr_returned_date_sk#74] +Condition : isnotnull(wr_web_page_sk#71) -(84) CometProject -Input [2]: [d_date_sk#80, d_date#81] -Arguments: [d_date_sk#80], [d_date_sk#80] +(68) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#76] -(85) CometBroadcastExchange -Input [1]: [d_date_sk#80] -Arguments: [d_date_sk#80] - -(86) CometBroadcastHashJoin -Left output [4]: [wr_web_page_sk#75, wr_return_amt#76, wr_net_loss#77, wr_returned_date_sk#78] -Right output [1]: [d_date_sk#80] -Arguments: [wr_returned_date_sk#78], [d_date_sk#80], Inner - -(87) CometProject -Input [5]: [wr_web_page_sk#75, wr_return_amt#76, wr_net_loss#77, wr_returned_date_sk#78, d_date_sk#80] -Arguments: [wr_web_page_sk#75, wr_return_amt#76, wr_net_loss#77], [wr_web_page_sk#75, wr_return_amt#76, wr_net_loss#77] - -(88) Scan parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#82] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(89) CometFilter -Input [1]: [wp_web_page_sk#82] -Condition : isnotnull(wp_web_page_sk#82) - -(90) CometBroadcastExchange -Input [1]: [wp_web_page_sk#82] -Arguments: [wp_web_page_sk#82] - -(91) CometBroadcastHashJoin -Left output [3]: [wr_web_page_sk#75, wr_return_amt#76, wr_net_loss#77] -Right output [1]: [wp_web_page_sk#82] -Arguments: [wr_web_page_sk#75], [wp_web_page_sk#82], Inner - -(92) CometProject -Input [4]: [wr_web_page_sk#75, wr_return_amt#76, wr_net_loss#77, wp_web_page_sk#82] -Arguments: [wr_return_amt#76, wr_net_loss#77, wp_web_page_sk#82], [wr_return_amt#76, wr_net_loss#77, wp_web_page_sk#82] - -(93) CometHashAggregate -Input [3]: [wr_return_amt#76, wr_net_loss#77, wp_web_page_sk#82] -Keys [1]: [wp_web_page_sk#82] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#76)), partial_sum(UnscaledValue(wr_net_loss#77))] - -(94) ColumnarToRow [codegen id : 10] -Input [3]: [wp_web_page_sk#82, sum#83, sum#84] - -(95) Exchange -Input [3]: [wp_web_page_sk#82, sum#83, sum#84] -Arguments: hashpartitioning(wp_web_page_sk#82, 5), ENSURE_REQUIREMENTS, [plan_id=8] - -(96) HashAggregate [codegen id : 11] -Input [3]: [wp_web_page_sk#82, sum#83, sum#84] -Keys [1]: [wp_web_page_sk#82] -Functions [2]: [sum(UnscaledValue(wr_return_amt#76)), sum(UnscaledValue(wr_net_loss#77))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#76))#85, sum(UnscaledValue(wr_net_loss#77))#86] -Results [3]: [wp_web_page_sk#82, MakeDecimal(sum(UnscaledValue(wr_return_amt#76))#85,17,2) AS returns#87, MakeDecimal(sum(UnscaledValue(wr_net_loss#77))#86,17,2) AS profit_loss#88] +(69) CometBroadcastHashJoin +Left output [4]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wr_returned_date_sk#74] +Right output [1]: [d_date_sk#76] +Arguments: [wr_returned_date_sk#74], [d_date_sk#76], Inner -(97) BroadcastExchange -Input [3]: [wp_web_page_sk#82, returns#87, profit_loss#88] +(70) CometProject +Input [5]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wr_returned_date_sk#74, d_date_sk#76] +Arguments: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73], [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73] + +(71) ReusedExchange [Reuses operator id: 59] +Output [1]: [wp_web_page_sk#77] + +(72) CometBroadcastHashJoin +Left output [3]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73] +Right output [1]: [wp_web_page_sk#77] +Arguments: [wr_web_page_sk#71], [wp_web_page_sk#77], Inner + +(73) CometProject +Input [4]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wp_web_page_sk#77] +Arguments: [wr_return_amt#72, wr_net_loss#73, wp_web_page_sk#77], [wr_return_amt#72, wr_net_loss#73, wp_web_page_sk#77] + +(74) CometHashAggregate +Input [3]: [wr_return_amt#72, wr_net_loss#73, wp_web_page_sk#77] +Keys [1]: [wp_web_page_sk#77] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#72)), partial_sum(UnscaledValue(wr_net_loss#73))] + +(75) ColumnarToRow [codegen id : 10] +Input [3]: [wp_web_page_sk#77, sum#78, sum#79] + +(76) Exchange +Input [3]: [wp_web_page_sk#77, sum#78, sum#79] +Arguments: hashpartitioning(wp_web_page_sk#77, 5), ENSURE_REQUIREMENTS, [plan_id=8] + +(77) HashAggregate [codegen id : 11] +Input [3]: [wp_web_page_sk#77, sum#78, sum#79] +Keys [1]: [wp_web_page_sk#77] +Functions [2]: [sum(UnscaledValue(wr_return_amt#72)), sum(UnscaledValue(wr_net_loss#73))] +Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#72))#80, sum(UnscaledValue(wr_net_loss#73))#81] +Results [3]: [wp_web_page_sk#77, MakeDecimal(sum(UnscaledValue(wr_return_amt#72))#80,17,2) AS returns#82, MakeDecimal(sum(UnscaledValue(wr_net_loss#73))#81,17,2) AS profit_loss#83] + +(78) BroadcastExchange +Input [3]: [wp_web_page_sk#77, returns#82, profit_loss#83] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -(98) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [wp_web_page_sk#68] -Right keys [1]: [wp_web_page_sk#82] +(79) BroadcastHashJoin [codegen id : 12] +Left keys [1]: [wp_web_page_sk#64] +Right keys [1]: [wp_web_page_sk#77] Join type: LeftOuter Join condition: None -(99) Project [codegen id : 12] -Output [5]: [web channel AS channel#89, wp_web_page_sk#68 AS id#90, sales#73, coalesce(returns#87, 0.00) AS returns#91, (profit#74 - coalesce(profit_loss#88, 0.00)) AS profit#92] -Input [6]: [wp_web_page_sk#68, sales#73, profit#74, wp_web_page_sk#82, returns#87, profit_loss#88] - -(100) Union - -(101) HashAggregate [codegen id : 13] -Input [5]: [channel#29, id#30, sales#13, returns#31, profit#32] -Keys [2]: [channel#29, id#30] -Functions [3]: [partial_sum(sales#13), partial_sum(returns#31), partial_sum(profit#32)] -Aggregate Attributes [6]: [sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98] -Results [8]: [channel#29, id#30, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104] - -(102) Exchange -Input [8]: [channel#29, id#30, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104] -Arguments: hashpartitioning(channel#29, id#30, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(103) HashAggregate [codegen id : 14] -Input [8]: [channel#29, id#30, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104] -Keys [2]: [channel#29, id#30] -Functions [3]: [sum(sales#13), sum(returns#31), sum(profit#32)] -Aggregate Attributes [3]: [sum(sales#13)#105, sum(returns#31)#106, sum(profit#32)#107] -Results [5]: [channel#29, id#30, cast(sum(sales#13)#105 as decimal(37,2)) AS sales#108, cast(sum(returns#31)#106 as decimal(37,2)) AS returns#109, cast(sum(profit#32)#107 as decimal(38,2)) AS profit#110] - -(104) ReusedExchange [Reuses operator id: 102] -Output [8]: [channel#29, id#30, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104] - -(105) HashAggregate [codegen id : 28] -Input [8]: [channel#29, id#30, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104] -Keys [2]: [channel#29, id#30] -Functions [3]: [sum(sales#13), sum(returns#31), sum(profit#32)] -Aggregate Attributes [3]: [sum(sales#13)#105, sum(returns#31)#106, sum(profit#32)#107] -Results [4]: [channel#29, sum(sales#13)#105 AS sales#111, sum(returns#31)#106 AS returns#112, sum(profit#32)#107 AS profit#113] - -(106) HashAggregate [codegen id : 28] -Input [4]: [channel#29, sales#111, returns#112, profit#113] -Keys [1]: [channel#29] -Functions [3]: [partial_sum(sales#111), partial_sum(returns#112), partial_sum(profit#113)] -Aggregate Attributes [6]: [sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] -Results [7]: [channel#29, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] - -(107) Exchange -Input [7]: [channel#29, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] -Arguments: hashpartitioning(channel#29, 5), ENSURE_REQUIREMENTS, [plan_id=11] - -(108) HashAggregate [codegen id : 29] -Input [7]: [channel#29, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] -Keys [1]: [channel#29] -Functions [3]: [sum(sales#111), sum(returns#112), sum(profit#113)] -Aggregate Attributes [3]: [sum(sales#111)#126, sum(returns#112)#127, sum(profit#113)#128] -Results [5]: [channel#29, null AS id#129, sum(sales#111)#126 AS sales#130, sum(returns#112)#127 AS returns#131, sum(profit#113)#128 AS profit#132] - -(109) ReusedExchange [Reuses operator id: 102] -Output [8]: [channel#29, id#30, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104] - -(110) HashAggregate [codegen id : 43] -Input [8]: [channel#29, id#30, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104] -Keys [2]: [channel#29, id#30] -Functions [3]: [sum(sales#13), sum(returns#31), sum(profit#32)] -Aggregate Attributes [3]: [sum(sales#13)#105, sum(returns#31)#106, sum(profit#32)#107] -Results [3]: [sum(sales#13)#105 AS sales#111, sum(returns#31)#106 AS returns#112, sum(profit#32)#107 AS profit#113] - -(111) HashAggregate [codegen id : 43] -Input [3]: [sales#111, returns#112, profit#113] +(80) Project [codegen id : 12] +Output [5]: [web channel AS channel#84, wp_web_page_sk#64 AS id#85, sales#69, coalesce(returns#82, 0.00) AS returns#86, (profit#70 - coalesce(profit_loss#83, 0.00)) AS profit#87] +Input [6]: [wp_web_page_sk#64, sales#69, profit#70, wp_web_page_sk#77, returns#82, profit_loss#83] + +(81) Union + +(82) HashAggregate [codegen id : 13] +Input [5]: [channel#28, id#29, sales#13, returns#30, profit#31] +Keys [2]: [channel#28, id#29] +Functions [3]: [partial_sum(sales#13), partial_sum(returns#30), partial_sum(profit#31)] +Aggregate Attributes [6]: [sum#88, isEmpty#89, sum#90, isEmpty#91, sum#92, isEmpty#93] +Results [8]: [channel#28, id#29, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] + +(83) Exchange +Input [8]: [channel#28, id#29, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] +Arguments: hashpartitioning(channel#28, id#29, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(84) HashAggregate [codegen id : 14] +Input [8]: [channel#28, id#29, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] +Keys [2]: [channel#28, id#29] +Functions [3]: [sum(sales#13), sum(returns#30), sum(profit#31)] +Aggregate Attributes [3]: [sum(sales#13)#100, sum(returns#30)#101, sum(profit#31)#102] +Results [5]: [channel#28, id#29, cast(sum(sales#13)#100 as decimal(37,2)) AS sales#103, cast(sum(returns#30)#101 as decimal(37,2)) AS returns#104, cast(sum(profit#31)#102 as decimal(38,2)) AS profit#105] + +(85) ReusedExchange [Reuses operator id: 83] +Output [8]: [channel#28, id#29, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] + +(86) HashAggregate [codegen id : 28] +Input [8]: [channel#28, id#29, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] +Keys [2]: [channel#28, id#29] +Functions [3]: [sum(sales#13), sum(returns#30), sum(profit#31)] +Aggregate Attributes [3]: [sum(sales#13)#100, sum(returns#30)#101, sum(profit#31)#102] +Results [4]: [channel#28, sum(sales#13)#100 AS sales#106, sum(returns#30)#101 AS returns#107, sum(profit#31)#102 AS profit#108] + +(87) HashAggregate [codegen id : 28] +Input [4]: [channel#28, sales#106, returns#107, profit#108] +Keys [1]: [channel#28] +Functions [3]: [partial_sum(sales#106), partial_sum(returns#107), partial_sum(profit#108)] +Aggregate Attributes [6]: [sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Results [7]: [channel#28, sum#115, isEmpty#116, sum#117, isEmpty#118, sum#119, isEmpty#120] + +(88) Exchange +Input [7]: [channel#28, sum#115, isEmpty#116, sum#117, isEmpty#118, sum#119, isEmpty#120] +Arguments: hashpartitioning(channel#28, 5), ENSURE_REQUIREMENTS, [plan_id=11] + +(89) HashAggregate [codegen id : 29] +Input [7]: [channel#28, sum#115, isEmpty#116, sum#117, isEmpty#118, sum#119, isEmpty#120] +Keys [1]: [channel#28] +Functions [3]: [sum(sales#106), sum(returns#107), sum(profit#108)] +Aggregate Attributes [3]: [sum(sales#106)#121, sum(returns#107)#122, sum(profit#108)#123] +Results [5]: [channel#28, null AS id#124, sum(sales#106)#121 AS sales#125, sum(returns#107)#122 AS returns#126, sum(profit#108)#123 AS profit#127] + +(90) ReusedExchange [Reuses operator id: 83] +Output [8]: [channel#28, id#29, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] + +(91) HashAggregate [codegen id : 43] +Input [8]: [channel#28, id#29, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] +Keys [2]: [channel#28, id#29] +Functions [3]: [sum(sales#13), sum(returns#30), sum(profit#31)] +Aggregate Attributes [3]: [sum(sales#13)#100, sum(returns#30)#101, sum(profit#31)#102] +Results [3]: [sum(sales#13)#100 AS sales#106, sum(returns#30)#101 AS returns#107, sum(profit#31)#102 AS profit#108] + +(92) HashAggregate [codegen id : 43] +Input [3]: [sales#106, returns#107, profit#108] Keys: [] -Functions [3]: [partial_sum(sales#111), partial_sum(returns#112), partial_sum(profit#113)] -Aggregate Attributes [6]: [sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] -Results [6]: [sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] +Functions [3]: [partial_sum(sales#106), partial_sum(returns#107), partial_sum(profit#108)] +Aggregate Attributes [6]: [sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] +Results [6]: [sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] -(112) Exchange -Input [6]: [sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] +(93) Exchange +Input [6]: [sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] -(113) HashAggregate [codegen id : 44] -Input [6]: [sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] +(94) HashAggregate [codegen id : 44] +Input [6]: [sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] Keys: [] -Functions [3]: [sum(sales#111), sum(returns#112), sum(profit#113)] -Aggregate Attributes [3]: [sum(sales#111)#145, sum(returns#112)#146, sum(profit#113)#147] -Results [5]: [null AS channel#148, null AS id#149, sum(sales#111)#145 AS sales#150, sum(returns#112)#146 AS returns#151, sum(profit#113)#147 AS profit#152] +Functions [3]: [sum(sales#106), sum(returns#107), sum(profit#108)] +Aggregate Attributes [3]: [sum(sales#106)#140, sum(returns#107)#141, sum(profit#108)#142] +Results [5]: [null AS channel#143, null AS id#144, sum(sales#106)#140 AS sales#145, sum(returns#107)#141 AS returns#146, sum(profit#108)#142 AS profit#147] -(114) Union +(95) Union -(115) HashAggregate [codegen id : 45] -Input [5]: [channel#29, id#30, sales#108, returns#109, profit#110] -Keys [5]: [channel#29, id#30, sales#108, returns#109, profit#110] +(96) HashAggregate [codegen id : 45] +Input [5]: [channel#28, id#29, sales#103, returns#104, profit#105] +Keys [5]: [channel#28, id#29, sales#103, returns#104, profit#105] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#29, id#30, sales#108, returns#109, profit#110] +Results [5]: [channel#28, id#29, sales#103, returns#104, profit#105] -(116) Exchange -Input [5]: [channel#29, id#30, sales#108, returns#109, profit#110] -Arguments: hashpartitioning(channel#29, id#30, sales#108, returns#109, profit#110, 5), ENSURE_REQUIREMENTS, [plan_id=13] +(97) Exchange +Input [5]: [channel#28, id#29, sales#103, returns#104, profit#105] +Arguments: hashpartitioning(channel#28, id#29, sales#103, returns#104, profit#105, 5), ENSURE_REQUIREMENTS, [plan_id=13] -(117) HashAggregate [codegen id : 46] -Input [5]: [channel#29, id#30, sales#108, returns#109, profit#110] -Keys [5]: [channel#29, id#30, sales#108, returns#109, profit#110] +(98) HashAggregate [codegen id : 46] +Input [5]: [channel#28, id#29, sales#103, returns#104, profit#105] +Keys [5]: [channel#28, id#29, sales#103, returns#104, profit#105] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#29, id#30, sales#108, returns#109, profit#110] +Results [5]: [channel#28, id#29, sales#103, returns#104, profit#105] -(118) TakeOrderedAndProject -Input [5]: [channel#29, id#30, sales#108, returns#109, profit#110] -Arguments: 100, [channel#29 ASC NULLS FIRST, id#30 ASC NULLS FIRST], [channel#29, id#30, sales#108, returns#109, profit#110] +(99) TakeOrderedAndProject +Input [5]: [channel#28, id#29, sales#103, returns#104, profit#105] +Arguments: 100, [channel#28 ASC NULLS FIRST, id#29 ASC NULLS FIRST], [channel#28, id#29, sales#103, returns#104, profit#105] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (123) -+- * ColumnarToRow (122) - +- CometProject (121) - +- CometFilter (120) - +- CometScan parquet spark_catalog.default.date_dim (119) +BroadcastExchange (104) ++- * ColumnarToRow (103) + +- CometProject (102) + +- CometFilter (101) + +- CometScan parquet spark_catalog.default.date_dim (100) -(119) Scan parquet spark_catalog.default.date_dim +(100) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#6, d_date#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] ReadSchema: struct -(120) CometFilter +(101) CometFilter Input [2]: [d_date_sk#6, d_date#7] Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) -(121) CometProject +(102) CometProject Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(122) ColumnarToRow [codegen id : 1] +(103) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(123) BroadcastExchange +(104) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] Subquery:2 Hosting operator id = 18 Hosting Expression = sr_returned_date_sk#18 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 38 Hosting Expression = cs_sold_date_sk#36 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#35 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 50 Hosting Expression = cr_returned_date_sk#48 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 42 Hosting Expression = cr_returned_date_sk#46 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#64 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 52 Hosting Expression = ws_sold_date_sk#61 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 80 Hosting Expression = wr_returned_date_sk#78 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 66 Hosting Expression = wr_returned_date_sk#74 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt index 8c07441f9..752b8c854 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt @@ -63,22 +63,17 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometFilter [sr_store_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #9 - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #10 - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] + ReusedExchange [d_date_sk] #5 + ReusedExchange [s_store_sk] #6 WholeStageCodegen (8) Project [cs_call_center_sk,sales,returns,profit,profit_loss] BroadcastNestedLoopJoin InputAdapter - BroadcastExchange #11 + BroadcastExchange #9 WholeStageCodegen (6) HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] InputAdapter - Exchange [cs_call_center_sk] #12 + Exchange [cs_call_center_sk] #10 WholeStageCodegen (5) ColumnarToRow InputAdapter @@ -87,13 +82,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #13 - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + ReusedExchange [d_date_sk] #5 HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] InputAdapter - Exchange #14 + Exchange #11 WholeStageCodegen (7) ColumnarToRow InputAdapter @@ -102,16 +94,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #15 - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + ReusedExchange [d_date_sk] #5 WholeStageCodegen (12) Project [wp_web_page_sk,sales,returns,profit,profit_loss] BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] InputAdapter - Exchange [wp_web_page_sk] #16 + Exchange [wp_web_page_sk] #12 WholeStageCodegen (9) ColumnarToRow InputAdapter @@ -123,19 +112,16 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometFilter [ws_web_page_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #17 - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #18 + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange #13 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter - BroadcastExchange #19 + BroadcastExchange #14 WholeStageCodegen (11) HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] InputAdapter - Exchange [wp_web_page_sk] #20 + Exchange [wp_web_page_sk] #15 WholeStageCodegen (10) ColumnarToRow InputAdapter @@ -147,17 +133,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometFilter [wr_web_page_sk] CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #21 - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #22 - CometFilter [wp_web_page_sk] - CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] + ReusedExchange [d_date_sk] #5 + ReusedExchange [wp_web_page_sk] #13 WholeStageCodegen (29) 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] #23 + Exchange [channel] #16 WholeStageCodegen (28) 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] @@ -166,7 +147,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (44) 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 #24 + Exchange #17 WholeStageCodegen (43) 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] 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 2e1444281..c5fef022c 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -63,6 +63,36 @@ class CometExecSuite extends CometTestBase { } } + test("fix CometNativeExec.doCanonicalize for ReusedExchangeExec") { + assume(isSpark34Plus, "ChunkedByteBuffer is not serializable before Spark 3.4+") + withSQLConf( + CometConf.COMET_EXEC_BROADCAST_FORCE_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + withTable("td") { + testData + .withColumn("bucket", $"key" % 3) + .write + .mode(SaveMode.Overwrite) + .bucketBy(2, "bucket") + .format("parquet") + .saveAsTable("td") + val df = sql(""" + |SELECT t1.key, t2.key, t3.key + |FROM td AS t1 + |JOIN td AS t2 ON t2.key = t1.key + |JOIN td AS t3 ON t3.key = t2.key + |WHERE t1.bucket = 1 AND t2.bucket = 1 AND t3.bucket = 1 + |""".stripMargin) + val reusedPlan = ReuseExchangeAndSubquery.apply(df.queryExecution.executedPlan) + val reusedExchanges = collect(reusedPlan) { case r: ReusedExchangeExec => + r + } + assert(reusedExchanges.size == 1) + assert(reusedExchanges.head.child.isInstanceOf[CometBroadcastExchangeExec]) + } + } + } + test("ReusedExchangeExec should work on CometBroadcastExchangeExec") { assume(isSpark34Plus, "ChunkedByteBuffer is not serializable before Spark 3.4+") withSQLConf(