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


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

(2) ColumnarToRow [codegen id : 9]
Input [2]: [s_store_sk#1, s_store_name#2]

(3) Filter [codegen id : 9]
Input [2]: [s_store_sk#1, s_store_name#2]
Condition : isnotnull(s_store_sk#1)

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

(5) ColumnarToRow [codegen id : 2]
Input [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6]

(6) Filter [codegen id : 2]
Input [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6]
Condition : (isnotnull(ss_store_sk#4) AND isnotnull(ss_item_sk#3))

(7) ReusedExchange [Reuses operator id: 44]
Output [1]: [d_date_sk#8]

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

(9) Project [codegen id : 2]
Output [3]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5]
Input [5]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6, d_date_sk#8]

(10) HashAggregate [codegen id : 2]
Input [3]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5]
Keys [2]: [ss_store_sk#4, ss_item_sk#3]
Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#5))]
Aggregate Attributes [1]: [sum#9]
Results [3]: [ss_store_sk#4, ss_item_sk#3, sum#10]

(11) Exchange
Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10]
Arguments: hashpartitioning(ss_store_sk#4, ss_item_sk#3, 5), ENSURE_REQUIREMENTS, [plan_id=1]

(12) HashAggregate [codegen id : 3]
Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10]
Keys [2]: [ss_store_sk#4, ss_item_sk#3]
Functions [1]: [sum(UnscaledValue(ss_sales_price#5))]
Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#5))#11]
Results [3]: [ss_store_sk#4, ss_item_sk#3, MakeDecimal(sum(UnscaledValue(ss_sales_price#5))#11,17,2) AS revenue#12]

(13) Filter [codegen id : 3]
Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#12]
Condition : isnotnull(revenue#12)

(14) BroadcastExchange
Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#12]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2]

(15) BroadcastHashJoin [codegen id : 9]
Left keys [1]: [s_store_sk#1]
Right keys [1]: [ss_store_sk#4]
Join type: Inner
Join condition: None

(16) Project [codegen id : 9]
Output [4]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12]
Input [5]: [s_store_sk#1, s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12]

(17) Scan parquet spark_catalog.default.item
Output [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17]
Batched: true
Location [not included in comparison]/{warehouse_dir}/item]
PushedFilters: [IsNotNull(i_item_sk)]
ReadSchema: struct<i_item_sk:int,i_item_desc:string,i_current_price:decimal(7,2),i_wholesale_cost:decimal(7,2),i_brand:string>

(18) ColumnarToRow [codegen id : 4]
Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17]

(19) Filter [codegen id : 4]
Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17]
Condition : isnotnull(i_item_sk#13)

(20) BroadcastExchange
Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3]

(21) BroadcastHashJoin [codegen id : 9]
Left keys [1]: [ss_item_sk#3]
Right keys [1]: [i_item_sk#13]
Join type: Inner
Join condition: None

(22) Project [codegen id : 9]
Output [7]: [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]
Input [9]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12, i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17]

(23) Scan parquet spark_catalog.default.store_sales
Output [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21]
Batched: true
Location: InMemoryFileIndex []
PartitionFilters: [isnotnull(ss_sold_date_sk#21), dynamicpruningexpression(ss_sold_date_sk#21 IN dynamicpruning#7)]
PushedFilters: [IsNotNull(ss_store_sk)]
ReadSchema: struct<ss_item_sk:int,ss_store_sk:int,ss_sales_price:decimal(7,2)>

(24) ColumnarToRow [codegen id : 6]
Input [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21]

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

(26) ReusedExchange [Reuses operator id: 44]
Output [1]: [d_date_sk#22]

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

(28) Project [codegen id : 6]
Output [3]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20]
Input [5]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21, d_date_sk#22]

(29) HashAggregate [codegen id : 6]
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))]
Aggregate Attributes [1]: [sum#23]
Results [3]: [ss_store_sk#19, ss_item_sk#18, sum#24]

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

(31) HashAggregate [codegen id : 7]
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))#25]
Results [2]: [ss_store_sk#19, MakeDecimal(sum(UnscaledValue(ss_sales_price#20))#25,17,2) AS revenue#26]

(32) HashAggregate [codegen id : 7]
Input [2]: [ss_store_sk#19, revenue#26]
Keys [1]: [ss_store_sk#19]
Functions [1]: [partial_avg(revenue#26)]
Aggregate Attributes [2]: [sum#27, count#28]
Results [3]: [ss_store_sk#19, sum#29, count#30]

(33) Exchange
Input [3]: [ss_store_sk#19, sum#29, count#30]
Arguments: hashpartitioning(ss_store_sk#19, 5), ENSURE_REQUIREMENTS, [plan_id=5]

(34) HashAggregate [codegen id : 8]
Input [3]: [ss_store_sk#19, sum#29, count#30]
Keys [1]: [ss_store_sk#19]
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]

(35) Filter [codegen id : 8]
Input [2]: [ss_store_sk#19, ave#32]
Condition : isnotnull(ave#32)

(36) BroadcastExchange
Input [2]: [ss_store_sk#19, ave#32]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6]

(37) BroadcastHashJoin [codegen id : 9]
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#32))

(38) Project [codegen id : 9]
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#32]

(39) 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 (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#8, d_month_seq#33]
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<d_date_sk:int,d_month_seq:int>

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

(42) Filter [codegen id : 1]
Input [2]: [d_date_sk#8, d_month_seq#33]
Condition : (((isnotnull(d_month_seq#33) AND (d_month_seq#33 >= 1176)) AND (d_month_seq#33 <= 1187)) AND isnotnull(d_date_sk#8))

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

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

Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#21 IN dynamicpruning#7


