== Physical Plan ==
TakeOrderedAndProject (71)
+- * Project (70)
   +- * BroadcastHashJoin Inner BuildRight (69)
      :- * Project (52)
      :  +- * BroadcastHashJoin Inner BuildRight (51)
      :     :- * BroadcastHashJoin Inner BuildRight (33)
      :     :  :- * Filter (16)
      :     :  :  +- * HashAggregate (15)
      :     :  :     +- Exchange (14)
      :     :  :        +- * HashAggregate (13)
      :     :  :           +- * Project (12)
      :     :  :              +- * BroadcastHashJoin Inner BuildRight (11)
      :     :  :                 :- * Project (9)
      :     :  :                 :  +- * BroadcastHashJoin Inner BuildRight (8)
      :     :  :                 :     :- * Filter (3)
      :     :  :                 :     :  +- * ColumnarToRow (2)
      :     :  :                 :     :     +- Scan parquet spark_catalog.default.customer (1)
      :     :  :                 :     +- BroadcastExchange (7)
      :     :  :                 :        +- * Filter (6)
      :     :  :                 :           +- * ColumnarToRow (5)
      :     :  :                 :              +- Scan parquet spark_catalog.default.store_sales (4)
      :     :  :                 +- ReusedExchange (10)
      :     :  +- BroadcastExchange (32)
      :     :     +- * HashAggregate (31)
      :     :        +- Exchange (30)
      :     :           +- * HashAggregate (29)
      :     :              +- * Project (28)
      :     :                 +- * BroadcastHashJoin Inner BuildRight (27)
      :     :                    :- * Project (25)
      :     :                    :  +- * BroadcastHashJoin Inner BuildRight (24)
      :     :                    :     :- * Filter (19)
      :     :                    :     :  +- * ColumnarToRow (18)
      :     :                    :     :     +- Scan parquet spark_catalog.default.customer (17)
      :     :                    :     +- BroadcastExchange (23)
      :     :                    :        +- * Filter (22)
      :     :                    :           +- * ColumnarToRow (21)
      :     :                    :              +- Scan parquet spark_catalog.default.store_sales (20)
      :     :                    +- ReusedExchange (26)
      :     +- BroadcastExchange (50)
      :        +- * Filter (49)
      :           +- * HashAggregate (48)
      :              +- Exchange (47)
      :                 +- * HashAggregate (46)
      :                    +- * Project (45)
      :                       +- * BroadcastHashJoin Inner BuildRight (44)
      :                          :- * Project (42)
      :                          :  +- * BroadcastHashJoin Inner BuildRight (41)
      :                          :     :- * Filter (36)
      :                          :     :  +- * ColumnarToRow (35)
      :                          :     :     +- Scan parquet spark_catalog.default.customer (34)
      :                          :     +- BroadcastExchange (40)
      :                          :        +- * Filter (39)
      :                          :           +- * ColumnarToRow (38)
      :                          :              +- Scan parquet spark_catalog.default.web_sales (37)
      :                          +- ReusedExchange (43)
      +- BroadcastExchange (68)
         +- * HashAggregate (67)
            +- Exchange (66)
               +- * HashAggregate (65)
                  +- * Project (64)
                     +- * BroadcastHashJoin Inner BuildRight (63)
                        :- * Project (61)
                        :  +- * BroadcastHashJoin Inner BuildRight (60)
                        :     :- * Filter (55)
                        :     :  +- * ColumnarToRow (54)
                        :     :     +- Scan parquet spark_catalog.default.customer (53)
                        :     +- BroadcastExchange (59)
                        :        +- * Filter (58)
                        :           +- * ColumnarToRow (57)
                        :              +- Scan parquet spark_catalog.default.web_sales (56)
                        +- ReusedExchange (62)


(1) Scan parquet spark_catalog.default.customer
Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4]
Batched: true
Location [not included in comparison]/{warehouse_dir}/customer]
PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)]
ReadSchema: struct<c_customer_sk:int,c_customer_id:string,c_first_name:string,c_last_name:string>

(2) ColumnarToRow [codegen id : 3]
Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4]

(3) Filter [codegen id : 3]
Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4]
Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2))

(4) Scan parquet spark_catalog.default.store_sales
Output [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7]
Batched: true
Location: InMemoryFileIndex []
PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)]
PushedFilters: [IsNotNull(ss_customer_sk)]
ReadSchema: struct<ss_customer_sk:int,ss_net_paid:decimal(7,2)>

(5) ColumnarToRow [codegen id : 1]
Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7]

