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


(1) Scan parquet spark_catalog.default.store_sales
Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4]
Batched: true
Location [not included in comparison]/{warehouse_dir}/store_sales]
PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4)]
ReadSchema: struct<ss_item_sk:int,ss_store_sk:int,ss_net_profit:decimal(7,2)>

(2) ColumnarToRow [codegen id : 1]
Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4]

(3) Filter [codegen id : 1]
Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4]
Condition : (isnotnull(ss_store_sk#2) AND (ss_store_sk#2 = 4))

(4) Project [codegen id : 1]
Output [2]: [ss_item_sk#1, ss_net_profit#3]
Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4]

(5) HashAggregate [codegen id : 1]
Input [2]: [ss_item_sk#1, ss_net_profit#3]
Keys [1]: [ss_item_sk#1]
Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#3))]
Aggregate Attributes [2]: [sum#5, count#6]
Results [3]: [ss_item_sk#1, sum#7, count#8]

(6) Exchange
Input [3]: [ss_item_sk#1, sum#7, count#8]
Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1]

(7) HashAggregate [codegen id : 2]
Input [3]: [ss_item_sk#1, sum#7, count#8]
Keys [1]: [ss_item_sk#1]
Functions [1]: [avg(UnscaledValue(ss_net_profit#3))]
Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#3))#9]
Results [2]: [ss_item_sk#1 AS item_sk#10, cast((avg(UnscaledValue(ss_net_profit#3))#9 / 100.0) as decimal(11,6)) AS rank_col#11]

(8) Filter [codegen id : 2]
Input [2]: [item_sk#10, rank_col#11]
Condition : (isnotnull(rank_col#11) AND (cast(rank_col#11 as decimal(13,7)) > (0.9 * Subquery scalar-subquery#12, [id=#2])))

(9) Sort [codegen id : 2]
Input [2]: [item_sk#10, rank_col#11]
Arguments: [rank_col#11 ASC NULLS FIRST], false, 0

(10) WindowGroupLimit
Input [2]: [item_sk#10, rank_col#11]
Arguments: [rank_col#11 ASC NULLS FIRST], rank(rank_col#11), 10, Partial

(11) Exchange
Input [2]: [item_sk#10, rank_col#11]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3]

(12) Sort [codegen id : 3]
Input [2]: [item_sk#10, rank_col#11]
Arguments: [rank_col#11 ASC NULLS FIRST], false, 0

(13) WindowGroupLimit
Input [2]: [item_sk#10, rank_col#11]
Arguments: [rank_col#11 ASC NULLS FIRST], rank(rank_col#11), 10, Final

(14) Window
Input [2]: [item_sk#10, rank_col#11]
Arguments: [rank(rank_col#11) windowspecdefinition(rank_col#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#13], [rank_col#11 ASC NULLS FIRST]

(15) Filter [codegen id : 4]
Input [3]: [item_sk#10, rank_col#11, rnk#13]
Condition : ((rnk#13 < 11) AND isnotnull(item_sk#10))

(16) Project [codegen id : 4]
Output [2]: [item_sk#10, rnk#13]
Input [3]: [item_sk#10, rank_col#11, rnk#13]

(17) Exchange
Input [2]: [item_sk#10, rnk#13]
Arguments: hashpartitioning(rnk#13, 5), ENSURE_REQUIREMENTS, [plan_id=4]

(18) Sort [codegen id : 5]
Input [2]: [item_sk#10, rnk#13]
Arguments: [rnk#13 ASC NULLS FIRST], false, 0

(19) ReusedExchange [Reuses operator id: 6]
Output [3]: [ss_item_sk#14, sum#15, count#16]

(20) HashAggregate [codegen id : 7]
Input [3]: [ss_item_sk#14, sum#15, count#16]
Keys [1]: [ss_item_sk#14]
Functions [1]: [avg(UnscaledValue(ss_net_profit#17))]
Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#17))#18]
Results [2]: [ss_item_sk#14 AS item_sk#19, cast((avg(UnscaledValue(ss_net_profit#17))#18 / 100.0) as decimal(11,6)) AS rank_col#20]

(21) Filter [codegen id : 7]
Input [2]: [item_sk#19, rank_col#20]
Condition : (isnotnull(rank_col#20) AND (cast(rank_col#20 as decimal(13,7)) > (0.9 * ReusedSubquery Subquery scalar-subquery#12, [id=#2])))

(22) Sort [codegen id : 7]
Input [2]: [item_sk#19, rank_col#20]
Arguments: [rank_col#20 DESC NULLS LAST], false, 0

(23) WindowGroupLimit
Input [2]: [item_sk#19, rank_col#20]
Arguments: [rank_col#20 DESC NULLS LAST], rank(rank_col#20), 10, Partial

(24) Exchange
Input [2]: [item_sk#19, rank_col#20]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5]

(25) Sort [codegen id : 8]
Input [2]: [item_sk#19, rank_col#20]
Arguments: [rank_col#20 DESC NULLS LAST], false, 0

(26) WindowGroupLimit
Input [2]: [item_sk#19, rank_col#20]
Arguments: [rank_col#20 DESC NULLS LAST], rank(rank_col#20), 10, Final

(27) Window
Input [2]: [item_sk#19, rank_col#20]
Arguments: [rank(rank_col#20) windowspecdefinition(rank_col#20 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#21], [rank_col#20 DESC NULLS LAST]

(28) Filter [codegen id : 9]
Input [3]: [item_sk#19, rank_col#20, rnk#21]
Condition : ((rnk#21 < 11) AND isnotnull(item_sk#19))

(29) Project [codegen id : 9]
Output [2]: [item_sk#19, rnk#21]
Input [3]: [item_sk#19, rank_col#20, rnk#21]

(30) Exchange
Input [2]: [item_sk#19, rnk#21]
Arguments: hashpartitioning(rnk#21, 5), ENSURE_REQUIREMENTS, [plan_id=6]

(31) Sort [codegen id : 10]
Input [2]: [item_sk#19, rnk#21]
Arguments: [rnk#21 ASC NULLS FIRST], false, 0

(32) SortMergeJoin [codegen id : 13]
Left keys [1]: [rnk#13]
Right keys [1]: [rnk#21]
Join type: Inner
Join condition: None

(33) Project [codegen id : 13]
Output [3]: [item_sk#10, rnk#13, item_sk#19]
Input [4]: [item_sk#10, rnk#13, item_sk#19, rnk#21]

(34) Scan parquet spark_catalog.default.item
Output [2]: [i_item_sk#22, i_product_name#23]
Batched: true
Location [not included in comparison]/{warehouse_dir}/item]
PushedFilters: [IsNotNull(i_item_sk)]
ReadSchema: struct<i_item_sk:int,i_product_name:string>

(35) ColumnarToRow [codegen id : 11]
Input [2]: [i_item_sk#22, i_product_name#23]

(36) Filter [codegen id : 11]
Input [2]: [i_item_sk#22, i_product_name#23]
Condition : isnotnull(i_item_sk#22)

(37) BroadcastExchange
Input [2]: [i_item_sk#22, i_product_name#23]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7]

(38) BroadcastHashJoin [codegen id : 13]
Left keys [1]: [item_sk#10]
Right keys [1]: [i_item_sk#22]
Join type: Inner
Join condition: None

(39) Project [codegen id : 13]
Output [3]: [rnk#13, item_sk#19, i_product_name#23]
Input [5]: [item_sk#10, rnk#13, item_sk#19, i_item_sk#22, i_product_name#23]

(40) ReusedExchange [Reuses operator id: 37]
Output [2]: [i_item_sk#24, i_product_name#25]

(41) BroadcastHashJoin [codegen id : 13]
Left keys [1]: [item_sk#19]
Right keys [1]: [i_item_sk#24]
Join type: Inner
Join condition: None

(42) Project [codegen id : 13]
Output [3]: [rnk#13, i_product_name#23 AS best_performing#26, i_product_name#25 AS worst_performing#27]
Input [5]: [rnk#13, item_sk#19, i_product_name#23, i_item_sk#24, i_product_name#25]

(43) TakeOrderedAndProject
Input [3]: [rnk#13, best_performing#26, worst_performing#27]
Arguments: 100, [rnk#13 ASC NULLS FIRST], [rnk#13, best_performing#26, worst_performing#27]

===== Subqueries =====

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


(44) Scan parquet spark_catalog.default.store_sales
Output [4]: [ss_addr_sk#28, ss_store_sk#29, ss_net_profit#30, ss_sold_date_sk#31]
Batched: true
Location [not included in comparison]/{warehouse_dir}/store_sales]
PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)]
ReadSchema: struct<ss_addr_sk:int,ss_store_sk:int,ss_net_profit:decimal(7,2)>

(45) ColumnarToRow [codegen id : 1]
Input [4]: [ss_addr_sk#28, ss_store_sk#29, ss_net_profit#30, ss_sold_date_sk#31]

(46) Filter [codegen id : 1]
Input [4]: [ss_addr_sk#28, ss_store_sk#29, ss_net_profit#30, ss_sold_date_sk#31]
Condition : ((isnotnull(ss_store_sk#29) AND (ss_store_sk#29 = 4)) AND isnull(ss_addr_sk#28))

(47) Project [codegen id : 1]
Output [2]: [ss_store_sk#29, ss_net_profit#30]
Input [4]: [ss_addr_sk#28, ss_store_sk#29, ss_net_profit#30, ss_sold_date_sk#31]

(48) HashAggregate [codegen id : 1]
Input [2]: [ss_store_sk#29, ss_net_profit#30]
Keys [1]: [ss_store_sk#29]
Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#30))]
Aggregate Attributes [2]: [sum#32, count#33]
Results [3]: [ss_store_sk#29, sum#34, count#35]

(49) Exchange
Input [3]: [ss_store_sk#29, sum#34, count#35]
Arguments: hashpartitioning(ss_store_sk#29, 5), ENSURE_REQUIREMENTS, [plan_id=8]

(50) HashAggregate [codegen id : 2]
Input [3]: [ss_store_sk#29, sum#34, count#35]
Keys [1]: [ss_store_sk#29]
Functions [1]: [avg(UnscaledValue(ss_net_profit#30))]
Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#30))#36]
Results [1]: [cast((avg(UnscaledValue(ss_net_profit#30))#36 / 100.0) as decimal(11,6)) AS rank_col#37]

Subquery:2 Hosting operator id = 21 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#2]


