== Physical Plan ==
TakeOrderedAndProject (39)
+- * Filter (38)
   +- * HashAggregate (37)
      +- Exchange (36)
         +- * HashAggregate (35)
            +- * Project (34)
               +- * BroadcastHashJoin Inner BuildRight (33)
                  :- * Project (18)
                  :  +- * BroadcastHashJoin Inner BuildRight (17)
                  :     :- * Project (15)
                  :     :  +- * BroadcastHashJoin Inner BuildRight (14)
                  :     :     :- * Project (9)
                  :     :     :  +- * BroadcastHashJoin Inner BuildRight (8)
                  :     :     :     :- * Filter (3)
                  :     :     :     :  +- * ColumnarToRow (2)
                  :     :     :     :     +- Scan parquet spark_catalog.default.customer_address (1)
                  :     :     :     +- BroadcastExchange (7)
                  :     :     :        +- * Filter (6)
                  :     :     :           +- * ColumnarToRow (5)
                  :     :     :              +- Scan parquet spark_catalog.default.customer (4)
                  :     :     +- BroadcastExchange (13)
                  :     :        +- * Filter (12)
                  :     :           +- * ColumnarToRow (11)
                  :     :              +- Scan parquet spark_catalog.default.store_sales (10)
                  :     +- ReusedExchange (16)
                  +- BroadcastExchange (32)
                     +- * Project (31)
                        +- * BroadcastHashJoin Inner BuildRight (30)
                           :- * Filter (21)
                           :  +- * ColumnarToRow (20)
                           :     +- Scan parquet spark_catalog.default.item (19)
                           +- BroadcastExchange (29)
                              +- * Filter (28)
                                 +- * HashAggregate (27)
                                    +- Exchange (26)
                                       +- * HashAggregate (25)
                                          +- * Filter (24)
                                             +- * ColumnarToRow (23)
                                                +- Scan parquet spark_catalog.default.item (22)


(1) Scan parquet spark_catalog.default.customer_address
Output [2]: [ca_address_sk#1, ca_state#2]
Batched: true
Location [not included in comparison]/{warehouse_dir}/customer_address]
PushedFilters: [IsNotNull(ca_address_sk)]
ReadSchema: struct<ca_address_sk:int,ca_state:string>