(6) Filter [codegen id : 1]
Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7]
Condition : isnotnull(ss_customer_sk#5)

(7) BroadcastExchange
Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1]

(8) BroadcastHashJoin [codegen id : 3]
Left keys [1]: [c_customer_sk#1]
Right keys [1]: [ss_customer_sk#5]
Join type: Inner
Join condition: None

(9) Project [codegen id : 3]
Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7]
Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7]

(10) ReusedExchange [Reuses operator id: 75]
Output [2]: [d_date_sk#9, d_year#10]

(11) BroadcastHashJoin [codegen id : 3]
Left keys [1]: [ss_sold_date_sk#7]
Right keys [1]: [d_date_sk#9]
Join type: Inner
Join condition: None

(12) Project [codegen id : 3]
Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10]
Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7, d_date_sk#9, d_year#10]

(13) HashAggregate [codegen id : 3]
Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10]
Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10]
Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#6))]
Aggregate Attributes [1]: [sum#11]
Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12]

(14) Exchange
Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12]
Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, 5), ENSURE_REQUIREMENTS, [plan_id=2]

(15) HashAggregate [codegen id : 16]
Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12]
Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10]
Functions [1]: [sum(UnscaledValue(ss_net_paid#6))]
Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#6))#13]
Results [2]: [c_customer_id#2 AS customer_id#14, MakeDecimal(sum(UnscaledValue(ss_net_paid#6))#13,17,2) AS year_total#15]

(16) Filter [codegen id : 16]
Input [2]: [customer_id#14, year_total#15]
Condition : (isnotnull(year_total#15) AND (year_total#15 > 0.00))

(17) Scan parquet spark_catalog.default.customer
Output [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19]
Batched: true
Location [not included in comparison]/{warehouse_dir}/customer]
PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)]
ReadSchema: struct<c_customer_sk:int,c_customer_id:string,c_first_name:string,c_last_name:string>

(18) ColumnarToRow [codegen id : 6]
Input [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19]

(19) Filter [codegen id : 6]
Input [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19]
Condition : (isnotnull(c_customer_sk#16) AND isnotnull(c_customer_id#17))

(20) Scan parquet spark_catalog.default.store_sales
Output [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22]
Batched: true
Location: InMemoryFileIndex []
PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_sold_date_sk#22 IN dynamicpruning#23)]
PushedFilters: [IsNotNull(ss_customer_sk)]
ReadSchema: struct<ss_customer_sk:int,ss_net_paid:decimal(7,2)>

(21) ColumnarToRow [codegen id : 4]
Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22]

(22) Filter [codegen id : 4]
Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22]
Condition : isnotnull(ss_customer_sk#20)

(23) BroadcastExchange
Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3]

(24) BroadcastHashJoin [codegen id : 6]
Left keys [1]: [c_customer_sk#16]
Right keys [1]: [ss_customer_sk#20]
Join type: Inner
Join condition: None

(25) Project [codegen id : 6]
Output [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22]
Input [7]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19, ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22]

(26) ReusedExchange [Reuses operator id: 79]
Output [2]: [d_date_sk#24, d_year#25]

(27) BroadcastHashJoin [codegen id : 6]
Left keys [1]: [ss_sold_date_sk#22]
Right keys [1]: [d_date_sk#24]
Join type: Inner
Join condition: None

(28) Project [codegen id : 6]
Output [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25]
Input [7]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22, d_date_sk#24, d_year#25]

(29) HashAggregate [codegen id : 6]
Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25]
Keys [4]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25]
Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#21))]
Aggregate Attributes [1]: [sum#26]
Results [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27]

(30) Exchange
Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27]
Arguments: hashpartitioning(c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, 5), ENSURE_REQUIREMENTS, [plan_id=4]

(31) HashAggregate [codegen id : 7]
Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27]
Keys [4]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25]
Functions [1]: [sum(UnscaledValue(ss_net_paid#21))]
Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#21))#13]
Results [4]: [c_customer_id#17 AS customer_id#28, c_first_name#18 AS customer_first_name#29, c_last_name#19 AS customer_last_name#30, MakeDecimal(sum(UnscaledValue(ss_net_paid#21))#13,17,2) AS year_total#31]

(32) BroadcastExchange
Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31]
Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5]

(33) BroadcastHashJoin [codegen id : 16]
Left keys [1]: [customer_id#14]
Right keys [1]: [customer_id#28]
Join type: Inner
Join condition: None

(34) Scan parquet spark_catalog.default.customer
Output [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35]
Batched: true
Location [not included in comparison]/{warehouse_dir}/customer]
PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)]
ReadSchema: struct<c_customer_sk:int,c_customer_id:string,c_first_name:string,c_last_name:string>

(35) ColumnarToRow [codegen id : 10]
Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35]

(36) Filter [codegen id : 10]
Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35]
Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_customer_id#33))

(37) Scan parquet spark_catalog.default.web_sales
Output [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38]
Batched: true
Location: InMemoryFileIndex []
PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#8)]
PushedFilters: [IsNotNull(ws_bill_customer_sk)]
ReadSchema: struct<ws_bill_customer_sk:int,ws_net_paid:decimal(7,2)>

(38) ColumnarToRow [codegen id : 8]
Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38]

(39) Filter [codegen id : 8]
Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38]
Condition : isnotnull(ws_bill_customer_sk#36)

(40) BroadcastExchange
Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6]

(41) BroadcastHashJoin [codegen id : 10]
Left keys [1]: [c_customer_sk#32]
Right keys [1]: [ws_bill_customer_sk#36]
Join type: Inner
Join condition: None

(42) Project [codegen id : 10]
Output [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38]
Input [7]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35, ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38]

(43) ReusedExchange [Reuses operator id: 75]
Output [2]: [d_date_sk#39, d_year#40]

(44) BroadcastHashJoin [codegen id : 10]
Left keys [1]: [ws_sold_date_sk#38]
Right keys [1]: [d_date_sk#39]
Join type: Inner
Join condition: None

(45) Project [codegen id : 10]
Output [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#40]
Input [7]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38, d_date_sk#39, d_year#40]

(46) HashAggregate [codegen id : 10]
Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#40]
Keys [4]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#40]
Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#37))]
Aggregate Attributes [1]: [sum#41]
Results [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#40, sum#42]

(47) Exchange
Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#40, sum#42]
Arguments: hashpartitioning(c_customer_id#33, c_first_name#34, c_last_name#35, d_year#40, 5), ENSURE_REQUIREMENTS, [plan_id=7]

(48) HashAggregate [codegen id : 11]
Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#40, sum#42]
Keys [4]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#40]
Functions [1]: [sum(UnscaledValue(ws_net_paid#37))]
Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#37))#43]
Results [2]: [c_customer_id#33 AS customer_id#44, MakeDecimal(sum(UnscaledValue(ws_net_paid#37))#43,17,2) AS year_total#45]

(49) Filter [codegen id : 11]
Input [2]: [customer_id#44, year_total#45]
Condition : (isnotnull(year_total#45) AND (year_total#45 > 0.00))

(50) BroadcastExchange
Input [2]: [customer_id#44, year_total#45]
Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8]

(51) BroadcastHashJoin [codegen id : 16]
Left keys [1]: [customer_id#14]
Right keys [1]: [customer_id#44]
Join type: Inner
Join condition: None

(52) Project [codegen id : 16]
Output [7]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#45]
Input [8]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#44, year_total#45]

(53) Scan parquet spark_catalog.default.customer
Output [4]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49]
Batched: true
Location [not included in comparison]/{warehouse_dir}/customer]
PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)]
ReadSchema: struct<c_customer_sk:int,c_customer_id:string,c_first_name:string,c_last_name:string>

(54) ColumnarToRow [codegen id : 14]
Input [4]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49]

(55) Filter [codegen id : 14]
Input [4]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49]
Condition : (isnotnull(c_customer_sk#46) AND isnotnull(c_customer_id#47))

(56) Scan parquet spark_catalog.default.web_sales
Output [3]: [ws_bill_customer_sk#50, ws_net_paid#51, ws_sold_date_sk#52]
Batched: true
Location: InMemoryFileIndex []
PartitionFilters: [isnotnull(ws_sold_date_sk#52), dynamicpruningexpression(ws_sold_date_sk#52 IN dynamicpruning#23)]
PushedFilters: [IsNotNull(ws_bill_customer_sk)]
ReadSchema: struct<ws_bill_customer_sk:int,ws_net_paid:decimal(7,2)>

(57) ColumnarToRow [codegen id : 12]
Input [3]: [ws_bill_customer_sk#50, ws_net_paid#51, ws_sold_date_sk#52]

(58) Filter [codegen id : 12]
Input [3]: [ws_bill_customer_sk#50, ws_net_paid#51, ws_sold_date_sk#52]
Condition : isnotnull(ws_bill_customer_sk#50)

(59) BroadcastExchange
Input [3]: [ws_bill_customer_sk#50, ws_net_paid#51, ws_sold_date_sk#52]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9]

(60) BroadcastHashJoin [codegen id : 14]
Left keys [1]: [c_customer_sk#46]
Right keys [1]: [ws_bill_customer_sk#50]
Join type: Inner
Join condition: None

(61) Project [codegen id : 14]
Output [5]: [c_customer_id#47, c_first_name#48, c_last_name#49, ws_net_paid#51, ws_sold_date_sk#52]
Input [7]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49, ws_bill_customer_sk#50, ws_net_paid#51, ws_sold_date_sk#52]

(62) ReusedExchange [Reuses operator id: 79]
Output [2]: [d_date_sk#53, d_year#54]

(63) BroadcastHashJoin [codegen id : 14]
Left keys [1]: [ws_sold_date_sk#52]
Right keys [1]: [d_date_sk#53]
Join type: Inner
Join condition: None

(64) Project [codegen id : 14]
Output [5]: [c_customer_id#47, c_first_name#48, c_last_name#49, ws_net_paid#51, d_year#54]
Input [7]: [c_customer_id#47, c_first_name#48, c_last_name#49, ws_net_paid#51, ws_sold_date_sk#52, d_date_sk#53, d_year#54]

(65) HashAggregate [codegen id : 14]
Input [5]: [c_customer_id#47, c_first_name#48, c_last_name#49, ws_net_paid#51, d_year#54]
Keys [4]: [c_customer_id#47, c_first_name#48, c_last_name#49, d_year#54]
Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#51))]
Aggregate Attributes [1]: [sum#55]
Results [5]: [c_customer_id#47, c_first_name#48, c_last_name#49, d_year#54, sum#56]

(66) Exchange
Input [5]: [c_customer_id#47, c_first_name#48, c_last_name#49, d_year#54, sum#56]
Arguments: hashpartitioning(c_customer_id#47, c_first_name#48, c_last_name#49, d_year#54, 5), ENSURE_REQUIREMENTS, [plan_id=10]

(67) HashAggregate [codegen id : 15]
Input [5]: [c_customer_id#47, c_first_name#48, c_last_name#49, d_year#54, sum#56]
Keys [4]: [c_customer_id#47, c_first_name#48, c_last_name#49, d_year#54]
Functions [1]: [sum(UnscaledValue(ws_net_paid#51))]
Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#51))#43]
Results [2]: [c_customer_id#47 AS customer_id#57, MakeDecimal(sum(UnscaledValue(ws_net_paid#51))#43,17,2) AS year_total#58]

(68) BroadcastExchange
Input [2]: [customer_id#57, year_total#58]
Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11]

(69) BroadcastHashJoin [codegen id : 16]
Left keys [1]: [customer_id#14]
Right keys [1]: [customer_id#57]
Join type: Inner
Join condition: (CASE WHEN (year_total#45 > 0.00) THEN (year_total#58 / year_total#45) END > CASE WHEN (year_total#15 > 0.00) THEN (year_total#31 / year_total#15) END)

(70) Project [codegen id : 16]
Output [3]: [customer_id#28, customer_first_name#29, customer_last_name#30]
Input [9]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#45, customer_id#57, year_total#58]

(71) TakeOrderedAndProject
Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30]
Arguments: 100, [customer_first_name#29 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_last_name#30 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30]

===== Subqueries =====

Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8
BroadcastExchange (75)
+- * Filter (74)
   +- * ColumnarToRow (73)
      +- Scan parquet spark_catalog.default.date_dim (72)


(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<d_date_sk:int,d_year:int>

(73) ColumnarToRow [codegen id : 1]
Input [2]: [d_date_sk#9, d_year#10]

(74) Filter [codegen id : 1]
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))

(75) BroadcastExchange
Input [2]: [d_date_sk#9, d_year#10]
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#22 IN dynamicpruning#23
BroadcastExchange (79)
+- * Filter (78)
   +- * ColumnarToRow (77)
      +- Scan parquet spark_catalog.default.date_dim (76)


(76) Scan parquet spark_catalog.default.date_dim
Output [2]: [d_date_sk#24, d_year#25]
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<d_date_sk:int,d_year:int>

(77) ColumnarToRow [codegen id : 1]
Input [2]: [d_date_sk#24, d_year#25]

(78) Filter [codegen id : 1]
Input [2]: [d_date_sk#24, d_year#25]
Condition : (((isnotnull(d_year#25) AND (d_year#25 = 2002)) AND d_year#25 IN (2001,2002)) AND isnotnull(d_date_sk#24))

(79) BroadcastExchange
Input [2]: [d_date_sk#24, d_year#25]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13]

Subquery:3 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#8

Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#23


