== Physical Plan ==
TakeOrderedAndProject (43)
+- * Filter (42)
   +- * HashAggregate (41)
      +- Exchange (40)
         +- * HashAggregate (39)
            +- * Project (38)
               +- * BroadcastHashJoin Inner BuildRight (37)
                  :- * Project (22)
                  :  +- * BroadcastHashJoin Inner BuildRight (21)
                  :     :- * 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)
                  :     +- BroadcastExchange (20)
                  :        +- * Project (19)
                  :           +- * Filter (18)
                  :              +- * ColumnarToRow (17)
                  :                 +- Scan parquet spark_catalog.default.date_dim (16)
                  +- BroadcastExchange (36)
                     +- * Project (35)
                        +- * BroadcastHashJoin Inner BuildRight (34)
                           :- * Filter (25)
                           :  +- * ColumnarToRow (24)
                           :     +- Scan parquet spark_catalog.default.item (23)
                           +- BroadcastExchange (33)
                              +- * Filter (32)
                                 +- * HashAggregate (31)
                                    +- Exchange (30)
                                       +- * HashAggregate (29)
                                          +- * Filter (28)
                                             +- * ColumnarToRow (27)
                                                +- Scan parquet spark_catalog.default.item (26)


(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)]
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) 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), IsNotNull(d_date_sk)]
ReadSchema: struct<d_date_sk:int,d_month_seq:int>

(17) ColumnarToRow [codegen id : 3]
Input [2]: [d_date_sk#8, d_month_seq#9]

(18) Filter [codegen id : 3]
Input [2]: [d_date_sk#8, d_month_seq#9]
Condition : ((isnotnull(d_month_seq#9) AND (d_month_seq#9 = Subquery scalar-subquery#10, [id=#11])) AND isnotnull(d_date_sk#8))

(19) Project [codegen id : 3]
Output [1]: [d_date_sk#8]
Input [2]: [d_date_sk#8, d_month_seq#9]

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

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

(22) 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#8]

(23) Scan parquet spark_catalog.default.item
Output [3]: [i_item_sk#12, i_current_price#13, i_category#14]
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>

(24) ColumnarToRow [codegen id : 6]
Input [3]: [i_item_sk#12, i_current_price#13, i_category#14]

(25) Filter [codegen id : 6]
Input [3]: [i_item_sk#12, i_current_price#13, i_category#14]
Condition : ((isnotnull(i_current_price#13) AND isnotnull(i_category#14)) AND isnotnull(i_item_sk#12))

(26) Scan parquet spark_catalog.default.item
Output [2]: [i_current_price#15, i_category#16]
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>

(27) ColumnarToRow [codegen id : 4]
Input [2]: [i_current_price#15, i_category#16]

(28) Filter [codegen id : 4]
Input [2]: [i_current_price#15, i_category#16]
Condition : isnotnull(i_category#16)

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

(30) Exchange
Input [3]: [i_category#16, sum#19, count#20]
Arguments: hashpartitioning(i_category#16, 5), ENSURE_REQUIREMENTS, [plan_id=4]

(31) HashAggregate [codegen id : 5]
Input [3]: [i_category#16, sum#19, count#20]
Keys [1]: [i_category#16]
Functions [1]: [avg(UnscaledValue(i_current_price#15))]
Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#15))#21]
Results [2]: [cast((avg(UnscaledValue(i_current_price#15))#21 / 100.0) as decimal(11,6)) AS avg(i_current_price)#22, i_category#16]

(32) Filter [codegen id : 5]
Input [2]: [avg(i_current_price)#22, i_category#16]
Condition : isnotnull(avg(i_current_price)#22)

(33) BroadcastExchange
Input [2]: [avg(i_current_price)#22, i_category#16]
Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=5]

(34) BroadcastHashJoin [codegen id : 6]
Left keys [1]: [i_category#14]
Right keys [1]: [i_category#16]
Join type: Inner
Join condition: (cast(i_current_price#13 as decimal(14,7)) > (1.2 * avg(i_current_price)#22))

(35) Project [codegen id : 6]
Output [1]: [i_item_sk#12]
Input [5]: [i_item_sk#12, i_current_price#13, i_category#14, avg(i_current_price)#22, i_category#16]

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

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

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

(39) HashAggregate [codegen id : 7]
Input [1]: [ca_state#2]
Keys [1]: [ca_state#2]
Functions [1]: [partial_count(1)]
Aggregate Attributes [1]: [count#23]
Results [2]: [ca_state#2, count#24]

(40) Exchange
Input [2]: [ca_state#2, count#24]
Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [plan_id=7]

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

(42) Filter [codegen id : 8]
Input [2]: [state#26, cnt#27]
Condition : (cnt#27 >= 10)

(43) TakeOrderedAndProject
Input [2]: [state#26, cnt#27]
Arguments: 100, [cnt#27 ASC NULLS FIRST], [state#26, cnt#27]

===== Subqueries =====

Subquery:1 Hosting operator id = 18 Hosting Expression = Subquery scalar-subquery#10, [id=#11]
* HashAggregate (50)
+- Exchange (49)
   +- * HashAggregate (48)
      +- * Project (47)
         +- * Filter (46)
            +- * ColumnarToRow (45)
               +- Scan parquet spark_catalog.default.date_dim (44)


(44) 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>

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

(46) 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))

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

(48) 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]

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

(50) 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]


