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


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

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

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

(4) ReusedExchange [Reuses operator id: 38]
Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10]

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

(6) Project [codegen id : 3]
Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10]
Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#8, d_moy#9, d_qoy#10]

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

(8) ColumnarToRow [codegen id : 2]
Input [2]: [s_store_sk#11, s_store_id#12]

(9) Filter [codegen id : 2]
Input [2]: [s_store_sk#11, s_store_id#12]
Condition : isnotnull(s_store_sk#11)

(10) BroadcastExchange
Input [2]: [s_store_sk#11, s_store_id#12]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1]

(11) BroadcastHashJoin [codegen id : 3]
Left keys [1]: [ss_store_sk#2]
Right keys [1]: [s_store_sk#11]
Join type: Inner
Join condition: None

(12) Project [codegen id : 3]
Output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#12]
Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_sk#11, s_store_id#12]

(13) Exchange
Input [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#12]
Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=2]

(14) Sort [codegen id : 4]
Input [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#12]
Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0

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

(16) ColumnarToRow [codegen id : 5]
Input [5]: [i_item_sk#13, i_brand#14, i_class#15, i_category#16, i_product_name#17]

(17) Filter [codegen id : 5]
Input [5]: [i_item_sk#13, i_brand#14, i_class#15, i_category#16, i_product_name#17]
Condition : isnotnull(i_item_sk#13)

(18) Exchange
Input [5]: [i_item_sk#13, i_brand#14, i_class#15, i_category#16, i_product_name#17]
Arguments: hashpartitioning(i_item_sk#13, 5), ENSURE_REQUIREMENTS, [plan_id=3]

(19) Sort [codegen id : 6]
Input [5]: [i_item_sk#13, i_brand#14, i_class#15, i_category#16, i_product_name#17]
Arguments: [i_item_sk#13 ASC NULLS FIRST], false, 0

(20) SortMergeJoin [codegen id : 7]
Left keys [1]: [ss_item_sk#1]
Right keys [1]: [i_item_sk#13]
Join type: Inner
Join condition: None

(21) Project [codegen id : 7]
Output [10]: [ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12]
Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#12, i_item_sk#13, i_brand#14, i_class#15, i_category#16, i_product_name#17]

(22) Expand [codegen id : 7]
Input [10]: [ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12]
Arguments: [[ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, 0], [ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, null, 1], [ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, null, null, 3], [ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, null, null, null, 7], [ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, i_brand#14, i_product_name#17, null, null, null, null, 15], [ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, i_brand#14, null, null, null, null, null, 31], [ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, null, null, null, null, null, null, 63], [ss_quantity#3, ss_sales_price#4, i_category#16, null, null, null, null, null, null, null, 127], [ss_quantity#3, ss_sales_price#4, null, null, null, null, null, null, null, null, 255]], [ss_quantity#3, ss_sales_price#4, i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26]

(23) HashAggregate [codegen id : 7]
Input [11]: [ss_quantity#3, ss_sales_price#4, i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26]
Keys [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26]
Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))]
Aggregate Attributes [2]: [sum#27, isEmpty#28]
Results [11]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, sum#29, isEmpty#30]

(24) Exchange
Input [11]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, sum#29, isEmpty#30]
Arguments: hashpartitioning(i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, 5), ENSURE_REQUIREMENTS, [plan_id=4]

(25) HashAggregate [codegen id : 8]
Input [11]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, sum#29, isEmpty#30]
Keys [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26]
Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))]
Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#31]
Results [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#31 AS sumsales#32]

(26) Sort [codegen id : 8]
Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32]
Arguments: [i_category#18 ASC NULLS FIRST, sumsales#32 DESC NULLS LAST], false, 0

(27) WindowGroupLimit
Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32]
Arguments: [i_category#18], [sumsales#32 DESC NULLS LAST], rank(sumsales#32), 100, Partial

(28) Exchange
Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32]
Arguments: hashpartitioning(i_category#18, 5), ENSURE_REQUIREMENTS, [plan_id=5]

(29) Sort [codegen id : 9]
Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32]
Arguments: [i_category#18 ASC NULLS FIRST, sumsales#32 DESC NULLS LAST], false, 0

(30) WindowGroupLimit
Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32]
Arguments: [i_category#18], [sumsales#32 DESC NULLS LAST], rank(sumsales#32), 100, Final

(31) Window
Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32]
Arguments: [rank(sumsales#32) windowspecdefinition(i_category#18, sumsales#32 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#33], [i_category#18], [sumsales#32 DESC NULLS LAST]

(32) Filter [codegen id : 10]
Input [10]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32, rk#33]
Condition : (rk#33 <= 100)

(33) TakeOrderedAndProject
Input [10]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32, rk#33]
Arguments: 100, [i_category#18 ASC NULLS FIRST, i_class#19 ASC NULLS FIRST, i_brand#20 ASC NULLS FIRST, i_product_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_qoy#23 ASC NULLS FIRST, d_moy#24 ASC NULLS FIRST, s_store_id#25 ASC NULLS FIRST, sumsales#32 ASC NULLS FIRST, rk#33 ASC NULLS FIRST], [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32, rk#33]

===== Subqueries =====

Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6
BroadcastExchange (38)
+- * Project (37)
   +- * Filter (36)
      +- * ColumnarToRow (35)
         +- Scan parquet spark_catalog.default.date_dim (34)


(34) Scan parquet spark_catalog.default.date_dim
Output [5]: [d_date_sk#7, d_month_seq#34, d_year#8, d_moy#9, d_qoy#10]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)]
ReadSchema: struct<d_date_sk:int,d_month_seq:int,d_year:int,d_moy:int,d_qoy:int>

(35) ColumnarToRow [codegen id : 1]
Input [5]: [d_date_sk#7, d_month_seq#34, d_year#8, d_moy#9, d_qoy#10]

(36) Filter [codegen id : 1]
Input [5]: [d_date_sk#7, d_month_seq#34, d_year#8, d_moy#9, d_qoy#10]
Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= 1200)) AND (d_month_seq#34 <= 1211)) AND isnotnull(d_date_sk#7))

(37) Project [codegen id : 1]
Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10]
Input [5]: [d_date_sk#7, d_month_seq#34, d_year#8, d_moy#9, d_qoy#10]

(38) BroadcastExchange
Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6]


