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


(1) Scan parquet spark_catalog.default.store_returns
Output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4]
Batched: true
Location: InMemoryFileIndex []
PartitionFilters: [isnotnull(sr_returned_date_sk#4), dynamicpruningexpression(sr_returned_date_sk#4 IN dynamicpruning#5)]
PushedFilters: [IsNotNull(sr_store_sk), IsNotNull(sr_customer_sk)]
ReadSchema: struct<sr_customer_sk:int,sr_store_sk:int,sr_return_amt:decimal(7,2)>

(2) ColumnarToRow [codegen id : 2]
Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4]

(3) Filter [codegen id : 2]
Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4]
Condition : (isnotnull(sr_store_sk#2) AND isnotnull(sr_customer_sk#1))

(4) ReusedExchange [Reuses operator id: 45]
Output [1]: [d_date_sk#6]

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

(6) Project [codegen id : 2]
Output [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3]
Input [5]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4, d_date_sk#6]

(7) HashAggregate [codegen id : 2]
Input [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3]
Keys [2]: [sr_customer_sk#1, sr_store_sk#2]
Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#3))]
Aggregate Attributes [1]: [sum#7]
Results [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8]

(8) Exchange
Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8]
Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, [plan_id=1]

(9) HashAggregate [codegen id : 9]
Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8]
Keys [2]: [sr_customer_sk#1, sr_store_sk#2]
Functions [1]: [sum(UnscaledValue(sr_return_amt#3))]
Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#3))#9]
Results [3]: [sr_customer_sk#1 AS ctr_customer_sk#10, sr_store_sk#2 AS ctr_store_sk#11, MakeDecimal(sum(UnscaledValue(sr_return_amt#3))#9,17,2) AS ctr_total_return#12]

(10) Filter [codegen id : 9]
Input [3]: [ctr_customer_sk#10, ctr_store_sk#11, ctr_total_return#12]
Condition : isnotnull(ctr_total_return#12)

(11) Scan parquet spark_catalog.default.store_returns
Output [4]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16]
Batched: true
Location: InMemoryFileIndex []
PartitionFilters: [isnotnull(sr_returned_date_sk#16), dynamicpruningexpression(sr_returned_date_sk#16 IN dynamicpruning#5)]
PushedFilters: [IsNotNull(sr_store_sk)]
ReadSchema: struct<sr_customer_sk:int,sr_store_sk:int,sr_return_amt:decimal(7,2)>

(12) ColumnarToRow [codegen id : 4]
Input [4]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16]

(13) Filter [codegen id : 4]
Input [4]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16]
Condition : isnotnull(sr_store_sk#14)

(14) ReusedExchange [Reuses operator id: 45]
Output [1]: [d_date_sk#17]

(15) BroadcastHashJoin [codegen id : 4]
Left keys [1]: [sr_returned_date_sk#16]
Right keys [1]: [d_date_sk#17]
Join type: Inner
Join condition: None

(16) Project [codegen id : 4]
Output [3]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15]
Input [5]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16, d_date_sk#17]

(17) HashAggregate [codegen id : 4]
Input [3]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15]
Keys [2]: [sr_customer_sk#13, sr_store_sk#14]
Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#15))]
Aggregate Attributes [1]: [sum#18]
Results [3]: [sr_customer_sk#13, sr_store_sk#14, sum#19]

(18) Exchange
Input [3]: [sr_customer_sk#13, sr_store_sk#14, sum#19]
Arguments: hashpartitioning(sr_customer_sk#13, sr_store_sk#14, 5), ENSURE_REQUIREMENTS, [plan_id=2]

(19) HashAggregate [codegen id : 5]
Input [3]: [sr_customer_sk#13, sr_store_sk#14, sum#19]
Keys [2]: [sr_customer_sk#13, sr_store_sk#14]
Functions [1]: [sum(UnscaledValue(sr_return_amt#15))]
Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#15))#9]
Results [2]: [sr_store_sk#14 AS ctr_store_sk#20, MakeDecimal(sum(UnscaledValue(sr_return_amt#15))#9,17,2) AS ctr_total_return#21]

(20) HashAggregate [codegen id : 5]
Input [2]: [ctr_store_sk#20, ctr_total_return#21]
Keys [1]: [ctr_store_sk#20]
Functions [1]: [partial_avg(ctr_total_return#21)]
Aggregate Attributes [2]: [sum#22, count#23]
Results [3]: [ctr_store_sk#20, sum#24, count#25]

(21) Exchange
Input [3]: [ctr_store_sk#20, sum#24, count#25]
Arguments: hashpartitioning(ctr_store_sk#20, 5), ENSURE_REQUIREMENTS, [plan_id=3]

(22) HashAggregate [codegen id : 6]
Input [3]: [ctr_store_sk#20, sum#24, count#25]
Keys [1]: [ctr_store_sk#20]
Functions [1]: [avg(ctr_total_return#21)]
Aggregate Attributes [1]: [avg(ctr_total_return#21)#26]
Results [2]: [(avg(ctr_total_return#21)#26 * 1.2) AS (avg(ctr_total_return) * 1.2)#27, ctr_store_sk#20]

(23) Filter [codegen id : 6]
Input [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_store_sk#20]
Condition : isnotnull((avg(ctr_total_return) * 1.2)#27)

(24) BroadcastExchange
Input [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_store_sk#20]
Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=4]

(25) BroadcastHashJoin [codegen id : 9]
Left keys [1]: [ctr_store_sk#11]
Right keys [1]: [ctr_store_sk#20]
Join type: Inner
Join condition: (cast(ctr_total_return#12 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#27)

(26) Project [codegen id : 9]
Output [2]: [ctr_customer_sk#10, ctr_store_sk#11]
Input [5]: [ctr_customer_sk#10, ctr_store_sk#11, ctr_total_return#12, (avg(ctr_total_return) * 1.2)#27, ctr_store_sk#20]

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

(28) ColumnarToRow [codegen id : 7]
Input [2]: [s_store_sk#28, s_state#29]

(29) Filter [codegen id : 7]
Input [2]: [s_store_sk#28, s_state#29]
Condition : ((isnotnull(s_state#29) AND (s_state#29 = TN)) AND isnotnull(s_store_sk#28))

(30) Project [codegen id : 7]
Output [1]: [s_store_sk#28]
Input [2]: [s_store_sk#28, s_state#29]

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

(32) BroadcastHashJoin [codegen id : 9]
Left keys [1]: [ctr_store_sk#11]
Right keys [1]: [s_store_sk#28]
Join type: Inner
Join condition: None

(33) Project [codegen id : 9]
Output [1]: [ctr_customer_sk#10]
Input [3]: [ctr_customer_sk#10, ctr_store_sk#11, s_store_sk#28]

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

(35) ColumnarToRow [codegen id : 8]
Input [2]: [c_customer_sk#30, c_customer_id#31]

(36) Filter [codegen id : 8]
Input [2]: [c_customer_sk#30, c_customer_id#31]
Condition : isnotnull(c_customer_sk#30)

(37) BroadcastExchange
Input [2]: [c_customer_sk#30, c_customer_id#31]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6]

(38) BroadcastHashJoin [codegen id : 9]
Left keys [1]: [ctr_customer_sk#10]
Right keys [1]: [c_customer_sk#30]
Join type: Inner
Join condition: None

(39) Project [codegen id : 9]
Output [1]: [c_customer_id#31]
Input [3]: [ctr_customer_sk#10, c_customer_sk#30, c_customer_id#31]

(40) TakeOrderedAndProject
Input [1]: [c_customer_id#31]
Arguments: 100, [c_customer_id#31 ASC NULLS FIRST], [c_customer_id#31]

===== Subqueries =====

Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5
BroadcastExchange (45)
+- * Project (44)
   +- * Filter (43)
      +- * ColumnarToRow (42)
         +- Scan parquet spark_catalog.default.date_dim (41)


(41) Scan parquet spark_catalog.default.date_dim
Output [2]: [d_date_sk#6, d_year#32]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)]
ReadSchema: struct<d_date_sk:int,d_year:int>

(42) ColumnarToRow [codegen id : 1]
Input [2]: [d_date_sk#6, d_year#32]

(43) Filter [codegen id : 1]
Input [2]: [d_date_sk#6, d_year#32]
Condition : ((isnotnull(d_year#32) AND (d_year#32 = 2000)) AND isnotnull(d_date_sk#6))

(44) Project [codegen id : 1]
Output [1]: [d_date_sk#6]
Input [2]: [d_date_sk#6, d_year#32]

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

Subquery:2 Hosting operator id = 11 Hosting Expression = sr_returned_date_sk#16 IN dynamicpruning#5