(2) ColumnarToRow [codegen id : 7]
Input [2]: [ca_address_sk#1, ca_state#2]

(3) Filter [codegen id : 7]
Input [2]: [ca_address_sk#1, ca_state#2]
Condition : isnotnull(ca_address_sk#1)

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

(5) ColumnarToRow [codegen id : 1]
Input [2]: [c_customer_sk#3, c_current_addr_sk#4]

(6) Filter [codegen id : 1]
Input [2]: [c_customer_sk#3, c_current_addr_sk#4]
Condition : (isnotnull(c_current_addr_sk#4) AND isnotnull(c_customer_sk#3))

(7) BroadcastExchange
Input [2]: [c_customer_sk#3, c_current_addr_sk#4]
Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1]

(8) BroadcastHashJoin [codegen id : 7]
Left keys [1]: [ca_address_sk#1]
Right keys [1]: [c_current_addr_sk#4]
Join type: Inner
Join condition: None

(9) Project [codegen id : 7]
Output [2]: [ca_state#2, c_customer_sk#3]
Input [4]: [ca_address_sk#1, ca_state#2, c_customer_sk#3, c_current_addr_sk#4]

(10) Scan parquet spark_catalog.default.store_sales
Output [3]: [ss_item_sk#5, ss_customer_sk#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), IsNotNull(ss_item_sk)]
ReadSchema: struct<ss_item_sk:int,ss_customer_sk:int>

(11) ColumnarToRow [codegen id : 2]
Input [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7]

(12) Filter [codegen id : 2]
Input [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7]
Condition : (isnotnull(ss_customer_sk#6) AND isnotnull(ss_item_sk#5))

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

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

(15) Project [codegen id : 7]
Output [3]: [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7]
Input [5]: [ca_state#2, c_customer_sk#3, ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7]

(16) ReusedExchange [Reuses operator id: 44]
Output [1]: [d_date_sk#9]

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

(18) Project [codegen id : 7]
Output [2]: [ca_state#2, ss_item_sk#5]
Input [4]: [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7, d_date_sk#9]

(19) Scan parquet spark_catalog.default.item
Output [3]: [i_item_sk#10, i_current_price#11, i_category#12]
Batched: true
Location [not included in comparison]/{warehouse_dir}/item]
PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_category), IsNotNull(i_item_sk)]
ReadSchema: struct<i_item_sk:int,i_current_price:decimal(7,2),i_category:string>

(20) ColumnarToRow [codegen id : 6]
Input [3]: [i_item_sk#10, i_current_price#11, i_category#12]

(21) Filter [codegen id : 6]
Input [3]: [i_item_sk#10, i_current_price#11, i_category#12]
Condition : ((isnotnull(i_current_price#11) AND isnotnull(i_category#12)) AND isnotnull(i_item_sk#10))

(22) Scan parquet spark_catalog.default.item
Output [2]: [i_current_price#13, i_category#14]
Batched: true
Location [not included in comparison]/{warehouse_dir}/item]
PushedFilters: [IsNotNull(i_category)]
ReadSchema: struct<i_current_price:decimal(7,2),i_category:string>

(23) ColumnarToRow [codegen id : 4]
Input [2]: [i_current_price#13, i_category#14]

(24) Filter [codegen id : 4]
Input [2]: [i_current_price#13, i_category#14]
Condition : isnotnull(i_category#14)

(25) HashAggregate [codegen id : 4]
Input [2]: [i_current_price#13, i_category#14]
Keys [1]: [i_category#14]
Functions [1]: [partial_avg(UnscaledValue(i_current_price#13))]
Aggregate Attributes [2]: [sum#15, count#16]
Results [3]: [i_category#14, sum#17, count#18]

(26) Exchange
Input [3]: [i_category#14, sum#17, count#18]
Arguments: hashpartitioning(i_category#14, 5), ENSURE_REQUIREMENTS, [plan_id=3]

(27) HashAggregate [codegen id : 5]
Input [3]: [i_category#14, sum#17, count#18]
Keys [1]: [i_category#14]
Functions [1]: [avg(UnscaledValue(i_current_price#13))]
Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#13))#19]
Results [2]: [cast((avg(UnscaledValue(i_current_price#13))#19 / 100.0) as decimal(11,6)) AS avg(i_current_price)#20, i_category#14]

(28) Filter [codegen id : 5]
Input [2]: [avg(i_current_price)#20, i_category#14]
Condition : isnotnull(avg(i_current_price)#20)

(29) BroadcastExchange
Input [2]: [avg(i_current_price)#20, i_category#14]
Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=4]

(30) BroadcastHashJoin [codegen id : 6]
Left keys [1]: [i_category#12]
Right keys [1]: [i_category#14]
Join type: Inner
Join condition: (cast(i_current_price#11 as decimal(14,7)) > (1.2 * avg(i_current_price)#20))

(31) Project [codegen id : 6]
Output [1]: [i_item_sk#10]
Input [5]: [i_item_sk#10, i_current_price#11, i_category#12, avg(i_current_price)#20, i_category#14]

(32) BroadcastExchange
Input [1]: [i_item_sk#10]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5]

(33) BroadcastHashJoin [codegen id : 7]
Left keys [1]: [ss_item_sk#5]
Right keys [1]: [i_item_sk#10]
Join type: Inner
Join condition: None

(34) Project [codegen id : 7]
Output [1]: [ca_state#2]
Input [3]: [ca_state#2, ss_item_sk#5, i_item_sk#10]

(35) HashAggregate [codegen id : 7]
Input [1]: [ca_state#2]
Keys [1]: [ca_state#2]
Functions [1]: [partial_count(1)]
Aggregate Attributes [1]: [count#21]
Results [2]: [ca_state#2, count#22]

(36) Exchange
Input [2]: [ca_state#2, count#22]
Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [plan_id=6]

(37) HashAggregate [codegen id : 8]
Input [2]: [ca_state#2, count#22]
Keys [1]: [ca_state#2]
Functions [1]: [count(1)]
Aggregate Attributes [1]: [count(1)#23]
Results [2]: [ca_state#2 AS state#24, count(1)#23 AS cnt#25]

(38) Filter [codegen id : 8]
Input [2]: [state#24, cnt#25]
Condition : (cnt#25 >= 10)

(39) TakeOrderedAndProject
Input [2]: [state#24, cnt#25]
Arguments: 100, [cnt#25 ASC NULLS FIRST], [state#24, cnt#25]

===== Subqueries =====

Subquery:1 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8
BroadcastExchange (44)
+- * Project (43)
   +- * Filter (42)
      +- * ColumnarToRow (41)
         +- Scan parquet spark_catalog.default.date_dim (40)


(40) Scan parquet spark_catalog.default.date_dim
Output [2]: [d_date_sk#9, d_month_seq#26]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_month_seq), EqualTo(d_month_seq,ScalarSubquery#27), IsNotNull(d_date_sk)]
ReadSchema: struct<d_date_sk:int,d_month_seq:int>

(41) ColumnarToRow [codegen id : 1]
Input [2]: [d_date_sk#9, d_month_seq#26]

(42) Filter [codegen id : 1]
Input [2]: [d_date_sk#9, d_month_seq#26]
Condition : ((isnotnull(d_month_seq#26) AND (d_month_seq#26 = ReusedSubquery Subquery scalar-subquery#27, [id=#7])) AND isnotnull(d_date_sk#9))

(43) Project [codegen id : 1]
Output [1]: [d_date_sk#9]
Input [2]: [d_date_sk#9, d_month_seq#26]

(44) BroadcastExchange
Input [1]: [d_date_sk#9]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8]

Subquery:2 Hosting operator id = 42 Hosting Expression = ReusedSubquery Subquery scalar-subquery#27, [id=#7]

Subquery:3 Hosting operator id = 40 Hosting Expression = Subquery scalar-subquery#27, [id=#7]
* HashAggregate (51)
+- Exchange (50)
   +- * HashAggregate (49)
      +- * Project (48)
         +- * Filter (47)
            +- * ColumnarToRow (46)
               +- Scan parquet spark_catalog.default.date_dim (45)


(45) Scan parquet spark_catalog.default.date_dim
Output [3]: [d_month_seq#28, d_year#29, d_moy#30]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)]
ReadSchema: struct<d_month_seq:int,d_year:int,d_moy:int>

(46) ColumnarToRow [codegen id : 1]
Input [3]: [d_month_seq#28, d_year#29, d_moy#30]

(47) Filter [codegen id : 1]
Input [3]: [d_month_seq#28, d_year#29, d_moy#30]
Condition : (((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2000)) AND (d_moy#30 = 1))

(48) Project [codegen id : 1]
Output [1]: [d_month_seq#28]
Input [3]: [d_month_seq#28, d_year#29, d_moy#30]

(49) HashAggregate [codegen id : 1]
Input [1]: [d_month_seq#28]
Keys [1]: [d_month_seq#28]
Functions: []
Aggregate Attributes: []
Results [1]: [d_month_seq#28]

(50) Exchange
Input [1]: [d_month_seq#28]
Arguments: hashpartitioning(d_month_seq#28, 5), ENSURE_REQUIREMENTS, [plan_id=9]

(51) HashAggregate [codegen id : 2]
Input [1]: [d_month_seq#28]
Keys [1]: [d_month_seq#28]
Functions: []
Aggregate Attributes: []
Results [1]: [d_month_seq#28]


