You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2020/06/21 08:00:28 UTC

[GitHub] [spark] peter-toth opened a new pull request #28885: [WIP][SPARK-29375][SPARK-28940][SQL] Whole plan exchange and subquery reuse

peter-toth opened a new pull request #28885:
URL: https://github.com/apache/spark/pull/28885


   ### What changes were proposed in this pull request?
   This PR:
   - Unifies exchange and subquery reuse into one rule as the current separate `ReuseExchange` and `ReuseSubquery` rule can interfere. One example of the issue is the TPCDS Q14a query where the definition of the exchange `id=13278` is missing but there are 3 reuse reference to it. In the case of Q14a `ReuseExchange` rule inserted the reuse references correctly but then the `ReuseSubquery` rule altered the original `id=13278` node to `id=15038` by inserting `ReusedSubquery` nodes under it. This PR fixes the issue by combining the 2 similar rules that insert references into the plan:
   ```
   == Physical Plan ==
   TakeOrderedAndProject(limit=100, orderBy=[channel#3384 ASC NULLS FIRST,i_brand_id#3385 ASC NULLS FIRST,i_class_id#3386 ASC NULLS FIRST,i_category_id#3387 ASC NULLS FIRST], output=[channel#3384,i_brand_id#3385,i_class_id#3386,i_category_id#3387,sum(sales)#3374,sum(number_sales)#3375L])
   +- *(119) HashAggregate(keys=[channel#3384, i_brand_id#3385, i_class_id#3386, i_category_id#3387, spark_grouping_id#3383L], functions=[sum(sales#3318), sum(number_sales#3319L)])
      +- Exchange hashpartitioning(channel#3384, i_brand_id#3385, i_class_id#3386, i_category_id#3387, spark_grouping_id#3383L, 4), true, [id=#15231]
         +- *(118) HashAggregate(keys=[channel#3384, i_brand_id#3385, i_class_id#3386, i_category_id#3387, spark_grouping_id#3383L], functions=[partial_sum(sales#3318), partial_sum(number_sales#3319L)])
            +- *(118) Expand [List(sales#3318, number_sales#3319L, channel#3317, i_brand_id#499, i_class_id#501, i_category_id#503, 0), List(sales#3318, number_sales#3319L, channel#3317, i_brand_id#499, i_class_id#501, null, 1), List(sales#3318, number_sales#3319L, channel#3317, i_brand_id#499, null, null, 3), List(sales#3318, number_sales#3319L, channel#3317, null, null, null, 7), List(sales#3318, number_sales#3319L, null, null, null, null, 15)], [sales#3318, number_sales#3319L, channel#3384, i_brand_id#3385, i_class_id#3386, i_category_id#3387, spark_grouping_id#3383L]
               +- Union
                  :- *(39) Project [sales#3318, number_sales#3319L, channel#3317, i_brand_id#499, i_class_id#501, i_category_id#503]
                  :  +- *(39) Filter (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#1163 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#1165 as decimal(12,2)))), DecimalType(18,2), true))#3353) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#1163 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#1165 as decimal(12,2)))), DecimalType(18,2), true))#3353 as decimal(32,6)) > cast(Subquery scalar-subquery#3320, [id=#10802] as decimal(32,6))))
                  :     :  +- Subquery scalar-subquery#3320, [id=#10802]
                  :     :     +- *(8) HashAggregate(keys=[], functions=[avg(CheckOverflow((promote_precision(cast(cast(quantity#3335 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#3336 as decimal(12,2)))), DecimalType(18,2), true))])
                  :     :        +- Exchange SinglePartition, true, [id=#10798]
                  :     :           +- *(7) HashAggregate(keys=[], functions=[partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#3335 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#3336 as decimal(12,2)))), DecimalType(18,2), true))])
                  :     :              +- Union
                  :     :                 :- *(2) Project [ss_quantity#1163 AS quantity#3335, ss_list_price#1165 AS list_price#3336]
                  :     :                 :  +- *(2) BroadcastHashJoin [ss_sold_date_sk#1176], [d_date_sk#329], Inner, BuildRight
                  :     :                 :     :- *(2) ColumnarToRow
                  :     :                 :     :  +- FileScan parquet [ss_quantity#1163,ss_list_price#1165,ss_sold_date_sk#1176] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark-sql-perf/data5/store_sales], PartitionFilters: [isnotnull(ss_sold_date_sk#1176), dynamicpruningexpression(ss_sold_date_sk#1176 IN dynamicpruning..., PushedFilters: [], ReadSchema: struct<ss_quantity:int,ss_list_price:decimal(7,2)>
                  :     :                 :     :        +- SubqueryBroadcast dynamicpruning#3413, 0, [d_date_sk#329], [id=#10730]
                  :     :                 :     :           +- ReusedExchange [d_date_sk#329], BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#10640]
                  :     :                 :     +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#10640]
                  :     :                 :        +- *(1) Project [d_date_sk#329]
                  :     :                 :           +- *(1) Filter (((isnotnull(d_year#335) AND (d_year#335 >= 1999)) AND (d_year#335 <= 2001)) AND isnotnull(d_date_sk#329))
                  :     :                 :              +- *(1) ColumnarToRow
                  :     :                 :                 +- FileScan parquet [d_date_sk#329,d_year#335] Batched: true, DataFilters: [isnotnull(d_year#335), (d_year#335 >= 1999), (d_year#335 <= 2001), isnotnull(d_date_sk#329)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark-sql-perf/data5/date_dim], PartitionFilters: [], PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_da..., ReadSchema: struct<d_date_sk:int,d_year:int>
                  :     :                 :- *(4) Project [cs_quantity#889 AS quantity#3337, cs_list_price#891 AS list_price#3338]
                  :     :                 :  +- *(4) BroadcastHashJoin [cs_sold_date_sk#905], [d_date_sk#329], Inner, BuildRight
                  :     :                 :     :- *(4) ColumnarToRow
                  :     :                 :     :  +- FileScan parquet [cs_quantity#889,cs_list_price#891,cs_sold_date_sk#905] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark-sql-perf/data5/catalog_sales], PartitionFilters: [isnotnull(cs_sold_date_sk#905), dynamicpruningexpression(cs_sold_date_sk#905 IN dynamicpruning#3..., PushedFilters: [], ReadSchema: struct<cs_quantity:int,cs_list_price:decimal(7,2)>
                  :     :                 :     :        +- ReusedSubquery SubqueryBroadcast dynamicpruning#3413, 0, [d_date_sk#329], [id=#10730]
                  :     :                 :     +- ReusedExchange [d_date_sk#329], BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#10640]
                  :     :                 +- *(6) Project [ws_quantity#1030 AS quantity#3339, ws_list_price#1032 AS list_price#3340]
                  :     :                    +- *(6) BroadcastHashJoin [ws_sold_date_sk#1046], [d_date_sk#329], Inner, BuildRight
                  :     :                       :- *(6) ColumnarToRow
                  :     :                       :  +- FileScan parquet [ws_quantity#1030,ws_list_price#1032,ws_sold_date_sk#1046] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark-sql-perf/data5/web_sales], PartitionFilters: [isnotnull(ws_sold_date_sk#1046), dynamicpruningexpression(ws_sold_date_sk#1046 IN dynamicpruning..., PushedFilters: [], ReadSchema: struct<ws_quantity:int,ws_list_price:decimal(7,2)>
                  :     :                       :        +- ReusedSubquery SubqueryBroadcast dynamicpruning#3413, 0, [d_date_sk#329], [id=#10730]
                  :     :                       +- ReusedExchange [d_date_sk#329], BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#10640]
                  :     +- *(39) HashAggregate(keys=[i_brand_id#499, i_class_id#501, i_category_id#503], functions=[sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#1163 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#1165 as decimal(12,2)))), DecimalType(18,2), true)), count(1)])
                  :        +- Exchange hashpartitioning(i_brand_id#499, i_class_id#501, i_category_id#503, 4), true, [id=#15050]
                  :           +- *(38) HashAggregate(keys=[i_brand_id#499, i_class_id#501, i_category_id#503], functions=[partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#1163 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#1165 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)])
                  :              +- *(38) Project [ss_quantity#1163, ss_list_price#1165, i_brand_id#499, i_class_id#501, i_category_id#503]
                  :                 +- *(38) BroadcastHashJoin [ss_sold_date_sk#1176], [d_date_sk#329], Inner, BuildRight
                  :                    :- *(38) Project [ss_quantity#1163, ss_list_price#1165, ss_sold_date_sk#1176, i_brand_id#499, i_class_id#501, i_category_id#503]
                  :                    :  +- *(38) BroadcastHashJoin [ss_item_sk#1155], [i_item_sk#492], Inner, BuildRight
                  :                    :     :- SortMergeJoin [ss_item_sk#1155], [ss_item_sk#3334], LeftSemi
                  :                    :     :  :- *(2) Sort [ss_item_sk#1155 ASC NULLS FIRST], false, 0
                  :                    :     :  :  +- Exchange hashpartitioning(ss_item_sk#1155, 4), true, [id=#14846]
                  :                    :     :  :     +- *(1) Project [ss_item_sk#1155, ss_quantity#1163, ss_list_price#1165, ss_sold_date_sk#1176]
                  :                    :     :  :        +- *(1) Filter isnotnull(ss_item_sk#1155)
                  :                    :     :  :           +- *(1) ColumnarToRow
                  :                    :     :  :              +- FileScan parquet [ss_item_sk#1155,ss_quantity#1163,ss_list_price#1165,ss_sold_date_sk#1176] Batched: true, DataFilters: [isnotnull(ss_item_sk#1155)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark-sql-perf/data5/store_sales], PartitionFilters: [isnotnull(ss_sold_date_sk#1176), dynamicpruningexpression(ss_sold_date_sk#1176 IN dynamicpruning..., PushedFilters: [IsNotNull(ss_item_sk)], ReadSchema: struct<ss_item_sk:int,ss_quantity:int,ss_list_price:decimal(7,2)>
                  :                    :     :  :                    +- SubqueryBroadcast dynamicpruning#3445, 0, [d_date_sk#329], [id=#14839]
                  :                    :     :  :                       +- ReusedExchange [d_date_sk#329], BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#12547]
                  :                    :     :  +- *(18) Sort [ss_item_sk#3334 ASC NULLS FIRST], false, 0
                  :                    :     :     +- Exchange hashpartitioning(ss_item_sk#3334, 4), true, [id=#15038]
                  :                    :     :        +- *(17) Project [i_item_sk#492 AS ss_item_sk#3334]
                  :                    :     :           +- *(17) BroadcastHashJoin [i_brand_id#499, i_class_id#501, i_category_id#503], [brand_id#3331, class_id#3332, category_id#3333], Inner, BuildLeft
                  :                    :     :              :- BroadcastExchange HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true])), [id=#12321]
                  :                    :     :              :  +- *(3) Project [i_item_sk#492, i_brand_id#499, i_class_id#501, i_category_id#503]
                  :                    :     :              :     +- *(3) Filter ((isnotnull(i_brand_id#499) AND isnotnull(i_category_id#503)) AND isnotnull(i_class_id#501))
                  :                    :     :              :        +- *(3) ColumnarToRow
                  :                    :     :              :           +- FileScan parquet [i_item_sk#492,i_brand_id#499,i_class_id#501,i_category_id#503] Batched: true, DataFilters: [isnotnull(i_brand_id#499), isnotnull(i_category_id#503), isnotnull(i_class_id#501)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark-sql-perf/data5/item], PartitionFilters: [], PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_category_id), IsNotNull(i_class_id)], ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
                  :                    :     :              +- *(17) HashAggregate(keys=[brand_id#3331, class_id#3332, category_id#3333], functions=[])
                  :                    :     :                 +- *(17) HashAggregate(keys=[brand_id#3331, class_id#3332, category_id#3333], functions=[])
                  :                    :     :                    +- *(17) HashAggregate(keys=[brand_id#3331, class_id#3332, category_id#3333], functions=[])
                  :                    :     :                       +- Exchange hashpartitioning(brand_id#3331, class_id#3332, category_id#3333, 4), true, [id=#15030]
                  :                    :     :                          +- *(16) HashAggregate(keys=[brand_id#3331, class_id#3332, category_id#3333], functions=[])
                  :                    :     :                             +- SortMergeJoin [coalesce(brand_id#3331, 0), isnull(brand_id#3331), coalesce(class_id#3332, 0), isnull(class_id#3332), coalesce(category_id#3333, 0), isnull(category_id#3333)], [coalesce(i_brand_id#499, 0), isnull(i_brand_id#499), coalesce(i_class_id#501, 0), isnull(i_class_id#501), coalesce(i_category_id#503, 0), isnull(i_category_id#503)], LeftSemi
                  :                    :     :                                :- SortMergeJoin [coalesce(brand_id#3331, 0), isnull(brand_id#3331), coalesce(class_id#3332, 0), isnull(class_id#3332), coalesce(category_id#3333, 0), isnull(category_id#3333)], [coalesce(i_brand_id#499, 0), isnull(i_brand_id#499), coalesce(i_class_id#501, 0), isnull(i_class_id#501), coalesce(i_category_id#503, 0), isnull(i_category_id#503)], LeftSemi
                  :                    :     :                                :  :- *(7) Sort [coalesce(brand_id#3331, 0) ASC NULLS FIRST, isnull(brand_id#3331) ASC NULLS FIRST, coalesce(class_id#3332, 0) ASC NULLS FIRST, isnull(class_id#3332) ASC NULLS FIRST, coalesce(category_id#3333, 0) ASC NULLS FIRST, isnull(category_id#3333) ASC NULLS FIRST], false, 0
                  :                    :     :                                :  :  +- Exchange hashpartitioning(coalesce(brand_id#3331, 0), isnull(brand_id#3331), coalesce(class_id#3332, 0), isnull(class_id#3332), coalesce(category_id#3333, 0), isnull(category_id#3333), 4), true, [id=#14862]
                  :                    :     :                                :  :     +- *(6) Project [i_brand_id#499 AS brand_id#3331, i_class_id#501 AS class_id#3332, i_category_id#503 AS category_id#3333]
                  :                    :     :                                :  :        +- *(6) BroadcastHashJoin [ss_sold_date_sk#1176], [d_date_sk#329], Inner, BuildRight
                  :                    :     :                                :  :           :- *(6) Project [ss_sold_date_sk#1176, i_brand_id#499, i_class_id#501, i_category_id#503]
                  :                    :     :                                :  :           :  +- *(6) BroadcastHashJoin [ss_item_sk#1155], [i_item_sk#492], Inner, BuildRight
                  :                    :     :                                :  :           :     :- *(6) Project [ss_item_sk#1155, ss_sold_date_sk#1176]
                  :                    :     :                                :  :           :     :  +- *(6) Filter isnotnull(ss_item_sk#1155)
                  :                    :     :                                :  :           :     :     +- *(6) ColumnarToRow
                  :                    :     :                                :  :           :     :        +- FileScan parquet [ss_item_sk#1155,ss_sold_date_sk#1176] Batched: true, DataFilters: [isnotnull(ss_item_sk#1155)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark-sql-perf/data5/store_sales], PartitionFilters: [isnotnull(ss_sold_date_sk#1176), dynamicpruningexpression(ss_sold_date_sk#1176 IN dynamicpruning..., PushedFilters: [IsNotNull(ss_item_sk)], ReadSchema: struct<ss_item_sk:int>
                  :                    :     :                                :  :           :     :              +- SubqueryBroadcast dynamicpruning#3416, 0, [d_date_sk#329], [id=#14851]
                  :                    :     :                                :  :           :     :                 +- ReusedExchange [d_date_sk#329], BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#12341]
                  :                    :     :                                :  :           :     +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#12332]
                  :                    :     :                                :  :           :        +- *(4) Project [i_item_sk#492, i_brand_id#499, i_class_id#501, i_category_id#503]
                  :                    :     :                                :  :           :           +- *(4) Filter (((isnotnull(i_item_sk#492) AND isnotnull(i_category_id#503)) AND isnotnull(i_class_id#501)) AND isnotnull(i_brand_id#499))
                  :                    :     :                                :  :           :              +- *(4) ColumnarToRow
                  :                    :     :                                :  :           :                 +- FileScan parquet [i_item_sk#492,i_brand_id#499,i_class_id#501,i_category_id#503] Batched: true, DataFilters: [isnotnull(i_item_sk#492), isnotnull(i_category_id#503), isnotnull(i_class_id#501), isnotnull(i_b..., Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark-sql-perf/data5/item], PartitionFilters: [], PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_category_id), IsNotNull(i_class_id), IsNotNull(i_brand_id)], ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
                  :                    :     :                                :  :           +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#12341]
                  :                    :     :                                :  :              +- *(5) Project [d_date_sk#329]
                  :                    :     :                                :  :                 +- *(5) Filter (((isnotnull(d_year#335) AND (d_year#335 >= 1999)) AND (d_year#335 <= 2001)) AND isnotnull(d_date_sk#329))
                  :                    :     :                                :  :                    +- *(5) ColumnarToRow
                  :                    :     :                                :  :                       +- FileScan parquet [d_date_sk#329,d_year#335] Batched: true, DataFilters: [isnotnull(d_year#335), (d_year#335 >= 1999), (d_year#335 <= 2001), isnotnull(d_date_sk#329)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark-sql-perf/data5/date_dim], PartitionFilters: [], PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_da..., ReadSchema: struct<d_date_sk:int,d_year:int>
                  :                    :     :                                :  +- *(11) Sort [coalesce(i_brand_id#499, 0) ASC NULLS FIRST, isnull(i_brand_id#499) ASC NULLS FIRST, coalesce(i_class_id#501, 0) ASC NULLS FIRST, isnull(i_class_id#501) ASC NULLS FIRST, coalesce(i_category_id#503, 0) ASC NULLS FIRST, isnull(i_category_id#503) ASC NULLS FIRST], false, 0
                  :                    :     :                                :     +- Exchange hashpartitioning(coalesce(i_brand_id#499, 0), isnull(i_brand_id#499), coalesce(i_class_id#501, 0), isnull(i_class_id#501), coalesce(i_category_id#503, 0), isnull(i_category_id#503), 4), true, [id=#15005]
                  :                    :     :                                :        +- *(10) Project [i_brand_id#499, i_class_id#501, i_category_id#503]
                  :                    :     :                                :           +- *(10) BroadcastHashJoin [cs_sold_date_sk#905], [d_date_sk#329], Inner, BuildRight
                  :                    :     :                                :              :- *(10) Project [cs_sold_date_sk#905, i_brand_id#499, i_class_id#501, i_category_id#503]
                  :                    :     :                                :              :  +- *(10) BroadcastHashJoin [cs_item_sk#886], [i_item_sk#492], Inner, BuildRight
                  :                    :     :                                :              :     :- *(10) Project [cs_item_sk#886, cs_sold_date_sk#905]
                  :                    :     :                                :              :     :  +- *(10) Filter isnotnull(cs_item_sk#886)
                  :                    :     :                                :              :     :     +- *(10) ColumnarToRow
                  :                    :     :                                :              :     :        +- FileScan parquet [cs_item_sk#886,cs_sold_date_sk#905] Batched: true, DataFilters: [isnotnull(cs_item_sk#886)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark-sql-perf/data5/catalog_sales], PartitionFilters: [isnotnull(cs_sold_date_sk#905), dynamicpruningexpression(cs_sold_date_sk#905 IN dynamicpruning#3..., PushedFilters: [IsNotNull(cs_item_sk)], ReadSchema: struct<cs_item_sk:int>
                  :                    :     :                                :              :     :              +- ReusedSubquery SubqueryBroadcast dynamicpruning#3416, 0, [d_date_sk#329], [id=#14851]
                  :                    :     :                                :              :     +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#12359]
                  :                    :     :                                :              :        +- *(8) Project [i_item_sk#492, i_brand_id#499, i_class_id#501, i_category_id#503]
                  :                    :     :                                :              :           +- *(8) Filter isnotnull(i_item_sk#492)
                  :                    :     :                                :              :              +- *(8) ColumnarToRow
                  :                    :     :                                :              :                 +- FileScan parquet [i_item_sk#492,i_brand_id#499,i_class_id#501,i_category_id#503] Batched: true, DataFilters: [isnotnull(i_item_sk#492)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark-sql-perf/data5/item], PartitionFilters: [], PushedFilters: [IsNotNull(i_item_sk)], ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
                  :                    :     :                                :              +- ReusedExchange [d_date_sk#329], BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#12341]
                  :                    :     :                                +- *(15) Sort [coalesce(i_brand_id#499, 0) ASC NULLS FIRST, isnull(i_brand_id#499) ASC NULLS FIRST, coalesce(i_class_id#501, 0) ASC NULLS FIRST, isnull(i_class_id#501) ASC NULLS FIRST, coalesce(i_category_id#503, 0) ASC NULLS FIRST, isnull(i_category_id#503) ASC NULLS FIRST], false, 0
                  :                    :     :                                   +- Exchange hashpartitioning(coalesce(i_brand_id#499, 0), isnull(i_brand_id#499), coalesce(i_class_id#501, 0), isnull(i_class_id#501), coalesce(i_category_id#503, 0), isnull(i_category_id#503), 4), true, [id=#15022]
                  :                    :     :                                      +- *(14) Project [i_brand_id#499, i_class_id#501, i_category_id#503]
                  :                    :     :                                         +- *(14) BroadcastHashJoin [ws_sold_date_sk#1046], [d_date_sk#329], Inner, BuildRight
                  :                    :     :                                            :- *(14) Project [ws_sold_date_sk#1046, i_brand_id#499, i_class_id#501, i_category_id#503]
                  :                    :     :                                            :  +- *(14) BroadcastHashJoin [ws_item_sk#1015], [i_item_sk#492], Inner, BuildRight
                  :                    :     :                                            :     :- *(14) Project [ws_item_sk#1015, ws_sold_date_sk#1046]
                  :                    :     :                                            :     :  +- *(14) Filter isnotnull(ws_item_sk#1015)
                  :                    :     :                                            :     :     +- *(14) ColumnarToRow
                  :                    :     :                                            :     :        +- FileScan parquet [ws_item_sk#1015,ws_sold_date_sk#1046] Batched: true, DataFilters: [isnotnull(ws_item_sk#1015)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark-sql-perf/data5/web_sales], PartitionFilters: [isnotnull(ws_sold_date_sk#1046), dynamicpruningexpression(ws_sold_date_sk#1046 IN dynamicpruning..., PushedFilters: [IsNotNull(ws_item_sk)], ReadSchema: struct<ws_item_sk:int>
                  :                    :     :                                            :     :              +- ReusedSubquery SubqueryBroadcast dynamicpruning#3416, 0, [d_date_sk#329], [id=#14851]
                  :                    :     :                                            :     +- ReusedExchange [i_item_sk#492, i_brand_id#499, i_class_id#501, i_category_id#503], BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#12359]
                  :                    :     :                                            +- ReusedExchange [d_date_sk#329], BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#12341]
                  :                    :     +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#13552]
                  :                    :        +- SortMergeJoin [i_item_sk#492], [ss_item_sk#3334], LeftSemi
                  :                    :           :- *(20) Sort [i_item_sk#492 ASC NULLS FIRST], false, 0
                  :                    :           :  +- Exchange hashpartitioning(i_item_sk#492, 4), true, [id=#12428]
                  :                    :           :     +- *(19) Project [i_item_sk#492, i_brand_id#499, i_class_id#501, i_category_id#503]
                  :                    :           :        +- *(19) Filter isnotnull(i_item_sk#492)
                  :                    :           :           +- *(19) ColumnarToRow
                  :                    :           :              +- FileScan parquet [i_item_sk#492,i_brand_id#499,i_class_id#501,i_category_id#503] Batched: true, DataFilters: [isnotnull(i_item_sk#492)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark-sql-perf/data5/item], PartitionFilters: [], PushedFilters: [IsNotNull(i_item_sk)], ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int>
                  :                    :           +- *(36) Sort [ss_item_sk#3334 ASC NULLS FIRST], false, 0
                  :                    :              +- ReusedExchange [ss_item_sk#3334], Exchange hashpartitioning(ss_item_sk#3334, 4), true, [id=#13278]
                  :                    +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#12547]
                  :                       +- *(37) Project [d_date_sk#329]
                  :                          +- *(37) Filter ((((isnotnull(d_year#335) AND isnotnull(d_moy#337)) AND (d_year#335 = 2001)) AND (d_moy#337 = 11)) AND isnotnull(d_date_sk#329))
                  :                             +- *(37) ColumnarToRow
                  :                                +- FileScan parquet [d_date_sk#329,d_year#335,d_moy#337] Batched: true, DataFilters: [isnotnull(d_year#335), isnotnull(d_moy#337), (d_year#335 = 2001), (d_moy#337 = 11), isnotnull(d_..., Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark-sql-perf/data5/date_dim], PartitionFilters: [], PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,11), IsNotNull(d_date_sk)], ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int>
                  :- *(78) Project [sales#3323, number_sales#3324L, channel#3322, i_brand_id#499, i_class_id#501, i_category_id#503]
                  :  +- *(78) Filter (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#889 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#891 as decimal(12,2)))), DecimalType(18,2), true))#3356) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#889 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#891 as decimal(12,2)))), DecimalType(18,2), true))#3356 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#3320, [id=#10802] as decimal(32,6))))
                  :     :  +- ReusedSubquery Subquery scalar-subquery#3320, [id=#10802]
                  :     +- *(78) HashAggregate(keys=[i_brand_id#499, i_class_id#501, i_category_id#503], functions=[sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#889 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#891 as decimal(12,2)))), DecimalType(18,2), true)), count(1)])
                  :        +- Exchange hashpartitioning(i_brand_id#499, i_class_id#501, i_category_id#503, 4), true, [id=#15154]
                  :           +- *(77) HashAggregate(keys=[i_brand_id#499, i_class_id#501, i_category_id#503], functions=[partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#889 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#891 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)])
                  :              +- *(77) Project [cs_quantity#889, cs_list_price#891, i_brand_id#499, i_class_id#501, i_category_id#503]
                  :                 +- *(77) BroadcastHashJoin [cs_sold_date_sk#905], [d_date_sk#329], Inner, BuildRight
                  :                    :- *(77) Project [cs_quantity#889, cs_list_price#891, cs_sold_date_sk#905, i_brand_id#499, i_class_id#501, i_category_id#503]
                  :                    :  +- *(77) BroadcastHashJoin [cs_item_sk#886], [i_item_sk#492], Inner, BuildRight
                  :                    :     :- SortMergeJoin [cs_item_sk#886], [ss_item_sk#3334], LeftSemi
                  :                    :     :  :- *(41) Sort [cs_item_sk#886 ASC NULLS FIRST], false, 0
                  :                    :     :  :  +- Exchange hashpartitioning(cs_item_sk#886, 4), true, [id=#15142]
                  :                    :     :  :     +- *(40) Project [cs_item_sk#886, cs_quantity#889, cs_list_price#891, cs_sold_date_sk#905]
                  :                    :     :  :        +- *(40) Filter isnotnull(cs_item_sk#886)
                  :                    :     :  :           +- *(40) ColumnarToRow
                  :                    :     :  :              +- FileScan parquet [cs_item_sk#886,cs_quantity#889,cs_list_price#891,cs_sold_date_sk#905] Batched: true, DataFilters: [isnotnull(cs_item_sk#886)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark-sql-perf/data5/catalog_sales], PartitionFilters: [isnotnull(cs_sold_date_sk#905), dynamicpruningexpression(cs_sold_date_sk#905 IN dynamicpruning#3..., PushedFilters: [IsNotNull(cs_item_sk)], ReadSchema: struct<cs_item_sk:int,cs_quantity:int,cs_list_price:decimal(7,2)>
                  :                    :     :  :                    +- ReusedSubquery SubqueryBroadcast dynamicpruning#3445, 0, [d_date_sk#329], [id=#14839]
                  :                    :     :  +- *(57) Sort [ss_item_sk#3334 ASC NULLS FIRST], false, 0
                  :                    :     :     +- ReusedExchange [ss_item_sk#3334], Exchange hashpartitioning(ss_item_sk#3334, 4), true, [id=#13278]
                  :                    :     +- ReusedExchange [i_item_sk#492, i_brand_id#499, i_class_id#501, i_category_id#503], BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#13552]
                  :                    +- ReusedExchange [d_date_sk#329], BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#12547]
                  +- *(117) Project [sales#3328, number_sales#3329L, channel#3327, i_brand_id#499, i_class_id#501, i_category_id#503]
                     +- *(117) Filter (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#1030 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#1032 as decimal(12,2)))), DecimalType(18,2), true))#3359) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#1030 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#1032 as decimal(12,2)))), DecimalType(18,2), true))#3359 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#3320, [id=#10802] as decimal(32,6))))
                        :  +- ReusedSubquery Subquery scalar-subquery#3320, [id=#10802]
                        +- *(117) HashAggregate(keys=[i_brand_id#499, i_class_id#501, i_category_id#503], functions=[sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#1030 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#1032 as decimal(12,2)))), DecimalType(18,2), true)), count(1)])
                           +- Exchange hashpartitioning(i_brand_id#499, i_class_id#501, i_category_id#503, 4), true, [id=#15220]
                              +- *(116) HashAggregate(keys=[i_brand_id#499, i_class_id#501, i_category_id#503], functions=[partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#1030 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#1032 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)])
                                 +- *(116) Project [ws_quantity#1030, ws_list_price#1032, i_brand_id#499, i_class_id#501, i_category_id#503]
                                    +- *(116) BroadcastHashJoin [ws_sold_date_sk#1046], [d_date_sk#329], Inner, BuildRight
                                       :- *(116) Project [ws_quantity#1030, ws_list_price#1032, ws_sold_date_sk#1046, i_brand_id#499, i_class_id#501, i_category_id#503]
                                       :  +- *(116) BroadcastHashJoin [ws_item_sk#1015], [i_item_sk#492], Inner, BuildRight
                                       :     :- SortMergeJoin [ws_item_sk#1015], [ss_item_sk#3334], LeftSemi
                                       :     :  :- *(80) Sort [ws_item_sk#1015 ASC NULLS FIRST], false, 0
                                       :     :  :  +- Exchange hashpartitioning(ws_item_sk#1015, 4), true, [id=#15208]
                                       :     :  :     +- *(79) Project [ws_item_sk#1015, ws_quantity#1030, ws_list_price#1032, ws_sold_date_sk#1046]
                                       :     :  :        +- *(79) Filter isnotnull(ws_item_sk#1015)
                                       :     :  :           +- *(79) ColumnarToRow
                                       :     :  :              +- FileScan parquet [ws_item_sk#1015,ws_quantity#1030,ws_list_price#1032,ws_sold_date_sk#1046] Batched: true, DataFilters: [isnotnull(ws_item_sk#1015)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark-sql-perf/data5/web_sales], PartitionFilters: [isnotnull(ws_sold_date_sk#1046), dynamicpruningexpression(ws_sold_date_sk#1046 IN dynamicpruning..., PushedFilters: [IsNotNull(ws_item_sk)], ReadSchema: struct<ws_item_sk:int,ws_quantity:int,ws_list_price:decimal(7,2)>
                                       :     :  :                    +- ReusedSubquery SubqueryBroadcast dynamicpruning#3445, 0, [d_date_sk#329], [id=#14839]
                                       :     :  +- *(96) Sort [ss_item_sk#3334 ASC NULLS FIRST], false, 0
                                       :     :     +- ReusedExchange [ss_item_sk#3334], Exchange hashpartitioning(ss_item_sk#3334, 4), true, [id=#13278]
                                       :     +- ReusedExchange [i_item_sk#492, i_brand_id#499, i_class_id#501, i_category_id#503], BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#13552]
                                       +- ReusedExchange [d_date_sk#329], BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#12547]
   ```
   
   - Improves the reuse of exchanges and subqueries by enabling reuse across the whole plan. This means that the new combined rule utilizes the reuse opportunities between parent and subqueries by traversing the whole plan (including subqueries) in a bottom up manner.
   
   - Improves the caching logic by avoiding canonicalization when possible. When at least 2 exchanges or subqueries have the same schema only then they are canonicalized.
   
   ### Why are the changes needed?
   Performance improvement.
   
   ### How was this patch tested?
   Existing and new UTs, TPCDS benchmarks.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] attilapiros commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r455801996



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/util/ReuseMap.scala
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.util
+
+import scala.collection.mutable.Map
+import scala.language.existentials
+
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Map of canonicalized plans that can be used to find reuse possibilities.
+ *
+ * To avoid costly canonicalization of a plan:
+ * - we use its schema first to check if it can be replaced to a reused one at all
+ * - we insert it into the map of canonicalized plans only when at least 2 have the same schema
+ */
+class ReuseMap[T <: QueryPlan[_]] {
+  // scalastyle:off structural.type
+  private val map = Map[StructType, (T, Map[T2 forSome { type T2 >: T }, T])]()
+  // scalastyle:on structural.type
+
+  /**
+   * Find a matching plan with the same canonicalized form in the map or add the new plan to the
+   * map otherwise.
+   *
+   * @param plan the input plan
+   * @return the matching plan or the input plan
+   */
+  def lookupOrElseAdd(plan: T): T = {
+    val (firstSameSchemaPlan, sameResultPlans) = map.getOrElseUpdate(plan.schema, plan -> Map())
+    if (firstSameSchemaPlan ne plan) {
+      if (sameResultPlans.isEmpty) {
+        sameResultPlans += firstSameSchemaPlan.canonicalized -> firstSameSchemaPlan
+      }
+      sameResultPlans.getOrElseUpdate(plan.canonicalized, plan)
+    } else {
+      plan
+    }
+  }
+
+  /**
+   * Find a matching plan with the same canonicalized form in the map and apply `f` on it or add
+   * the new plan to the map otherwise.
+   *
+   * @param plan the input plan
+   * @param f the function to apply
+   * @return the matching plan with `f` applied or the input plan
+   */
+  def reuseOrElseAdd[T2 >: T](plan: T, f: T => T2): T2 = {

Review comment:
       Nit: this might be a better name: `transformReusedOrAdd` and/or consider rename `f` to `funcOnReused`. What do you think?
   
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [WIP][SPARK-29375][SPARK-28940][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-647094520






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-705422879


   retest this please


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-652578311


   **[Test build #124810 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124810/testReport)** for PR 28885 at commit [`fd522ea`](https://github.com/apache/spark/commit/fd522ea3b9ddb3953122b4e9ac2e74ef34d66140).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-662960607


   **[Test build #126408 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126408/testReport)** for PR 28885 at commit [`5df4c53`](https://github.com/apache/spark/commit/5df4c53d8f2805699bbcc2eae99380b59fd3ac3d).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on pull request #28885: [WIP][SPARK-29375][SPARK-28940][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-647097876


   I measured considerable improvement (~30%) on TPCDS Q14a, Q14b, Q23a, Q23b, Q47 and Q57 queries. 
   
   
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-647122614


   **[Test build #124334 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124334/testReport)** for PR 28885 at commit [`dbd8606`](https://github.com/apache/spark/commit/dbd860667ed3cae4c56ebddbb214056c426ef9dd).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `case class WholePlanReuse(conf: SQLConf) extends Rule[SparkPlan] `


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-773464818


   **[Test build #134885 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/134885/testReport)** for PR 28885 at commit [`ec2324a`](https://github.com/apache/spark/commit/ec2324a25c9fb02272d84b29678155e240f5ce8f).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-773632699


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/134885/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-691435855






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-766963390


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/39039/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sririshindra commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
sririshindra commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r445859500



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/Reuse.scala
##########
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.reuse
+
+import scala.collection.mutable.Map
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
+ * use the same exhange or subquery for all the references.
+ */
+case class WholePlanReuse(conf: SQLConf) extends Rule[SparkPlan] {
+
+  def apply(plan: SparkPlan): SparkPlan = {
+    if (conf.exchangeReuseEnabled || conf.subqueryReuseEnabled) {

Review comment:
       Shouldn't this be `if (conf.wholePlanReuseEnabled) { `instead of `if (conf.exchangeReuseEnabled || conf.subqueryReuseEnabled) {`. 
   
   We could take the position that if wholePlanReuse is enabled then it means both exchangeReuseEnabled and subqueryReuseEnabled are set to true. So setting wholePlanReuseEnabled to true will override whatever user specified value for exchangeReuseEnabled and subqueryReuseEnabled is. if that is case then in the current commit even `if (conf.wholePlanReuseEnabled) { ` will be redundant because this piece of code will only be triggered when wholePlanReuseEnabled is set to true.
   
   But if we want to say that the flags exchangeReuseEnabled and subqueryReuseEnabled are independent of the flag wholePlanReuseEnabled, then `if (conf.exchangeReuseEnabled || conf.subqueryReuseEnabled) {` is saying that if one flag is true then the other is true as well i.e if one flag is true then both exchange and subquery are being reused irrespective of the value set by the user for the other. This could be confusing to the user. Here we are basically enabling both exchangeReuse and subqueryReuse as along as wholePlanReuseEnabled is set to true.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-691488559






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r459266941



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/util/ReuseMap.scala
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.util
+
+import scala.collection.mutable.Map
+import scala.language.existentials
+
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Map of canonicalized plans that can be used to find reuse possibilities.
+ *
+ * To avoid costly canonicalization of a plan:
+ * - we use its schema first to check if it can be replaced to a reused one at all
+ * - we insert it into the map of canonicalized plans only when at least 2 have the same schema
+ */
+class ReuseMap[T <: QueryPlan[_]] {
+  // scalastyle:off structural.type
+  private val map = Map[StructType, (T, Map[T2 forSome { type T2 >: T }, T])]()
+  // scalastyle:on structural.type
+
+  /**
+   * Find a matching plan with the same canonicalized form in the map or add the new plan to the
+   * map otherwise.
+   *
+   * @param plan the input plan
+   * @return the matching plan or the input plan
+   */
+  def lookupOrElseAdd(plan: T): T = {
+    val (firstSameSchemaPlan, sameResultPlans) = map.getOrElseUpdate(plan.schema, plan -> Map())

Review comment:
       I see your point and I'm fine with reverting this change to the `ArrayBuffer`.
   
   To me `Map[StructType, ArrayBuffer[SparkPlan]]` doesn't look intuitive at all exactly because of (a). Plan nodes are immutable case classes that provide consistent `equals()` and `hashcode()` and IMHO in those cases a map is better fit to look up an item. Using the `ArrayBuffer` suggests that something is weird with those nodes but I don't think it is. (We use canonicalized map of plans in `ReuseAdaptiveSubquery` too.)
   To answer (b) as well, no, they very likely doesn't bring much improvement in a real word use case. The change is mainly due to the above reason.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r446918380



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/Reuse.scala
##########
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.reuse
+
+import scala.collection.mutable.Map
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
+ * use the same exhange or subquery for all the references.
+ */
+case class ReuseExchangeAndSubquery(conf: SQLConf) extends Rule[SparkPlan] {
+
+  private class ReuseCache[T <: SparkPlan] {
+    // To avoid costly canonicalization of an exchange or a subquery:
+    // - we use its schema first to check if it can be replaced to a reused one at all
+    // - we insert it into the map of canonicalized plans only when at least 2 have the same schema
+    private val cache = Map[StructType, (T, Map[SparkPlan, T])]()
+
+    def lookup(plan: T): T = {
+      val (firstSameSchemaPlan, sameResultPlans) = cache.getOrElseUpdate(plan.schema, plan -> Map())
+      if (firstSameSchemaPlan.ne(plan)) {
+        if (sameResultPlans.isEmpty) {
+          sameResultPlans += firstSameSchemaPlan.canonicalized -> firstSameSchemaPlan
+        }
+        sameResultPlans.getOrElseUpdate(plan.canonicalized, plan)
+      } else {
+        plan
+      }
+    }
+  }
+
+  def apply(plan: SparkPlan): SparkPlan = {
+    if (conf.exchangeReuseEnabled || conf.subqueryReuseEnabled) {
+      val exchanges = new ReuseCache[Exchange]()
+      val subqueries = new ReuseCache[BaseSubqueryExec]()
+
+      def reuse(plan: SparkPlan): SparkPlan = plan.transformUp {
+        case exchange: Exchange if conf.exchangeReuseEnabled =>
+          val cached = exchanges.lookup(exchange)
+          if (cached.ne(exchange)) {
+            ReusedExchangeExec(exchange.output, cached)
+          } else {
+            exchange
+          }
+
+        case other => other.transformExpressionsUp {
+          case sub: ExecSubqueryExpression =>
+            val subquery = reuse(sub.plan).asInstanceOf[BaseSubqueryExec]
+            sub.withNewPlan(
+              if (conf.subqueryReuseEnabled) {

Review comment:
       I added `addOrElse` in the latest commits.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dbaliafroozeh commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
dbaliafroozeh commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r446698444



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/Reuse.scala
##########
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.reuse
+
+import scala.collection.mutable.Map
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
+ * use the same exhange or subquery for all the references.
+ */
+case class ReuseExchangeAndSubquery(conf: SQLConf) extends Rule[SparkPlan] {
+
+  private class ReuseCache[T <: SparkPlan] {

Review comment:
       I like this new abstraction, just two nitpicky comments:
   - This is not really a cache IMO, in a sense that we need to hold on to all the values, it's more a memo table. How about renaming it to `ReuseMap`? or something in this line?
   - Can we move this class somewhere else? I know it's being used only here, but I think it'll be great to use this anytime we wanna lookup something with canonicalized forms. If you agree to move it to a util package, let's also make the type upperbound `QueryPlan` so that we can use it for other node types.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth edited a comment on pull request #28885: [SPARK-29375][SPARK-28940][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth edited a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-647103671


   I've just realized that this PR partly overlaps with https://github.com/apache/spark/pull/28881. @prakharjain09 opened it 2 hours before I did, but my PR does a bit more than that and actually does the combined reuse in a bit different way so I wouldn't close mine yet.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r655261429



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/ReuseExchangeAndSubquery.scala
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.reuse
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern._
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.util.ReuseMap
+
+/**
+ * Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
+ * use the same exchange or subquery for all the references.
+ *
+ * Note that the Spark plan is a mutually recursive data structure:
+ * SparkPlan -> Expr -> Subquery -> SparkPlan -> Expr -> Subquery -> ...
+ * Therefore, in this rule, we recursively rewrite the exchanges and subqueries in a bottom-up way,
+ * in one go.
+ */
+case object ReuseExchangeAndSubquery extends Rule[SparkPlan] {

Review comment:
       Yea non-AQE doesn't need thread safety, but I feel it's still better to unify the major idea:
   1. `Map[<canonicalized plan>, <plan>]`
   2. `Map[<schema>, ArrayBuffer[<plan>]]`
   
   I agree it's not a big deal for perf, but code consistency is also important. `Map[<canonicalized plan>, <plan>]` looks better as it's simpler, we can remove `ReuseMap`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-762978850


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/38817/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-691436044






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r446631916



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/Reuse.scala
##########
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.reuse
+
+import scala.collection.mutable.Map
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
+ * use the same exhange or subquery for all the references.
+ */
+case class WholePlanReuse(conf: SQLConf) extends Rule[SparkPlan] {

Review comment:
       Thanks, fixed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-762898416


   **[Test build #134232 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/134232/testReport)** for PR 28885 at commit [`7e1e5bf`](https://github.com/apache/spark/commit/7e1e5bfcac0f3f30becf30687e64feca74eedca7).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-705376018






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-705204281


   **[Test build #129528 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129528/testReport)** for PR 28885 at commit [`913483c`](https://github.com/apache/spark/commit/913483c9df599b63ee48e3b84f5bb73ae6374bb9).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-705159621


   **[Test build #129528 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129528/testReport)** for PR 28885 at commit [`913483c`](https://github.com/apache/spark/commit/913483c9df599b63ee48e3b84f5bb73ae6374bb9).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-705375084






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-762978850


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/38817/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-705423444


   **[Test build #129552 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129552/testReport)** for PR 28885 at commit [`012ef8f`](https://github.com/apache/spark/commit/012ef8f3ee6af6d68b976f5ea125cf05d29c0a1d).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sririshindra commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
sririshindra commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r445859500



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/Reuse.scala
##########
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.reuse
+
+import scala.collection.mutable.Map
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
+ * use the same exhange or subquery for all the references.
+ */
+case class WholePlanReuse(conf: SQLConf) extends Rule[SparkPlan] {
+
+  def apply(plan: SparkPlan): SparkPlan = {
+    if (conf.exchangeReuseEnabled || conf.subqueryReuseEnabled) {

Review comment:
       Shouldn't this be `if (conf.wholePlanReuseEnabled) { `instead of `if (conf.exchangeReuseEnabled || conf.subqueryReuseEnabled) {`. 
   
   We could take the position that if wholePlanReuse is enabled then it means both exchangeReuseEnabled and subqueryReuseEnabled are set to true. So setting wholePlanReuseEnabled to true will override whatever user specified value for exchangeReuseEnabled and subqueryReuseEnabled is. if that is the case then in the current commit even `if (conf.wholePlanReuseEnabled) { ` will be redundant because this piece of code will only be triggered when wholePlanReuseEnabled is set to true.
   
   But if we want to say that the flags exchangeReuseEnabled and subqueryReuseEnabled are independent of the flag wholePlanReuseEnabled, then `if (conf.exchangeReuseEnabled || conf.subqueryReuseEnabled) {` is saying that if one flag is true then the other is true as well i.e if one flag is true then both exchange and subquery are being reused irrespective of the value set by the user for the other. This could be confusing to the user. Here we are basically enabling both exchangeReuse and subqueryReuse as along as wholePlanReuseEnabled is set to true.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-691435855






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-647122823






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] squito commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
squito commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r459091567



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/Exchange.scala
##########
@@ -95,46 +89,3 @@ case class ReusedExchangeExec(override val output: Seq[Attribute], child: Exchan
        |""".stripMargin
   }
 }
-
-/**
- * Find out duplicated exchanges in the spark plan, then use the same exchange for all the
- * references.
- */
-case class ReuseExchange(conf: SQLConf) extends Rule[SparkPlan] {
-
-  def apply(plan: SparkPlan): SparkPlan = {
-    if (!conf.exchangeReuseEnabled) {
-      return plan
-    }
-    // Build a hash map using schema of exchanges to avoid O(N*N) sameResult calls.
-    val exchanges = mutable.HashMap[StructType, ArrayBuffer[Exchange]]()
-
-    // Replace a Exchange duplicate with a ReusedExchange
-    def reuse: PartialFunction[Exchange, SparkPlan] = {
-      case exchange: Exchange =>
-        val sameSchema = exchanges.getOrElseUpdate(exchange.schema, ArrayBuffer[Exchange]())
-        val samePlan = sameSchema.find { e =>
-          exchange.sameResult(e)
-        }
-        if (samePlan.isDefined) {
-          // Keep the output of this exchange, the following plans require that to resolve
-          // attributes.
-          ReusedExchangeExec(exchange.output, samePlan.get)
-        } else {
-          sameSchema += exchange
-          exchange
-        }
-    }
-
-    plan transformUp {
-      case exchange: Exchange => reuse(exchange)
-    } transformAllExpressions {
-      // Lookup inside subqueries for duplicate exchanges
-      case in: InSubqueryExec =>
-        val newIn = in.plan.transformUp {
-          case exchange: Exchange => reuse(exchange)
-        }
-        in.copy(plan = newIn.asInstanceOf[BaseSubqueryExec])
-    }
-  }

Review comment:
       can you tell me the exact test case that demonstrates this issue?
   
   I kinda see the issue you're talking about, but my thinking is that maybe this just requires fixing `ReuseExchange`, rather than combining both into one rule.  




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-663125786


   **[Test build #126408 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126408/testReport)** for PR 28885 at commit [`5df4c53`](https://github.com/apache/spark/commit/5df4c53d8f2805699bbcc2eae99380b59fd3ac3d).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `class ReuseMap[T <: T2, T2 <: QueryPlan[T2]] `


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r460109287



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala
##########
@@ -156,4 +158,46 @@ class ExchangeSuite extends SparkPlanTest with SharedSparkSession {
     val projection2 = cached.select("_1", "_3").queryExecution.executedPlan
     assert(!projection1.sameResult(projection2))
   }
+
+  test("Exchange reuse across the whole plan") {
+    val df = sql(
+      """
+        |SELECT
+        |  (SELECT max(a.key) FROM testData AS a JOIN testData AS b ON b.key = a.key),
+        |  a.key
+        |FROM testData AS a
+        |JOIN testData AS b ON b.key = a.key
+      """.stripMargin)
+
+    val plan = df.queryExecution.executedPlan
+
+    val exchangeIds = plan.collectWithSubqueries { case e: Exchange => e.id }
+    val reusedExchangeIds = plan.collectWithSubqueries {
+      case re: ReusedExchangeExec => re.child.id
+    }
+
+    assert(exchangeIds.size == 2, "Whole plan exchange reusing not working correctly")
+    assert(reusedExchangeIds.size == 3, "Whole plan exchange reusing not working correctly")
+    assert(reusedExchangeIds.forall(exchangeIds.contains(_)),
+      "ReusedExchangeExec should reuse an existing exchange")
+
+    val df2 = sql(

Review comment:
       This is exactly what I suggested in this PR. Currently `ReuseExchange` and `ReuseSubquery` are the 2 last rules before we get the final `executedPlan` in `QueryExecution`: https://github.com/apache/spark/pull/28885/files#diff-16d8dfddab6ffbffb4204a6955ce3d26L340-L341. After this PR `ReuseExchangeAndSubquery` will be the last rule so we don't need to worry about further transformation that might change a referenced node.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-766433954






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-763437539


   I've updated this PR with minor changes due to AQE is now enabled by default.
   
   Without this PR invalid reuse nodes are in many TPCDS queries causing performance degradation: q14a, q14b, q23a, q23b, q47, q57, q5a-v2.7, q14-v2.7, q14a-v2.7, q22a-v2.7, q36a-v2.7, q47-v2.7, q51a-v2.7, q57-v2.7, q67a-v2.7, q70a-v2.7, q77a-v2.7, q80a-v2.7, q86a-v2.7
   
   https://github.com/apache/spark/pull/31243 updates PlanStabilitySuite tests to cover partitioned tables (DPP kicks in) and the issue appears as `ReusedExchange [Reuses operator id: unknown]` nodes in `explain.txt`s of that PR.  
   
   @cloud-fan @viirya @wangyum could you please review this PR? 
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dbaliafroozeh commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
dbaliafroozeh commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r446673965



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
##########
@@ -326,7 +327,8 @@ object QueryExecution {
    */
   private[execution] def preparations(
       sparkSession: SparkSession,
-      adaptiveExecutionRule: Option[InsertAdaptiveSparkPlan] = None): Seq[Rule[SparkPlan]] = {
+      adaptiveExecutionRule: Option[InsertAdaptiveSparkPlan] = None,
+      subquery: Boolean): Seq[Rule[SparkPlan]] = {

Review comment:
       Yes, from the performance perspective makes sense to exclude them. I sort of don't like having another parameter and select rules based on that, so was thinking if it's not a huge performance difference let's not do it, but it can be expensive with canonicalization, etc. I guess we don't have any other way of detecting if a physical plan is a subquery locally inside the new rule, so it's fine to do it like this, maybe we need a more explicit name for `QueryExecution.prepareExecutedPlan` in the future, like `PrepareSubqueryForExecution` to make it more clear that this method is only called for subqueries. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-691435855






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-704814287


   **[Test build #129500 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129500/testReport)** for PR 28885 at commit [`e0f1ee1`](https://github.com/apache/spark/commit/e0f1ee1a3f132fce5801871fc2e39586aa6ee41d).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-766425750






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-762898416


   **[Test build #134232 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/134232/testReport)** for PR 28885 at commit [`7e1e5bf`](https://github.com/apache/spark/commit/7e1e5bfcac0f3f30becf30687e64feca74eedca7).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-803009280


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/136257/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r447567127



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/Reuse.scala
##########
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.reuse
+
+import scala.collection.mutable.Map
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
+ * use the same exhange or subquery for all the references.
+ */
+case class ReuseExchangeAndSubquery(conf: SQLConf) extends Rule[SparkPlan] {
+
+  private class ReuseCache[T <: SparkPlan] {
+    // To avoid costly canonicalization of an exchange or a subquery:
+    // - we use its schema first to check if it can be replaced to a reused one at all
+    // - we insert it into the map of canonicalized plans only when at least 2 have the same schema
+    private val cache = Map[StructType, (T, Map[SparkPlan, T])]()
+
+    def lookup(plan: T): T = {
+      val (firstSameSchemaPlan, sameResultPlans) = cache.getOrElseUpdate(plan.schema, plan -> Map())
+      if (firstSameSchemaPlan.ne(plan)) {
+        if (sameResultPlans.isEmpty) {
+          sameResultPlans += firstSameSchemaPlan.canonicalized -> firstSameSchemaPlan
+        }
+        sameResultPlans.getOrElseUpdate(plan.canonicalized, plan)
+      } else {
+        plan
+      }
+    }
+  }
+
+  def apply(plan: SparkPlan): SparkPlan = {
+    if (conf.exchangeReuseEnabled || conf.subqueryReuseEnabled) {
+      val exchanges = new ReuseCache[Exchange]()
+      val subqueries = new ReuseCache[BaseSubqueryExec]()
+
+      def reuse(plan: SparkPlan): SparkPlan = plan.transformUp {
+        case exchange: Exchange if conf.exchangeReuseEnabled =>
+          val cached = exchanges.lookup(exchange)
+          if (cached.ne(exchange)) {
+            ReusedExchangeExec(exchange.output, cached)
+          } else {
+            exchange
+          }
+
+        case other => other.transformExpressionsUp {
+          case sub: ExecSubqueryExpression =>
+            val subquery = reuse(sub.plan).asInstanceOf[BaseSubqueryExec]
+            sub.withNewPlan(
+              if (conf.subqueryReuseEnabled) {

Review comment:
       I agree that the name `lookup` doesn't capture exactly what the method does, but `getOrElseUpdate` has well known meaning and parameter list on scala maps and the first method doesn't match to that so I would use a different name like `getOrElseAdd` or `lookupOrElseAdd`.
   Yes, it's even harder to find a good name for the second method. How about `reuseOrElseAdd`? It doesn't match any of the existing methods of a map and captures both reuse (with applying `f`) and add functionality.
   
   The old behaviour was that on the second encounter of the same instance it was wrapped into a reuse node. This actually works different in this PR and the instance doesn't get wrapped on the second encounter. I did this way for 3 reasons:
   - From performance perspective it doesn't make any difference
   - An exchange or subquery instance can appear 2 times in the plan only if someone manually crafted the plan, which I don't think happens in a real word use case
   - I'm using `getOrElseUpdate` 2 times in `lookup` to simplify the code, but from the result of `getOrElseUpdate` we can't tell if the key had already been in the map or it was added during the call. Besides simple code I also like using `getOrElseUpdate` because some concurrent map implementation supports it as atomic operation. I admit it doesn't matter in this PR, but I think it would be nice to incorporate `ReuseAdaptiveSubquery` functionality into this rule in a separate PR in the future.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-691458773


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/128590/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-660180874






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r457214868



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/util/ReuseMap.scala
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.util
+
+import scala.collection.mutable.Map
+import scala.language.existentials
+
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Map of canonicalized plans that can be used to find reuse possibilities.
+ *
+ * To avoid costly canonicalization of a plan:
+ * - we use its schema first to check if it can be replaced to a reused one at all
+ * - we insert it into the map of canonicalized plans only when at least 2 have the same schema
+ */
+class ReuseMap[T <: QueryPlan[_]] {
+  // scalastyle:off structural.type
+  private val map = Map[StructType, (T, Map[T2 forSome { type T2 >: T }, T])]()
+  // scalastyle:on structural.type
+
+  /**
+   * Find a matching plan with the same canonicalized form in the map or add the new plan to the
+   * map otherwise.
+   *
+   * @param plan the input plan
+   * @return the matching plan or the input plan
+   */
+  def lookupOrElseAdd(plan: T): T = {
+    val (firstSameSchemaPlan, sameResultPlans) = map.getOrElseUpdate(plan.schema, plan -> Map())
+    if (firstSameSchemaPlan ne plan) {
+      if (sameResultPlans.isEmpty) {
+        sameResultPlans += firstSameSchemaPlan.canonicalized -> firstSameSchemaPlan

Review comment:
       Thanks, I added some more comments to make this part more readable.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-705422879


   retest this please


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-705376143






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-705347605






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r655291568



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/ReuseExchangeAndSubquery.scala
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.reuse
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern._
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.util.ReuseMap
+
+/**
+ * Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
+ * use the same exchange or subquery for all the references.
+ *
+ * Note that the Spark plan is a mutually recursive data structure:
+ * SparkPlan -> Expr -> Subquery -> SparkPlan -> Expr -> Subquery -> ...
+ * Therefore, in this rule, we recursively rewrite the exchanges and subqueries in a bottom-up way,
+ * in one go.
+ */
+case object ReuseExchangeAndSubquery extends Rule[SparkPlan] {

Review comment:
       Ok, I can file a follow-up PR soon.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-691458762






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth edited a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth edited a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-864199872






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-766432638


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/39001/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r655291568



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/ReuseExchangeAndSubquery.scala
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.reuse
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern._
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.util.ReuseMap
+
+/**
+ * Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
+ * use the same exchange or subquery for all the references.
+ *
+ * Note that the Spark plan is a mutually recursive data structure:
+ * SparkPlan -> Expr -> Subquery -> SparkPlan -> Expr -> Subquery -> ...
+ * Therefore, in this rule, we recursively rewrite the exchanges and subqueries in a bottom-up way,
+ * in one go.
+ */
+case object ReuseExchangeAndSubquery extends Rule[SparkPlan] {

Review comment:
       Ok, I can file a follow-up PR today or tomorrow.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r655073897



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/ReuseExchangeAndSubquery.scala
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.reuse
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern._
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.util.ReuseMap
+
+/**
+ * Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
+ * use the same exchange or subquery for all the references.
+ *
+ * Note that the Spark plan is a mutually recursive data structure:
+ * SparkPlan -> Expr -> Subquery -> SparkPlan -> Expr -> Subquery -> ...
+ * Therefore, in this rule, we recursively rewrite the exchanges and subqueries in a bottom-up way,
+ * in one go.
+ */
+case object ReuseExchangeAndSubquery extends Rule[SparkPlan] {

Review comment:
       Actually, the AQE way may be better: just use the canonicalized plan as the key, instead of calling `sameResult`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-766433954


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/134415/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-705459673






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-705347605






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r655072173



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/ReuseExchangeAndSubquery.scala
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.reuse
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern._
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.util.ReuseMap
+
+/**
+ * Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
+ * use the same exchange or subquery for all the references.
+ *
+ * Note that the Spark plan is a mutually recursive data structure:
+ * SparkPlan -> Expr -> Subquery -> SparkPlan -> Expr -> Subquery -> ...
+ * Therefore, in this rule, we recursively rewrite the exchanges and subqueries in a bottom-up way,
+ * in one go.
+ */
+case object ReuseExchangeAndSubquery extends Rule[SparkPlan] {

Review comment:
       AQE has a totally different way to reuse exchange/subquery. It has a query-global map to store created exchange/subquery (see `AdaptiveExecutionContext`), and AQE executes leaf subqueries first. This is exactly the same as what this rule is doing for exchange/subquery reuse.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-802847553


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/40839/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-802854434


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/40839/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-706676195


   **[Test build #129637 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129637/testReport)** for PR 28885 at commit [`ab2f369`](https://github.com/apache/spark/commit/ab2f3699eb8313c72bcb29a4c4e3d7b142b00c5e).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-650732399






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-776246481


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/39665/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-766425750


   **[Test build #134415 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/134415/testReport)** for PR 28885 at commit [`8f24add`](https://github.com/apache/spark/commit/8f24adda6fc72262cc50054e48b82c73a3cc213f).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-691435855






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-650623623






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r459266941



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/util/ReuseMap.scala
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.util
+
+import scala.collection.mutable.Map
+import scala.language.existentials
+
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Map of canonicalized plans that can be used to find reuse possibilities.
+ *
+ * To avoid costly canonicalization of a plan:
+ * - we use its schema first to check if it can be replaced to a reused one at all
+ * - we insert it into the map of canonicalized plans only when at least 2 have the same schema
+ */
+class ReuseMap[T <: QueryPlan[_]] {
+  // scalastyle:off structural.type
+  private val map = Map[StructType, (T, Map[T2 forSome { type T2 >: T }, T])]()
+  // scalastyle:on structural.type
+
+  /**
+   * Find a matching plan with the same canonicalized form in the map or add the new plan to the
+   * map otherwise.
+   *
+   * @param plan the input plan
+   * @return the matching plan or the input plan
+   */
+  def lookupOrElseAdd(plan: T): T = {
+    val (firstSameSchemaPlan, sameResultPlans) = map.getOrElseUpdate(plan.schema, plan -> Map())

Review comment:
       I see your point and I'm fine with reverting this change to the `ArrayBuffer`.
   
   To me `Map[StructType, ArrayBuffer[SparkPlan]]` doesn't look intuitive at all exactly because of (a). Plan nodes are immutable case classes which provide consistent `equals()` and `hashcode()` and IMHO in those cases a map is better fit to look up an item. Using the `ArrayBuffer` suggests that something is weird with those nodes but I don't think it is. (We use canonicalized map of plans in `ReuseAdaptiveSubquery` too.)
   To answer (b) as well, no, they very likely doesn't bring much improvement in a real word use case. The change is mainly due to the above reason.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-650623357


   **[Test build #124572 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124572/testReport)** for PR 28885 at commit [`c49a0f9`](https://github.com/apache/spark/commit/c49a0f99b25522dd7ed89e8b41d2c5c80dbaf170).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r655072608



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/ReuseExchangeAndSubquery.scala
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.reuse
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern._
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.util.ReuseMap
+
+/**
+ * Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
+ * use the same exchange or subquery for all the references.
+ *
+ * Note that the Spark plan is a mutually recursive data structure:
+ * SparkPlan -> Expr -> Subquery -> SparkPlan -> Expr -> Subquery -> ...
+ * Therefore, in this rule, we recursively rewrite the exchanges and subqueries in a bottom-up way,
+ * in one go.
+ */
+case object ReuseExchangeAndSubquery extends Rule[SparkPlan] {

Review comment:
       One followup we can do is to leverage the new `ReuseMap` in `AdaptiveExecutionContext`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-705448544


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34158/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-802814755


   **[Test build #136257 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136257/testReport)** for PR 28885 at commit [`8822acc`](https://github.com/apache/spark/commit/8822accf758ee8696f981591cd538a67d65e96cf).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-667550219






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-766430779


   **[Test build #134415 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/134415/testReport)** for PR 28885 at commit [`8f24add`](https://github.com/apache/spark/commit/8f24adda6fc72262cc50054e48b82c73a3cc213f).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-650780371


   **[Test build #124602 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124602/testReport)** for PR 28885 at commit [`2b3cde2`](https://github.com/apache/spark/commit/2b3cde22f62374c25b5748ff2253e168ec4e5d95).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] squito commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
squito commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r465997277



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala
##########
@@ -156,4 +158,46 @@ class ExchangeSuite extends SparkPlanTest with SharedSparkSession {
     val projection2 = cached.select("_1", "_3").queryExecution.executedPlan
     assert(!projection1.sameResult(projection2))
   }
+
+  test("Exchange reuse across the whole plan") {
+    val df = sql(
+      """
+        |SELECT
+        |  (SELECT max(a.key) FROM testData AS a JOIN testData AS b ON b.key = a.key),
+        |  a.key
+        |FROM testData AS a
+        |JOIN testData AS b ON b.key = a.key
+      """.stripMargin)
+
+    val plan = df.queryExecution.executedPlan
+
+    val exchangeIds = plan.collectWithSubqueries { case e: Exchange => e.id }
+    val reusedExchangeIds = plan.collectWithSubqueries {
+      case re: ReusedExchangeExec => re.child.id
+    }
+
+    assert(exchangeIds.size == 2, "Whole plan exchange reusing not working correctly")
+    assert(reusedExchangeIds.size == 3, "Whole plan exchange reusing not working correctly")
+    assert(reusedExchangeIds.forall(exchangeIds.contains(_)),
+      "ReusedExchangeExec should reuse an existing exchange")
+
+    val df2 = sql(

Review comment:
       Sorry I am getting back to this so late.
   
   So, I see what you are saying, but this sounds super brittle.  Again, I'm not an expert here, but I wouldn't have expected rules to be so specific to their ordering.  What if someone else adds another rule which they decide also has to be last, for some other reason?  The abstraction suggests the rules should operate relatively independently.
   
   It would seem better to have a mechanism in `transform()` etc. functions to deal with this kind of thing.  Eg. there already is a call to `copyTagsFrom()`, couldn't there be something similar there to keep the references between "reused" nodes intact?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-707053033


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-707155690






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-706706454


   **[Test build #129637 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129637/testReport)** for PR 28885 at commit [`ab2f369`](https://github.com/apache/spark/commit/ab2f3699eb8313c72bcb29a4c4e3d7b142b00c5e).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r446632136



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/Reuse.scala
##########
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.reuse
+
+import scala.collection.mutable.Map
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
+ * use the same exhange or subquery for all the references.
+ */
+case class WholePlanReuse(conf: SQLConf) extends Rule[SparkPlan] {
+
+  def apply(plan: SparkPlan): SparkPlan = {
+    if (conf.exchangeReuseEnabled || conf.subqueryReuseEnabled) {
+      // To avoid costly canonicalization of an exchange or a subquery:
+      // - we use its schema first to check if it can be replaced to a reused one at all
+      // - we insert it into the map of canonicalized plans only when at least 2 have the same
+      //   schema
+      val exchanges = Map[StructType, (Exchange, Map[SparkPlan, Exchange])]()
+      val subqueries = Map[StructType, (BaseSubqueryExec, Map[SparkPlan, BaseSubqueryExec])]()

Review comment:
       Please find answer in https://github.com/apache/spark/pull/28885#discussion_r446632000




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r459389895



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/util/ReuseMap.scala
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.util
+
+import scala.collection.mutable.Map
+import scala.language.existentials
+
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Map of canonicalized plans that can be used to find reuse possibilities.
+ *
+ * To avoid costly canonicalization of a plan:
+ * - we use its schema first to check if it can be replaced to a reused one at all
+ * - we insert it into the map of canonicalized plans only when at least 2 have the same schema
+ */
+class ReuseMap[T <: QueryPlan[_]] {
+  // scalastyle:off structural.type
+  private val map = Map[StructType, (T, Map[T2 forSome { type T2 >: T }, T])]()
+  // scalastyle:on structural.type
+
+  /**
+   * Find a matching plan with the same canonicalized form in the map or add the new plan to the
+   * map otherwise.
+   *
+   * @param plan the input plan
+   * @return the matching plan or the input plan
+   */
+  def lookupOrElseAdd(plan: T): T = {
+    val (firstSameSchemaPlan, sameResultPlans) = map.getOrElseUpdate(plan.schema, plan -> Map())

Review comment:
       Commit https://github.com/apache/spark/pull/28885/commits/5df4c53d8f2805699bbcc2eae99380b59fd3ac3d reverts `ReuseMap` backing data structure to `Map[StructType, ArrayBuffer[Spark]]`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r446632217



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
##########
@@ -326,7 +327,8 @@ object QueryExecution {
    */
   private[execution] def preparations(
       sparkSession: SparkSession,
-      adaptiveExecutionRule: Option[InsertAdaptiveSparkPlan] = None): Seq[Rule[SparkPlan]] = {
+      adaptiveExecutionRule: Option[InsertAdaptiveSparkPlan] = None,
+      subquery: Boolean): Seq[Rule[SparkPlan]] = {

Review comment:
       Sorry, I don't get this, why would we run this rule multiple times? This new rule traverses through the whole plan, does it make any sense to run in on subqueries and then run it on a main query which also incorporates traversing on subqueries?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-705391110


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34152/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-667579885






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dbaliafroozeh commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
dbaliafroozeh commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r446514163



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
##########
@@ -326,7 +327,8 @@ object QueryExecution {
    */
   private[execution] def preparations(
       sparkSession: SparkSession,
-      adaptiveExecutionRule: Option[InsertAdaptiveSparkPlan] = None): Seq[Rule[SparkPlan]] = {
+      adaptiveExecutionRule: Option[InsertAdaptiveSparkPlan] = None,
+      subquery: Boolean): Seq[Rule[SparkPlan]] = {

Review comment:
       Why do we need this boolean parameter here? What will happen if we just always run the `WholePlanReuse` rule? 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-683252192


   **[Test build #128013 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128013/testReport)** for PR 28885 at commit [`cd064e4`](https://github.com/apache/spark/commit/cd064e460f90984d57397bac499058a261bc7205).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-660180874


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-705558034


   **[Test build #129552 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129552/testReport)** for PR 28885 at commit [`012ef8f`](https://github.com/apache/spark/commit/012ef8f3ee6af6d68b976f5ea125cf05d29c0a1d).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-766449350






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-660180150


   **[Test build #126051 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126051/testReport)** for PR 28885 at commit [`100e5c2`](https://github.com/apache/spark/commit/100e5c2319866734facd8de6f86dac92f438402b).
    * This patch **fails PySpark pip packaging tests**.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `class ReuseExchangeAndSubquerySuite extends SparkPlanTest with SharedSparkSession `


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-652415968


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-650732238


   **[Test build #124602 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124602/testReport)** for PR 28885 at commit [`2b3cde2`](https://github.com/apache/spark/commit/2b3cde22f62374c25b5748ff2253e168ec4e5d95).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-776252896


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/39665/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-650066534






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-660914307


   **[Test build #126174 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126174/testReport)** for PR 28885 at commit [`0d18e28`](https://github.com/apache/spark/commit/0d18e28637c07209dfdbc9c1777a6f458cd0ad69).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-802814755


   **[Test build #136257 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136257/testReport)** for PR 28885 at commit [`8822acc`](https://github.com/apache/spark/commit/8822accf758ee8696f981591cd538a67d65e96cf).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-773499553


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/39471/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-705459634


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34158/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-863565048






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r654206429



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/ExplainUtils.scala
##########
@@ -41,22 +38,15 @@ object ExplainUtils extends AdaptiveSparkPlanHelper {
    *
    * @param plan Input query plan to process
    * @param append function used to append the explain output
-   * @param startOperatorID The start value of operation id. The subsequent operations will
-   *                         be assigned higher value.
    *
    * @return The last generated operation id for this input plan. This is to ensure we

Review comment:
       We should remove this now.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/ReuseExchangeAndSubquery.scala
##########
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.reuse
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern._
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.util.ReuseMap
+
+/**
+ * Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
+ * use the same exchange or subquery for all the references.

Review comment:
       Maybe we should explain why we reuse exchange and subquery in this single rule
   ```
   Note that the Spark plan is a mutually recursive data structure:
     SparkPlan -> Expr -> Subquery -> SparkPlan -> Expr -> Subquery -> ...
   
   Therefore, in this rule, we recursively rewrite the exchanges and subqueries in a bottom-up way, in one go.
   ```

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/util/ReuseMap.scala
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.util
+
+import scala.collection.mutable.{ArrayBuffer, Map}
+
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Map of canonicalized plans that can be used to find reuse possibilities.
+ *
+ * To avoid costly canonicalization of a plan:
+ * - we use its schema first to check if it can be replaced to a reused one at all
+ * - we insert it into the map of canonicalized plans only when at least 2 have the same schema
+ *
+ * @tparam T the type of the node we want to reuse
+ * @tparam T2 the type of the canonicalized node
+ */
+class ReuseMap[T <: T2, T2 <: QueryPlan[T2]] {
+  private val map = Map[StructType, ArrayBuffer[T]]()
+
+  /**
+   * Find a matching plan with the same canonicalized form in the map or add the new plan to the
+   * map otherwise.
+   *
+   * @param plan the input plan
+   * @return the matching plan or the input plan
+   */
+  def lookupOrElseAdd(plan: T): T = {

Review comment:
       This can be private




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-773455991






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-691458762


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-705375084






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [WIP][SPARK-29375][SPARK-28940][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-647094460


   **[Test build #124334 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124334/testReport)** for PR 28885 at commit [`dbd8606`](https://github.com/apache/spark/commit/dbd860667ed3cae4c56ebddbb214056c426ef9dd).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-705391128






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-667579674


   **[Test build #126921 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126921/testReport)** for PR 28885 at commit [`5d5fd04`](https://github.com/apache/spark/commit/5d5fd04cc875145e82aa39a546c37dd8e2cb099f).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-704814287


   **[Test build #129500 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129500/testReport)** for PR 28885 at commit [`e0f1ee1`](https://github.com/apache/spark/commit/e0f1ee1a3f132fce5801871fc2e39586aa6ee41d).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dbaliafroozeh commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
dbaliafroozeh commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r446697950



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/Reuse.scala
##########
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.reuse
+
+import scala.collection.mutable.Map
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
+ * use the same exhange or subquery for all the references.
+ */
+case class ReuseExchangeAndSubquery(conf: SQLConf) extends Rule[SparkPlan] {
+
+  private class ReuseCache[T <: SparkPlan] {
+    // To avoid costly canonicalization of an exchange or a subquery:
+    // - we use its schema first to check if it can be replaced to a reused one at all
+    // - we insert it into the map of canonicalized plans only when at least 2 have the same schema
+    private val cache = Map[StructType, (T, Map[SparkPlan, T])]()
+
+    def lookup(plan: T): T = {
+      val (firstSameSchemaPlan, sameResultPlans) = cache.getOrElseUpdate(plan.schema, plan -> Map())
+      if (firstSameSchemaPlan.ne(plan)) {
+        if (sameResultPlans.isEmpty) {
+          sameResultPlans += firstSameSchemaPlan.canonicalized -> firstSameSchemaPlan
+        }
+        sameResultPlans.getOrElseUpdate(plan.canonicalized, plan)
+      } else {
+        plan
+      }
+    }
+  }
+
+  def apply(plan: SparkPlan): SparkPlan = {
+    if (conf.exchangeReuseEnabled || conf.subqueryReuseEnabled) {
+      val exchanges = new ReuseCache[Exchange]()
+      val subqueries = new ReuseCache[BaseSubqueryExec]()
+
+      def reuse(plan: SparkPlan): SparkPlan = plan.transformUp {
+        case exchange: Exchange if conf.exchangeReuseEnabled =>
+          val cached = exchanges.lookup(exchange)
+          if (cached.ne(exchange)) {
+            ReusedExchangeExec(exchange.output, cached)
+          } else {
+            exchange
+          }
+
+        case other => other.transformExpressionsUp {
+          case sub: ExecSubqueryExpression =>
+            val subquery = reuse(sub.plan).asInstanceOf[BaseSubqueryExec]
+            sub.withNewPlan(
+              if (conf.subqueryReuseEnabled) {

Review comment:
       I like the new abstraction for hiding the loop, how about one step further and also abstract over this pattern when the found element is the same instance as the one in the cache, something like this:
   
   ```
   def lookup(plan: T, f: T => T) {
   	val res = ....
   	if (res eq plan) {
   		plan
   	} else {
   		f(res)
   	}
   }
   ```
   
   and then the call site becomes: `lookupOrElse(subquery, ReusedSubqueryExec(_))`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r457213451



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/util/ReuseMap.scala
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.util
+
+import scala.collection.mutable.Map
+import scala.language.existentials
+
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Map of canonicalized plans that can be used to find reuse possibilities.
+ *
+ * To avoid costly canonicalization of a plan:
+ * - we use its schema first to check if it can be replaced to a reused one at all
+ * - we insert it into the map of canonicalized plans only when at least 2 have the same schema
+ */
+class ReuseMap[T <: QueryPlan[_]] {
+  // scalastyle:off structural.type
+  private val map = Map[StructType, (T, Map[T2 forSome { type T2 >: T }, T])]()
+  // scalastyle:on structural.type
+
+  /**
+   * Find a matching plan with the same canonicalized form in the map or add the new plan to the
+   * map otherwise.
+   *
+   * @param plan the input plan
+   * @return the matching plan or the input plan
+   */
+  def lookupOrElseAdd(plan: T): T = {
+    val (firstSameSchemaPlan, sameResultPlans) = map.getOrElseUpdate(plan.schema, plan -> Map())
+    if (firstSameSchemaPlan ne plan) {
+      if (sameResultPlans.isEmpty) {
+        sameResultPlans += firstSameSchemaPlan.canonicalized -> firstSameSchemaPlan
+      }
+      sameResultPlans.getOrElseUpdate(plan.canonicalized, plan)
+    } else {
+      plan
+    }
+  }
+
+  /**
+   * Find a matching plan with the same canonicalized form in the map and apply `f` on it or add
+   * the new plan to the map otherwise.
+   *
+   * @param plan the input plan
+   * @param f the function to apply
+   * @return the matching plan with `f` applied or the input plan
+   */
+  def reuseOrElseAdd[T2 >: T](plan: T, f: T => T2): T2 = {

Review comment:
       Thanks @attilapiros for the feedback. I agree with you and I'm happy to rename the method and but I would wait a bit for some more feedback from others as I've renamed this method a few times.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-691435855






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on pull request #28885: [SPARK-29375][SPARK-28940][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-647100996


   cc @cloud-fan, @dongjoon-hyun 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-652242395


   Merged build finished. Test PASSed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-705204490


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/129528/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r446859514



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/Reuse.scala
##########
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.reuse
+
+import scala.collection.mutable.Map
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
+ * use the same exhange or subquery for all the references.
+ */
+case class ReuseExchangeAndSubquery(conf: SQLConf) extends Rule[SparkPlan] {
+
+  private class ReuseCache[T <: SparkPlan] {
+    // To avoid costly canonicalization of an exchange or a subquery:
+    // - we use its schema first to check if it can be replaced to a reused one at all
+    // - we insert it into the map of canonicalized plans only when at least 2 have the same schema
+    private val cache = Map[StructType, (T, Map[SparkPlan, T])]()
+
+    def lookup(plan: T): T = {
+      val (firstSameSchemaPlan, sameResultPlans) = cache.getOrElseUpdate(plan.schema, plan -> Map())
+      if (firstSameSchemaPlan.ne(plan)) {
+        if (sameResultPlans.isEmpty) {
+          sameResultPlans += firstSameSchemaPlan.canonicalized -> firstSameSchemaPlan
+        }
+        sameResultPlans.getOrElseUpdate(plan.canonicalized, plan)
+      } else {
+        plan
+      }
+    }
+  }
+
+  def apply(plan: SparkPlan): SparkPlan = {
+    if (conf.exchangeReuseEnabled || conf.subqueryReuseEnabled) {
+      val exchanges = new ReuseCache[Exchange]()
+      val subqueries = new ReuseCache[BaseSubqueryExec]()
+
+      def reuse(plan: SparkPlan): SparkPlan = plan.transformUp {
+        case exchange: Exchange if conf.exchangeReuseEnabled =>
+          val cached = exchanges.lookup(exchange)
+          if (cached.ne(exchange)) {
+            ReusedExchangeExec(exchange.output, cached)
+          } else {
+            exchange
+          }
+
+        case other => other.transformExpressionsUp {
+          case sub: ExecSubqueryExpression =>
+            val subquery = reuse(sub.plan).asInstanceOf[BaseSubqueryExec]
+            sub.withNewPlan(
+              if (conf.subqueryReuseEnabled) {

Review comment:
       The name `lookupOrElse` sounds a bit weird to me, it suggests that `f` is applied if lookup fails (item isn't in the map yet). But in this case `f` should be applied if lookup founds the item in the map. So if we want that abstraction, shouldn't we call the method `putOrElse` or `addOrElse`?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-704846008


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34106/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [WIP][SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-862109061


   **[Test build #139857 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/139857/testReport)** for PR 28885 at commit [`bf29f1a`](https://github.com/apache/spark/commit/bf29f1a764927b9bf8006d8c950885f6eea24ddd).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-766943772


   **[Test build #134453 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/134453/testReport)** for PR 28885 at commit [`4db5801`](https://github.com/apache/spark/commit/4db5801f10888c7566d8beffe193583cc83f1031).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r467773298



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala
##########
@@ -156,4 +158,46 @@ class ExchangeSuite extends SparkPlanTest with SharedSparkSession {
     val projection2 = cached.select("_1", "_3").queryExecution.executedPlan
     assert(!projection1.sameResult(projection2))
   }
+
+  test("Exchange reuse across the whole plan") {
+    val df = sql(
+      """
+        |SELECT
+        |  (SELECT max(a.key) FROM testData AS a JOIN testData AS b ON b.key = a.key),
+        |  a.key
+        |FROM testData AS a
+        |JOIN testData AS b ON b.key = a.key
+      """.stripMargin)
+
+    val plan = df.queryExecution.executedPlan
+
+    val exchangeIds = plan.collectWithSubqueries { case e: Exchange => e.id }
+    val reusedExchangeIds = plan.collectWithSubqueries {
+      case re: ReusedExchangeExec => re.child.id
+    }
+
+    assert(exchangeIds.size == 2, "Whole plan exchange reusing not working correctly")
+    assert(reusedExchangeIds.size == 3, "Whole plan exchange reusing not working correctly")
+    assert(reusedExchangeIds.forall(exchangeIds.contains(_)),
+      "ReusedExchangeExec should reuse an existing exchange")
+
+    val df2 = sql(

Review comment:
       > Back to your earlier comment:
   > 
   > > * rule ReuseExchange inserted the ReusedExchange pointing to id=#761 into the plan
   > > * rule ReuseExchange altered the exchange node id=#761 to id=X
   > > * rule ReuseSubquery altered the exchange node id=X to id=#975
   > 
   > Just checking:
   > 
   > So the second bullet point is issue number 1 and the third bullet point is issue number 2?
   
   Yes.
   
   > Also checking: did the ReuseSubquery alter the exchange node to #975 or #979?
   
   No, sorry this was a typo, I meant id=X to id=#979. (Fixed in my original comment too.)
   
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-776251465


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/39665/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-767121644


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/134453/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-864725703


   thanks, merging to master!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-652578815






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-705347605


   **[Test build #129542 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129542/testReport)** for PR 28885 at commit [`abf0a4a`](https://github.com/apache/spark/commit/abf0a4af598016ca4e2f1c166c6cb93a3b0ad02e).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-683281814






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r459253058



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/util/ReuseMap.scala
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.util
+
+import scala.collection.mutable.Map
+import scala.language.existentials
+
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Map of canonicalized plans that can be used to find reuse possibilities.
+ *
+ * To avoid costly canonicalization of a plan:
+ * - we use its schema first to check if it can be replaced to a reused one at all
+ * - we insert it into the map of canonicalized plans only when at least 2 have the same schema
+ */
+class ReuseMap[T <: QueryPlan[_]] {
+  // scalastyle:off structural.type
+  private val map = Map[StructType, (T, Map[T2 forSome { type T2 >: T }, T])]()
+  // scalastyle:on structural.type
+
+  /**
+   * Find a matching plan with the same canonicalized form in the map or add the new plan to the
+   * map otherwise.
+   *
+   * @param plan the input plan
+   * @return the matching plan or the input plan
+   */
+  def lookupOrElseAdd(plan: T): T = {
+    val (firstSameSchemaPlan, sameResultPlans) = map.getOrElseUpdate(plan.schema, plan -> Map())

Review comment:
       This is the 1. point in the description and tested by the case `SPARK-32041: No reuse interference inside ReuseExchange` in the new `ReuseExchangeAndSubquerySuite`: https://github.com/apache/spark/pull/28885/files#diff-f6f54d5cfc4254d8ed9122013394351bR28




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-707017312


   **[Test build #129680 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129680/testReport)** for PR 28885 at commit [`b4bcbce`](https://github.com/apache/spark/commit/b4bcbce03b0d1bc763554cba31b9bbc2cb016a0b).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-704870370


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/129500/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r446631822



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
##########
@@ -341,7 +341,7 @@ object QueryPlan extends PredicateHelper {
         // Normalize the outer references in the subquery plan.
         val normalizedPlan = s.plan.transformAllExpressions {
           case OuterReference(r) => OuterReference(QueryPlan.normalizeExpressions(r, input))
-        }
+        }.canonicalized

Review comment:
       Thanks, good catch, fixed it in ScalarSubquery.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-691435855


   **[Test build #128590 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128590/testReport)** for PR 28885 at commit [`050ab40`](https://github.com/apache/spark/commit/050ab40ff2ada204ebc5c231818969e568cd003f).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-650733793


   > Thanks for doing this! I think the idea of whole plan reuse is good and your approach is correct, but I think some parts can be done differently IMO, I left some comments.
   
   Thanks for the review and comments @dbaliafroozeh!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-650180350


   **[Test build #124538 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124538/testReport)** for PR 28885 at commit [`8eca64a`](https://github.com/apache/spark/commit/8eca64aed6ddc30e731023ccf1d9ec32e4d95c21).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dbaliafroozeh commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
dbaliafroozeh commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r447072694



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/Reuse.scala
##########
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.reuse
+
+import scala.collection.mutable.Map
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
+ * use the same exhange or subquery for all the references.
+ */
+case class ReuseExchangeAndSubquery(conf: SQLConf) extends Rule[SparkPlan] {
+
+  private class ReuseCache[T <: SparkPlan] {
+    // To avoid costly canonicalization of an exchange or a subquery:
+    // - we use its schema first to check if it can be replaced to a reused one at all
+    // - we insert it into the map of canonicalized plans only when at least 2 have the same schema
+    private val cache = Map[StructType, (T, Map[SparkPlan, T])]()
+
+    def lookup(plan: T): T = {
+      val (firstSameSchemaPlan, sameResultPlans) = cache.getOrElseUpdate(plan.schema, plan -> Map())
+      if (firstSameSchemaPlan.ne(plan)) {
+        if (sameResultPlans.isEmpty) {
+          sameResultPlans += firstSameSchemaPlan.canonicalized -> firstSameSchemaPlan
+        }
+        sameResultPlans.getOrElseUpdate(plan.canonicalized, plan)
+      } else {
+        plan
+      }
+    }
+  }
+
+  def apply(plan: SparkPlan): SparkPlan = {
+    if (conf.exchangeReuseEnabled || conf.subqueryReuseEnabled) {
+      val exchanges = new ReuseCache[Exchange]()
+      val subqueries = new ReuseCache[BaseSubqueryExec]()
+
+      def reuse(plan: SparkPlan): SparkPlan = plan.transformUp {
+        case exchange: Exchange if conf.exchangeReuseEnabled =>
+          val cached = exchanges.lookup(exchange)
+          if (cached.ne(exchange)) {
+            ReusedExchangeExec(exchange.output, cached)
+          } else {
+            exchange
+          }
+
+        case other => other.transformExpressionsUp {
+          case sub: ExecSubqueryExpression =>
+            val subquery = reuse(sub.plan).asInstanceOf[BaseSubqueryExec]
+            sub.withNewPlan(
+              if (conf.subqueryReuseEnabled) {

Review comment:
       Yeah, I agree `lookupOrElse` is not the best name. I was thinking more about it, so we want a method with the following semantics:
   
   Try  to find an existing node with the same canonicalized form:
   - if no such node found, add the node to the map and return the node itself
   - if a node found that refers the same instance as the key, do nothing, and return the node itself
   - otherwise, call the given function on the retrieved node from the map and then return the result
   
   I think it's not like anything (I mean common methods) on the Map interfaces that I know of. Somehow reminds me of Java 8 `putIfAbsent` method, but with the difference it calls the passed function if it's present. 
   
   On second thought, maybe we should rename the `lookup` method to `getOrElseUpdate` because it doesn't just lookup. 
   And call the second method `applyIfPresent`. Maybe also doesn't make sense to have the second method anymore since it's very specific and hard to capture with a method name what's going on.
   
   One last thing, what was happening before if we had two exchanges in the query plan that were referring to the same instance? Were we leaving them intact or replacing it with a reuse node? Because now we just leave it as is. I'm not sure though if such a situation actually can happen to have to exchanges referring to the same instance in the query plan, if not, maybe we can remove the check and (the second method) altogether. Also maybe putting a reuse node when there are the same instances is even better, it's just a wrapper and signals the presence of the same exchange node.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-863555729






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-706706737






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-691525196






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-705179016


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-652721460






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r457214608



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/util/ReuseMap.scala
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.util
+
+import scala.collection.mutable.Map
+import scala.language.existentials
+
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Map of canonicalized plans that can be used to find reuse possibilities.
+ *
+ * To avoid costly canonicalization of a plan:
+ * - we use its schema first to check if it can be replaced to a reused one at all
+ * - we insert it into the map of canonicalized plans only when at least 2 have the same schema
+ */
+class ReuseMap[T <: QueryPlan[_]] {
+  // scalastyle:off structural.type
+  private val map = Map[StructType, (T, Map[T2 forSome { type T2 >: T }, T])]()
+  // scalastyle:on structural.type
+
+  /**
+   * Find a matching plan with the same canonicalized form in the map or add the new plan to the
+   * map otherwise.
+   *
+   * @param plan the input plan
+   * @return the matching plan or the input plan
+   */
+  def lookupOrElseAdd(plan: T): T = {
+    val (firstSameSchemaPlan, sameResultPlans) = map.getOrElseUpdate(plan.schema, plan -> Map())
+    if (firstSameSchemaPlan ne plan) {

Review comment:
       I'm fine with this one as well, but I would wait a bit in this case as well if you don't mind.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-705159621


   **[Test build #129528 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129528/testReport)** for PR 28885 at commit [`913483c`](https://github.com/apache/spark/commit/913483c9df599b63ee48e3b84f5bb73ae6374bb9).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-766984473


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/39039/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-662961008






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dbaliafroozeh commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
dbaliafroozeh commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r446516852



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/Reuse.scala
##########
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.reuse
+
+import scala.collection.mutable.Map
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
+ * use the same exhange or subquery for all the references.
+ */
+case class WholePlanReuse(conf: SQLConf) extends Rule[SparkPlan] {
+
+  def apply(plan: SparkPlan): SparkPlan = {
+    if (conf.exchangeReuseEnabled || conf.subqueryReuseEnabled) {
+      // To avoid costly canonicalization of an exchange or a subquery:
+      // - we use its schema first to check if it can be replaced to a reused one at all
+      // - we insert it into the map of canonicalized plans only when at least 2 have the same
+      //   schema
+      val exchanges = Map[StructType, (Exchange, Map[SparkPlan, Exchange])]()
+      val subqueries = Map[StructType, (BaseSubqueryExec, Map[SparkPlan, BaseSubqueryExec])]()

Review comment:
       The nested map somehow makese the logic unnecessarily complicated. Can we define a class like `Canonicalized[T]` where `T` can be an `Exchange` or `BaseSubqueryExec` and then implement the `equals` and `hashcode` of this class to first check for the schema equality? Then, we can have simply a map like `Map[Canonicalized[T], T]`, which  will simplify the code quite a bit.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] bersprockets commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
bersprockets commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r467641981



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala
##########
@@ -156,4 +158,46 @@ class ExchangeSuite extends SparkPlanTest with SharedSparkSession {
     val projection2 = cached.select("_1", "_3").queryExecution.executedPlan
     assert(!projection1.sameResult(projection2))
   }
+
+  test("Exchange reuse across the whole plan") {
+    val df = sql(
+      """
+        |SELECT
+        |  (SELECT max(a.key) FROM testData AS a JOIN testData AS b ON b.key = a.key),
+        |  a.key
+        |FROM testData AS a
+        |JOIN testData AS b ON b.key = a.key
+      """.stripMargin)
+
+    val plan = df.queryExecution.executedPlan
+
+    val exchangeIds = plan.collectWithSubqueries { case e: Exchange => e.id }
+    val reusedExchangeIds = plan.collectWithSubqueries {
+      case re: ReusedExchangeExec => re.child.id
+    }
+
+    assert(exchangeIds.size == 2, "Whole plan exchange reusing not working correctly")
+    assert(reusedExchangeIds.size == 3, "Whole plan exchange reusing not working correctly")
+    assert(reusedExchangeIds.forall(exchangeIds.contains(_)),
+      "ReusedExchangeExec should reuse an existing exchange")
+
+    val df2 = sql(

Review comment:
       Back to your earlier comment:
   
   > * rule ReuseExchange inserted the ReusedExchange pointing to id=#761 into the plan
   > * rule ReuseExchange altered the exchange node id=#761 to id=X
   > * rule ReuseSubquery altered the exchange node id=X to id=#975
   
   Just checking: 
   
   So the second bullet point is issue number 1 and the third bullet point is issue number 2?
   
   Also checking: did the ReuseSubquery alter the exchange node to #975 or #979?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-683281814






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-704870362






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r655142981



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/ReuseExchangeAndSubquery.scala
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.reuse
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern._
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.util.ReuseMap
+
+/**
+ * Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
+ * use the same exchange or subquery for all the references.
+ *
+ * Note that the Spark plan is a mutually recursive data structure:
+ * SparkPlan -> Expr -> Subquery -> SparkPlan -> Expr -> Subquery -> ...
+ * Therefore, in this rule, we recursively rewrite the exchanges and subqueries in a bottom-up way,
+ * in one go.
+ */
+case object ReuseExchangeAndSubquery extends Rule[SparkPlan] {

Review comment:
       `ReuseMap` has changed since the first version of this PR.
   Unfortunately, I rebased the PR already so only some discussion remained: https://github.com/apache/spark/pull/28885#discussion_r455692637 (about reverting 2nd to 3rd).
   
   The 1st version used a simple `Map[<canonicalized plan>, <plan>]` as `AdaptiveExecutionContext` does.
   The 2nd version was `Map[<schema>, Map[<canonicalized plan>, <plan>]]` with lazy initialization of the inner map to avoid canonicalization if there are no matching schemas but still provide quick lookup by canonicalized plans.
   This 3rd version reverted to the original `Map[<schema>, ArrayBuffer[<plan>]] idea that `ReuseExchange` and `ReuseSubquery` had used.
   
   I can open a follow-up PR to improve `ReuseMap` to 2nd version if you think so, but I'm not sure that the improvement would be visible with TPCDS or real life queries.
   
   If we want to consolidate reuse map logic then I think we should also take into account that `ReuseAdaptiveSubquery` uses a concurrent, lock-free `TrieMap` map implementation which is not required by this non-AQE rule.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/ReuseExchangeAndSubquery.scala
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.reuse
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern._
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.util.ReuseMap
+
+/**
+ * Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
+ * use the same exchange or subquery for all the references.
+ *
+ * Note that the Spark plan is a mutually recursive data structure:
+ * SparkPlan -> Expr -> Subquery -> SparkPlan -> Expr -> Subquery -> ...
+ * Therefore, in this rule, we recursively rewrite the exchanges and subqueries in a bottom-up way,
+ * in one go.
+ */
+case object ReuseExchangeAndSubquery extends Rule[SparkPlan] {

Review comment:
       `ReuseMap` has changed since the first version of this PR.
   Unfortunately, I rebased the PR already so only some discussion remained: https://github.com/apache/spark/pull/28885#discussion_r455692637 (about reverting 2nd to 3rd).
   
   The 1st version used a simple `Map[<canonicalized plan>, <plan>]` as `AdaptiveExecutionContext` does.
   The 2nd version was `Map[<schema>, Map[<canonicalized plan>, <plan>]]` with lazy initialization of the inner map to avoid canonicalization if there are no matching schemas but still provide quick lookup by canonicalized plans.
   This 3rd version reverted to the original `Map[<schema>, ArrayBuffer[<plan>]]` idea that `ReuseExchange` and `ReuseSubquery` had used.
   
   I can open a follow-up PR to improve `ReuseMap` to 2nd version if you think so, but I'm not sure that the improvement would be visible with TPCDS or real life queries.
   
   If we want to consolidate reuse map logic then I think we should also take into account that `ReuseAdaptiveSubquery` uses a concurrent, lock-free `TrieMap` map implementation which is not required by this non-AQE rule.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/ReuseExchangeAndSubquery.scala
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.reuse
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern._
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.util.ReuseMap
+
+/**
+ * Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
+ * use the same exchange or subquery for all the references.
+ *
+ * Note that the Spark plan is a mutually recursive data structure:
+ * SparkPlan -> Expr -> Subquery -> SparkPlan -> Expr -> Subquery -> ...
+ * Therefore, in this rule, we recursively rewrite the exchanges and subqueries in a bottom-up way,
+ * in one go.
+ */
+case object ReuseExchangeAndSubquery extends Rule[SparkPlan] {

Review comment:
       `ReuseMap` has changed since the first version of this PR.
   Unfortunately, I rebased the PR already so only some discussion remained: https://github.com/apache/spark/pull/28885#discussion_r455692637 (about reverting 2nd to 3rd).
   
   The 1st version used a simple `Map[<canonicalized plan>, <plan>]` as `AdaptiveExecutionContext` does.
   The 2nd version was `Map[<schema>, Map[<canonicalized plan>, <plan>]]` with lazy initialization of the inner map to avoid canonicalization if there are no matching schemas but still provide quick lookup by canonicalized plans.
   This 3rd version reverted to the original `Map[<schema>, ArrayBuffer[<plan>]]` idea that `ReuseExchange` and `ReuseSubquery` had used.
   
   I can open a follow-up PR to improve `ReuseMap` to 2nd version if required, but I'm not sure that the improvement would be visible with TPCDS or real life queries.
   
   If we want to consolidate reuse map logic then I think we should also take into account that `ReuseAdaptiveSubquery` uses a concurrent, lock-free `TrieMap` map implementation which is not required by this non-AQE rule.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/ReuseExchangeAndSubquery.scala
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.reuse
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern._
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.util.ReuseMap
+
+/**
+ * Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
+ * use the same exchange or subquery for all the references.
+ *
+ * Note that the Spark plan is a mutually recursive data structure:
+ * SparkPlan -> Expr -> Subquery -> SparkPlan -> Expr -> Subquery -> ...
+ * Therefore, in this rule, we recursively rewrite the exchanges and subqueries in a bottom-up way,
+ * in one go.
+ */
+case object ReuseExchangeAndSubquery extends Rule[SparkPlan] {

Review comment:
       Ok, I can file a follow-up PR soon.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/ReuseExchangeAndSubquery.scala
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.reuse
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern._
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.util.ReuseMap
+
+/**
+ * Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
+ * use the same exchange or subquery for all the references.
+ *
+ * Note that the Spark plan is a mutually recursive data structure:
+ * SparkPlan -> Expr -> Subquery -> SparkPlan -> Expr -> Subquery -> ...
+ * Therefore, in this rule, we recursively rewrite the exchanges and subqueries in a bottom-up way,
+ * in one go.
+ */
+case object ReuseExchangeAndSubquery extends Rule[SparkPlan] {

Review comment:
       Ok, I can file a follow-up PR today or tomorrow.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/ReuseExchangeAndSubquery.scala
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.reuse
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern._
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.util.ReuseMap
+
+/**
+ * Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
+ * use the same exchange or subquery for all the references.
+ *
+ * Note that the Spark plan is a mutually recursive data structure:
+ * SparkPlan -> Expr -> Subquery -> SparkPlan -> Expr -> Subquery -> ...
+ * Therefore, in this rule, we recursively rewrite the exchanges and subqueries in a bottom-up way,
+ * in one go.
+ */
+case object ReuseExchangeAndSubquery extends Rule[SparkPlan] {

Review comment:
       `ReuseMap` has changed since the first version of this PR.
   Unfortunately, I rebased the PR already so only some discussion remained: https://github.com/apache/spark/pull/28885#discussion_r455692637 (about reverting 2nd to 3rd).
   
   The 1st version used a simple `Map[<canonicalized plan>, <plan>]` as `AdaptiveExecutionContext` does.
   The 2nd version was `Map[<schema>, (<first plan with this schema>, Map[<canonicalized plan>, <plan>])]` with lazy initialization of the inner map to avoid canonicalization if there are no matching schemas but still provide quick lookup by canonicalized plans.
   This 3rd version reverted to the original `Map[<schema>, ArrayBuffer[<plan>]]` idea that `ReuseExchange` and `ReuseSubquery` had used.
   
   I can open a follow-up PR to improve `ReuseMap` to 2nd version if required, but I'm not sure that the improvement would be visible with TPCDS or real life queries.
   
   If we want to consolidate reuse map logic then I think we should also take into account that `ReuseAdaptiveSubquery` uses a concurrent, lock-free `TrieMap` map implementation which is not required by this non-AQE rule.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-662911048


   Thanks @squito for the review, very appreciated.
   
   > I discussed this offline with Peter a little bit. This is not my area of expertise, so take my comments with a grain of salt.
   > 
   > * I think the internal details of `ReuseMap` are a little more complicated than they need to be, given the main purpose of this PR.  I'd prefer to defer the optimizations to a separate change, and keep the initial implementation simple.
   
   I'm fine with this. I can revert it and some UTs to it in the next commit. If anyone has any comments on this, please let us know.
   
   > * I'd like to see some unit tests on ReuseMap.  Even just really simple stuff, but at least making sure that if there are two things with a matching schema, you get the right one.
   
   Ok, I will add it soon.
   
   > * You mention a few cases when a ReusedExchange refers to something which no longer exists.  Does spark outright fail in those cases?  if so, I'd highlight that in the description, that these are bugs and not just performance improvements (maybe this obvious to everyone other than me)
   
   As I mentioned I don't know how to categorize this kind of issue, it doesn't affect the correctness of the result, but can cause serious performance issues: https://github.com/apache/spark/pull/28885#discussion_r459273107 
   
   > * I'm not totally clear which test covers part 3 of your description -- I guess its `ExchangeSuite."Exchange reuse across the whole plan"`? or maybe just part of it?  A comment in the PR description at least, and perhaps even one on the test itself would be helpful.
   
   3. is about whole plan reuse, this naturally comes from the 1 pass, whole plan, bottom-up traversal I suggest in this PR to fix 1. and 2. Please note that currently `ExchangeReuse` already does a limited version of multi-level exchange reuse to support DPP :
   ```
       } transformAllExpressions {
         // Lookup inside subqueries for duplicate exchanges
         case in: InSubqueryExec =>
           val newIn = in.plan.transformUp {
             case exchange: Exchange => reuse(exchange)
           }
           in.copy(plan = newIn.asInstanceOf[BaseSubqueryExec])
       }
   ```
   I don't see a reason why we couldn't extend this to other type of subqueries (`ScalarSubquery`) and to all levels of the plan which results the very clean new code in `ReuseExchangeAndSubquery`.
   I added test cases  to `DynamicPartitionPruningSuite`, `SubquerySuite` and `ExchangeSuite` cover it.
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r459314352



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala
##########
@@ -156,4 +158,46 @@ class ExchangeSuite extends SparkPlanTest with SharedSparkSession {
     val projection2 = cached.select("_1", "_3").queryExecution.executedPlan
     assert(!projection1.sameResult(projection2))
   }
+
+  test("Exchange reuse across the whole plan") {
+    val df = sql(
+      """
+        |SELECT
+        |  (SELECT max(a.key) FROM testData AS a JOIN testData AS b ON b.key = a.key),
+        |  a.key
+        |FROM testData AS a
+        |JOIN testData AS b ON b.key = a.key
+      """.stripMargin)
+
+    val plan = df.queryExecution.executedPlan
+
+    val exchangeIds = plan.collectWithSubqueries { case e: Exchange => e.id }
+    val reusedExchangeIds = plan.collectWithSubqueries {
+      case re: ReusedExchangeExec => re.child.id
+    }
+
+    assert(exchangeIds.size == 2, "Whole plan exchange reusing not working correctly")
+    assert(reusedExchangeIds.size == 3, "Whole plan exchange reusing not working correctly")
+    assert(reusedExchangeIds.forall(exchangeIds.contains(_)),
+      "ReusedExchangeExec should reuse an existing exchange")
+
+    val df2 = sql(

Review comment:
       Ok I will add some comments to here.
   
   About the one rule issue, please see 2. in the description and https://github.com/apache/spark/pull/28885#discussion_r459255589
   But let me add some more explanation. Currently the plan of the query in 2. is the following:
   ```
   == Physical Plan ==
   *(15) SortMergeJoin [id#46L], [id#58L], Inner
   :- *(7) Sort [id#46L ASC NULLS FIRST], false, 0
   :  +- Exchange hashpartitioning(id#46L, 5), true, [id=#979]
   :     +- *(6) HashAggregate(keys=[id#46L, k#49L], functions=[])
   :        +- Exchange hashpartitioning(id#46L, k#49L, 5), true, [id=#975]
   :           +- *(5) HashAggregate(keys=[id#46L, k#49L], functions=[])
   :              +- Union
   :                 :- *(2) Project [id#46L, k#49L]
   :                 :  +- *(2) BroadcastHashJoin [k#47L], [k#49L], Inner, BuildRight
   :                 :     :- *(2) Project [id#46L, k#47L]
   :                 :     :  +- *(2) Filter isnotnull(id#46L)
   :                 :     :     +- *(2) ColumnarToRow
   :                 :     :        +- FileScan parquet default.df1[id#46L,k#47L] Batched: true, DataFilters: [isnotnull(id#46L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [isnotnull(k#47L), dynamicpruningexpression(k#47L IN dynamicpruning#66)], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint>
   :                 :     :              +- SubqueryBroadcast dynamicpruning#66, 0, [k#49L], [id=#926]
   :                 :     :                 +- ReusedExchange [k#49L], BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, true])), [id=#656]
   :                 :     +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, true])), [id=#656]
   :                 :        +- *(1) Project [k#49L]
   :                 :           +- *(1) Filter ((isnotnull(id#48L) AND (id#48L < 2)) AND isnotnull(k#49L))
   :                 :              +- *(1) ColumnarToRow
   :                 :                 +- FileScan parquet default.df2[id#48L,k#49L] Batched: true, DataFilters: [isnotnull(id#48L), (id#48L < 2), isnotnull(k#49L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [], PushedFilters: [IsNotNull(id), LessThan(id,2), IsNotNull(k)], ReadSchema: struct<id:bigint,k:bigint>
   :                 +- *(4) Project [id#46L, k#49L]
   :                    +- *(4) BroadcastHashJoin [k#47L], [k#49L], Inner, BuildRight
   :                       :- *(4) Project [id#46L, k#47L]
   :                       :  +- *(4) Filter isnotnull(id#46L)
   :                       :     +- *(4) ColumnarToRow
   :                       :        +- FileScan parquet default.df1[id#46L,k#47L] Batched: true, DataFilters: [isnotnull(id#46L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [isnotnull(k#47L), dynamicpruningexpression(k#47L IN dynamicpruning#66)], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint>
   :                       :              +- ReusedSubquery SubqueryBroadcast dynamicpruning#66, 0, [k#49L], [id=#926]
   :                       +- ReusedExchange [k#49L], BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, true])), [id=#656]
   +- *(14) Sort [id#58L ASC NULLS FIRST], false, 0
      +- ReusedExchange [id#58L, k#61L], Exchange hashpartitioning(id#46L, 5), true, [id=#761] <== this reuse node points to a non-existing node
   ```
   Obviously the `ReusedExchange` pointing to the non-existing `id=#761` is the issue. It should be pointing to `id=#979`. But why did this happen? When the rule `ReuseExchange` inserted the `ReusedExchange` node into the plan, the id of the referenced exchange was `id=#761`. But then `ReuseSubquery` came and replaced the immutable `id=#761` node to another instance (`id=#979`). It didn't replace that particular exchange node, it just made a change under it somewhere and the change propagated up even to the root. (The nodes are immutable, so if you change a node in the tree all the ancestors will be replaced to a different instance). How do we know that this did happen due to the rule `ReuseSubquery`? Well there is a `ReusedSubquery` node under `id=#979` that can be there due to `ReuseSubquery` only.
   
   Just to mention all the details, the example of 2. contains the issue of 1. as well (I simply couldn't figure out an issue where only the 2. happens, but that doesn't mean that they are the same issue). You can find a `ReusedExchange` node pointing `id=#656`. That reuse node is correct, as `id=#656` does exist in the plan, but it is inserted under the exchange node `id=#761` by the second traversal (`transformAllExpressions`) in `ReuseExchange`, which also altered `id=#761` so in fact the in 2. the following events happened:
   - rule `ReuseExchange` inserted the `ReusedExchange` pointing to `id=#761` into the plan
   - rule `ReuseExchange` altered the exchange node `id=#761` to `id=X`
   - rule `ReuseSubquery` altered the  exchange node `id=X` to ~~`id=#975`~~`id=#979`
   
   The source of issues is that currently we might alter a node that a previous sweep was referenced to. The fix I suggest in this PR is to do 1 combined sweep, do it on the whole plan, and do it bottom-up manner.
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] attilapiros commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r455826311



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/util/ReuseMap.scala
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.util
+
+import scala.collection.mutable.Map
+import scala.language.existentials
+
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Map of canonicalized plans that can be used to find reuse possibilities.
+ *
+ * To avoid costly canonicalization of a plan:
+ * - we use its schema first to check if it can be replaced to a reused one at all
+ * - we insert it into the map of canonicalized plans only when at least 2 have the same schema
+ */
+class ReuseMap[T <: QueryPlan[_]] {
+  // scalastyle:off structural.type
+  private val map = Map[StructType, (T, Map[T2 forSome { type T2 >: T }, T])]()
+  // scalastyle:on structural.type
+
+  /**
+   * Find a matching plan with the same canonicalized form in the map or add the new plan to the
+   * map otherwise.
+   *
+   * @param plan the input plan
+   * @return the matching plan or the input plan
+   */
+  def lookupOrElseAdd(plan: T): T = {
+    val (firstSameSchemaPlan, sameResultPlans) = map.getOrElseUpdate(plan.schema, plan -> Map())
+    if (firstSameSchemaPlan ne plan) {
+      if (sameResultPlans.isEmpty) {
+        sameResultPlans += firstSameSchemaPlan.canonicalized -> firstSameSchemaPlan

Review comment:
       Could you please add more comments here. 
   
   Now I understand the `firstSameSchemaPlan` is also needed to be canonicalized but only when the `schema` reappears in multiple times but to get this I needed some offline help from you :) 
   
   And again thanks for explaining me this part!

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/util/ReuseMap.scala
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.util
+
+import scala.collection.mutable.Map
+import scala.language.existentials
+
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Map of canonicalized plans that can be used to find reuse possibilities.
+ *
+ * To avoid costly canonicalization of a plan:
+ * - we use its schema first to check if it can be replaced to a reused one at all
+ * - we insert it into the map of canonicalized plans only when at least 2 have the same schema
+ */
+class ReuseMap[T <: QueryPlan[_]] {
+  // scalastyle:off structural.type
+  private val map = Map[StructType, (T, Map[T2 forSome { type T2 >: T }, T])]()
+  // scalastyle:on structural.type
+
+  /**
+   * Find a matching plan with the same canonicalized form in the map or add the new plan to the
+   * map otherwise.
+   *
+   * @param plan the input plan
+   * @return the matching plan or the input plan
+   */
+  def lookupOrElseAdd(plan: T): T = {
+    val (firstSameSchemaPlan, sameResultPlans) = map.getOrElseUpdate(plan.schema, plan -> Map())
+    if (firstSameSchemaPlan ne plan) {
+      if (sameResultPlans.isEmpty) {
+        sameResultPlans += firstSameSchemaPlan.canonicalized -> firstSameSchemaPlan
+      }
+      sameResultPlans.getOrElseUpdate(plan.canonicalized, plan)
+    } else {
+      plan
+    }
+  }
+
+  /**
+   * Find a matching plan with the same canonicalized form in the map and apply `f` on it or add
+   * the new plan to the map otherwise.
+   *
+   * @param plan the input plan
+   * @param f the function to apply
+   * @return the matching plan with `f` applied or the input plan
+   */
+  def reuseOrElseAdd[T2 >: T](plan: T, f: T => T2): T2 = {

Review comment:
       Nit: this might be a better name `transformReusedOrAdd` and/or consider rename `f` to `funcOnReused`. What do you think?
   
   

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/util/ReuseMap.scala
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.util
+
+import scala.collection.mutable.Map
+import scala.language.existentials
+
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Map of canonicalized plans that can be used to find reuse possibilities.
+ *
+ * To avoid costly canonicalization of a plan:
+ * - we use its schema first to check if it can be replaced to a reused one at all
+ * - we insert it into the map of canonicalized plans only when at least 2 have the same schema
+ */
+class ReuseMap[T <: QueryPlan[_]] {
+  // scalastyle:off structural.type
+  private val map = Map[StructType, (T, Map[T2 forSome { type T2 >: T }, T])]()
+  // scalastyle:on structural.type
+
+  /**
+   * Find a matching plan with the same canonicalized form in the map or add the new plan to the
+   * map otherwise.
+   *
+   * @param plan the input plan
+   * @return the matching plan or the input plan
+   */
+  def lookupOrElseAdd(plan: T): T = {
+    val (firstSameSchemaPlan, sameResultPlans) = map.getOrElseUpdate(plan.schema, plan -> Map())
+    if (firstSameSchemaPlan ne plan) {

Review comment:
       Nit: Is this reference check needed only for the `getOrElseUpdate`? I might go for to use a simple `get` and handle the `None` by adding the `plan -> Map()`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-656634873


   @cloud-fan, @maryannxue, @maropu, @viirya could you please review this PR?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-667550219






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-660910914






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-652720581


   **[Test build #124810 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124810/testReport)** for PR 28885 at commit [`fd522ea`](https://github.com/apache/spark/commit/fd522ea3b9ddb3953122b4e9ac2e74ef34d66140).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r446633810



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala
##########
@@ -1646,4 +1646,25 @@ class SubquerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark
     checkAnswer(df, df2)
     checkAnswer(df, Nil)
   }
+
+  test("Subquery reuse across the whole plan") {
+    val df = sql(

Review comment:
       Sure, I added the plan in commit: https://github.com/apache/spark/pull/28885/commits/2b3cde22f62374c25b5748ff2253e168ec4e5d95
   Let me know if a sample plan would be beneficial in other test cases too.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-704846021






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-704837477


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34106/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r452926017



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/util/ReuseMap.scala
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.util
+
+import scala.collection.mutable.Map
+import scala.language.existentials
+
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Map of canonicalized plans that can be used to find reuse possibilities.
+ *
+ * To avoid costly canonicalization of a plan:
+ * - we use its schema first to check if it can be replaced to a reused one at all
+ * - we insert it into the map of canonicalized plans only when at least 2 have the same schema
+ */
+class ReuseMap[T <: QueryPlan[_]] {
+  // scalastyle:off structural.type
+  private val map = Map[StructType, (T, Map[T2 forSome { type T2 >: T }, T])]()

Review comment:
       If the existence type `T2`, which stands for the canonicalized type of `T`, is hard to read then we can move `T2` to the class definition like this:
   ```
   class ReuseMap[T <: QueryPlan[_], T2 >: T] {
     private val map = Map[StructType, (T, Map[T2, T])]()
   ```
   and initialize the `ReuseMap`s as:
   ```
         val exchanges = new ReuseMap[Exchange, SparkPlan]()
         val subqueries = new ReuseMap[BaseSubqueryExec, SparkPlan]()
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-707155690






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r459273107



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/ReuseExchangeAndSubquerySuite.scala
##########
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution
+
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.functions.col
+import org.apache.spark.sql.test.SharedSparkSession
+
+class ReuseExchangeAndSubquerySuite extends SparkPlanTest with SharedSparkSession {
+
+  val tableFormat: String = "parquet"
+
+  test("SPARK-32041: No reuse interference inside ReuseExchange") {
+    withTable("df1", "df2") {
+      spark.range(100)
+        .select(col("id"), col("id").as("k"))
+        .write
+        .partitionBy("k")
+        .format(tableFormat)
+        .mode("overwrite")
+        .saveAsTable("df1")
+
+      spark.range(10)
+        .select(col("id"), col("id").as("k"))
+        .write
+        .format(tableFormat)
+        .mode("overwrite")
+        .saveAsTable("df2")
+
+      val df = sql(
+        """
+          |WITH t AS (
+          |  SELECT df1.id, df2.k
+          |  FROM df1 JOIN df2 ON df1.k = df2.k
+          |  WHERE df2.id < 2
+          |)
+          |SELECT * FROM t AS a JOIN t AS b ON a.id = b.id
+          |""".stripMargin)
+
+      val plan = df.queryExecution.executedPlan
+
+      val exchangeIds = plan.collectWithSubqueries { case e: Exchange => e.id }
+      val reusedExchangeIds = plan.collectWithSubqueries {
+        case re: ReusedExchangeExec => re.child.id
+      }
+
+      assert(reusedExchangeIds.forall(exchangeIds.contains(_)),
+        "ReusedExchangeExec should reuse an existing exchange")

Review comment:
       I don't know how to categorize this issue. Currently, without this PR the query runs and returns the expected result but reuse doesn't happen, which can have serious negative impact on performance. See TPCDS Q14a benchmark results: https://github.com/apache/spark/pull/28885#issuecomment-647097876
   So I don't know if this PR is a bugfix or an improvement. @dongjoon-hyun any idea?
   
   When a reuse node points to an exchange or subquery instance that doesn't appear in the plan then that reuse node simply doesn't makes any sense. The plan will work and produce the expected results but the point of reuse is lost.
   
   To answer your question the test case without the PR fails due to the assert, but the query doesn't.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-705179016






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-660914307


   **[Test build #126174 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126174/testReport)** for PR 28885 at commit [`0d18e28`](https://github.com/apache/spark/commit/0d18e28637c07209dfdbc9c1777a6f458cd0ad69).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-650066534






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] squito commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
squito commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r460099447



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala
##########
@@ -156,4 +158,46 @@ class ExchangeSuite extends SparkPlanTest with SharedSparkSession {
     val projection2 = cached.select("_1", "_3").queryExecution.executedPlan
     assert(!projection1.sameResult(projection2))
   }
+
+  test("Exchange reuse across the whole plan") {
+    val df = sql(
+      """
+        |SELECT
+        |  (SELECT max(a.key) FROM testData AS a JOIN testData AS b ON b.key = a.key),
+        |  a.key
+        |FROM testData AS a
+        |JOIN testData AS b ON b.key = a.key
+      """.stripMargin)
+
+    val plan = df.queryExecution.executedPlan
+
+    val exchangeIds = plan.collectWithSubqueries { case e: Exchange => e.id }
+    val reusedExchangeIds = plan.collectWithSubqueries {
+      case re: ReusedExchangeExec => re.child.id
+    }
+
+    assert(exchangeIds.size == 2, "Whole plan exchange reusing not working correctly")
+    assert(reusedExchangeIds.size == 3, "Whole plan exchange reusing not working correctly")
+    assert(reusedExchangeIds.forall(exchangeIds.contains(_)),
+      "ReusedExchangeExec should reuse an existing exchange")
+
+    val df2 = sql(

Review comment:
       ah thanks, this helps a lot!
   but, then isn't this a problem for *any* other rule which further modifies the nodes, not just `ReuseSubquery`?  It seems to solve this properly, you actually want each `Exchange` to keep a reference to all the other exchanges which will reuse it, which is preserved across the transformations.  And then some final pass which just resolves these links?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-650181579






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-663126801






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-773464818


   **[Test build #134885 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/134885/testReport)** for PR 28885 at commit [`ec2324a`](https://github.com/apache/spark/commit/ec2324a25c9fb02272d84b29678155e240f5ce8f).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-691488559






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-691489458


   **[Test build #128592 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128592/testReport)** for PR 28885 at commit [`e1db0d4`](https://github.com/apache/spark/commit/e1db0d40624d9b202670df25b850dc982c5f0e2e).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-650065879


   **[Test build #124538 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124538/testReport)** for PR 28885 at commit [`8eca64a`](https://github.com/apache/spark/commit/8eca64aed6ddc30e731023ccf1d9ec32e4d95c21).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r446632302



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala
##########
@@ -474,9 +475,9 @@ class PlannerSuite extends SharedSparkSession with AdaptiveSparkPlanHelper {
       Inner,
       None,
       shuffle,
-      shuffle)
+      shuffle.copy())

Review comment:
       Yes, so I think the whole point of exchange reuse is to find different exchange instances which result the same data and keep only one instance and reuse its output where we can (i.e. call `execute`/`executeBroadcast` multiple times on one exchange instance instead of call `execute`/`executeBroadcast` once on each instances). In this example the 2 children points to the same exchange instance (`shuffle`) so there is no point in reuse here.
   (We could wrap one of them in a `ReusedExcahngeExec` node, but it wouldn't make any difference from performance point of view.)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-660180884


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/126051/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-661074360


   **[Test build #126174 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126174/testReport)** for PR 28885 at commit [`0d18e28`](https://github.com/apache/spark/commit/0d18e28637c07209dfdbc9c1777a6f458cd0ad69).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-766437383


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/39001/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r655261429



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/ReuseExchangeAndSubquery.scala
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.reuse
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern._
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.util.ReuseMap
+
+/**
+ * Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
+ * use the same exchange or subquery for all the references.
+ *
+ * Note that the Spark plan is a mutually recursive data structure:
+ * SparkPlan -> Expr -> Subquery -> SparkPlan -> Expr -> Subquery -> ...
+ * Therefore, in this rule, we recursively rewrite the exchanges and subqueries in a bottom-up way,
+ * in one go.
+ */
+case object ReuseExchangeAndSubquery extends Rule[SparkPlan] {

Review comment:
       Yea non-AQE doesn't need thread safety, but I feel it's still better to unify the major idea:
   1. Map[<canonicalized plan>, <plan>]
   2. Map[<schema>, ArrayBuffer[<plan>]]
   
   I agree it's not a big deal for perf, but code consistency is also important. `Map[<canonicalized plan>, <plan>]` looks better as it's simpler, we can remove `ReuseMap`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-766943772


   **[Test build #134453 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/134453/testReport)** for PR 28885 at commit [`4db5801`](https://github.com/apache/spark/commit/4db5801f10888c7566d8beffe193583cc83f1031).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r655142981



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/ReuseExchangeAndSubquery.scala
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.reuse
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern._
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.util.ReuseMap
+
+/**
+ * Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
+ * use the same exchange or subquery for all the references.
+ *
+ * Note that the Spark plan is a mutually recursive data structure:
+ * SparkPlan -> Expr -> Subquery -> SparkPlan -> Expr -> Subquery -> ...
+ * Therefore, in this rule, we recursively rewrite the exchanges and subqueries in a bottom-up way,
+ * in one go.
+ */
+case object ReuseExchangeAndSubquery extends Rule[SparkPlan] {

Review comment:
       `ReuseMap` has changed since the first version of this PR.
   Unfortunately, I rebased the PR already so only some discussion remained: https://github.com/apache/spark/pull/28885#discussion_r455692637 (about reverting 2nd to 3rd).
   
   The 1st version used a simple `Map[<canonicalized plan>, <plan>]` as `AdaptiveExecutionContext` does.
   The 2nd version was `Map[<schema>, Map[<canonicalized plan>, <plan>]]` with lazy initialization of the inner map to avoid canonicalization if there are no matching schemas but still provide quick lookup by canonicalized plans.
   This 3rd version reverted to the original `Map[<schema>, ArrayBuffer[<plan>]] idea that `ReuseExchange` and `ReuseSubquery` had used.
   
   I can open a follow-up PR to improve `ReuseMap` to 2nd version if you think so, but I'm not sure that the improvement would be visible with TPCDS or real life queries.
   
   If we want to consolidate reuse map logic then I think we should also take into account that `ReuseAdaptiveSubquery` uses a concurrent, lock-free `TrieMap` map implementation which is not required by this non-AQE rule.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-767530441


   Now that https://github.com/apache/spark/pull/31243 got merged the invalid reuse references (`Reuses operator id: unknown`) show up in many golden files. Each of them mean a missed reuse opportunity and a wrong explain plan.
   
   Before this PR:
   ```
   tpcds-plan-stability petertoth$ grep -Ril "Reuses operator id: unknown" . | sort
   ./approved-plans-v1_4/q14a.sf100/explain.txt
   ./approved-plans-v1_4/q14a/explain.txt
   ./approved-plans-v1_4/q14b.sf100/explain.txt
   ./approved-plans-v1_4/q14b/explain.txt
   ./approved-plans-v1_4/q23a.sf100/explain.txt
   ./approved-plans-v1_4/q23a/explain.txt
   ./approved-plans-v1_4/q23b.sf100/explain.txt
   ./approved-plans-v1_4/q23b/explain.txt
   ./approved-plans-v1_4/q47.sf100/explain.txt
   ./approved-plans-v1_4/q47/explain.txt
   ./approved-plans-v1_4/q57.sf100/explain.txt
   ./approved-plans-v1_4/q57/explain.txt
   ./approved-plans-v2_7/q14.sf100/explain.txt
   ./approved-plans-v2_7/q14/explain.txt
   ./approved-plans-v2_7/q14a.sf100/explain.txt
   ./approved-plans-v2_7/q14a/explain.txt
   ./approved-plans-v2_7/q18a.sf100/explain.txt
   ./approved-plans-v2_7/q22a.sf100/explain.txt
   ./approved-plans-v2_7/q22a/explain.txt
   ./approved-plans-v2_7/q36a.sf100/explain.txt
   ./approved-plans-v2_7/q36a/explain.txt
   ./approved-plans-v2_7/q47.sf100/explain.txt
   ./approved-plans-v2_7/q47/explain.txt
   ./approved-plans-v2_7/q51a.sf100/explain.txt
   ./approved-plans-v2_7/q51a/explain.txt
   ./approved-plans-v2_7/q57.sf100/explain.txt
   ./approved-plans-v2_7/q57/explain.txt
   ./approved-plans-v2_7/q5a.sf100/explain.txt
   ./approved-plans-v2_7/q5a/explain.txt
   ./approved-plans-v2_7/q67a.sf100/explain.txt
   ./approved-plans-v2_7/q67a/explain.txt
   ./approved-plans-v2_7/q70a.sf100/explain.txt
   ./approved-plans-v2_7/q70a/explain.txt
   ./approved-plans-v2_7/q77a.sf100/explain.txt
   ./approved-plans-v2_7/q77a/explain.txt
   ./approved-plans-v2_7/q80a.sf100/explain.txt
   ./approved-plans-v2_7/q80a/explain.txt
   ./approved-plans-v2_7/q86a.sf100/explain.txt
   ./approved-plans-v2_7/q86a/explain.txt
   ```
   
   After this PR:
   ```
   tpcds-plan-stability petertoth$ grep -Ril "Reuses operator id: unknown" . | sort
   tpcds-plan-stability petertoth$
   ```


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-776252896


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/39665/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-776355721


   **[Test build #135083 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135083/testReport)** for PR 28885 at commit [`c3cbdb2`](https://github.com/apache/spark/commit/c3cbdb2d6ff9c42b34fa1850645bb248cdc8fe6b).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `trait UserDefinedExpression `
     * `  implicit class MetadataColumnHelper(attr: Attribute) `
     * `case class SubqueryAdaptiveBroadcastExec(`
     * `case class PlanAdaptiveDynamicPruningFilters(`
     * `case class PlanAdaptiveSubqueries(`


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r656118091



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/ReuseExchangeAndSubquery.scala
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.reuse
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern._
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.util.ReuseMap
+
+/**
+ * Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
+ * use the same exchange or subquery for all the references.
+ *
+ * Note that the Spark plan is a mutually recursive data structure:
+ * SparkPlan -> Expr -> Subquery -> SparkPlan -> Expr -> Subquery -> ...
+ * Therefore, in this rule, we recursively rewrite the exchanges and subqueries in a bottom-up way,
+ * in one go.
+ */
+case object ReuseExchangeAndSubquery extends Rule[SparkPlan] {

Review comment:
       I've opened the follow-up PR here: https://github.com/apache/spark/pull/33021




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-864803292


   > thanks, merging to master!
   
   Thanks for the review!
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-705367434


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34148/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-661075328






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-691436044






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-691435855


   **[Test build #128590 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128590/testReport)** for PR 28885 at commit [`050ab40`](https://github.com/apache/spark/commit/050ab40ff2ada204ebc5c231818969e568cd003f).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [WIP][SPARK-29375][SPARK-28940][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-647094520






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-647094460


   **[Test build #124334 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124334/testReport)** for PR 28885 at commit [`dbd8606`](https://github.com/apache/spark/commit/dbd860667ed3cae4c56ebddbb214056c426ef9dd).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-652578311


   **[Test build #124810 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124810/testReport)** for PR 28885 at commit [`fd522ea`](https://github.com/apache/spark/commit/fd522ea3b9ddb3953122b4e9ac2e74ef34d66140).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-707017312


   **[Test build #129680 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129680/testReport)** for PR 28885 at commit [`b4bcbce`](https://github.com/apache/spark/commit/b4bcbce03b0d1bc763554cba31b9bbc2cb016a0b).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-704846021






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-652242395






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r446632000



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/Reuse.scala
##########
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.reuse
+
+import scala.collection.mutable.Map
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
+ * use the same exhange or subquery for all the references.
+ */
+case class WholePlanReuse(conf: SQLConf) extends Rule[SparkPlan] {
+
+  def apply(plan: SparkPlan): SparkPlan = {
+    if (conf.exchangeReuseEnabled || conf.subqueryReuseEnabled) {
+      // To avoid costly canonicalization of an exchange or a subquery:

Review comment:
       I think the `Map[StructType, ...` way of caching has been there for quite some time. A simple map of canonicalized plans naturally comes to my mind too and I feel that it would do the thing without any performance degradation for most of the queries. But I'm afraid that there can be edge cases where it could introduce degradation so just to be on the safe side I wouldn't touch this preliminary schema matching when looking up in the cache.
   On the other hand I think the old `ArrayBuffer[...` can be easily replaced to a map of canonicalized plans to speed up look ups in the cache when schema matches.
   I saw your other comment: https://github.com/apache/spark/pull/28885#discussion_r446516852 on this topic and I think the `Canonicalized[T]` wrapper would be exactly same as the old `Map[StructType, ArrayBuffer[T]]` cache map, just a bit more complicated.
   What I did in my latest commit: https://github.com/apache/spark/pull/28885/commits/c49a0f99b25522dd7ed89e8b41d2c5c80dbaf170 is that I extracted the cache code and I think it became quite easy to follow. But I'm open for suggestions and will change the implementation if you think it is still too complicated.
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-662960607


   **[Test build #126408 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126408/testReport)** for PR 28885 at commit [`5df4c53`](https://github.com/apache/spark/commit/5df4c53d8f2805699bbcc2eae99380b59fd3ac3d).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-803009280


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/136257/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-683283412


   @cloud-fan, @maropu, @viirya, can you please help me how to move forward with this PR?
   The latest commit updates expected plans of PlanStability suites where you can see the new reuse nodes this PR adds to TPCDS queries. My benchmarks showed that this PR brings ~30% improvement to some of the queries. Please let me know if you have any concerns with this PR.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-683281617


   **[Test build #128013 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128013/testReport)** for PR 28885 at commit [`cd064e4`](https://github.com/apache/spark/commit/cd064e460f90984d57397bac499058a261bc7205).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dbaliafroozeh commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
dbaliafroozeh commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r447624697



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/Reuse.scala
##########
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.reuse
+
+import scala.collection.mutable.Map
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
+ * use the same exhange or subquery for all the references.
+ */
+case class ReuseExchangeAndSubquery(conf: SQLConf) extends Rule[SparkPlan] {
+
+  private class ReuseCache[T <: SparkPlan] {
+    // To avoid costly canonicalization of an exchange or a subquery:
+    // - we use its schema first to check if it can be replaced to a reused one at all
+    // - we insert it into the map of canonicalized plans only when at least 2 have the same schema
+    private val cache = Map[StructType, (T, Map[SparkPlan, T])]()
+
+    def lookup(plan: T): T = {
+      val (firstSameSchemaPlan, sameResultPlans) = cache.getOrElseUpdate(plan.schema, plan -> Map())
+      if (firstSameSchemaPlan.ne(plan)) {
+        if (sameResultPlans.isEmpty) {
+          sameResultPlans += firstSameSchemaPlan.canonicalized -> firstSameSchemaPlan
+        }
+        sameResultPlans.getOrElseUpdate(plan.canonicalized, plan)
+      } else {
+        plan
+      }
+    }
+  }
+
+  def apply(plan: SparkPlan): SparkPlan = {
+    if (conf.exchangeReuseEnabled || conf.subqueryReuseEnabled) {
+      val exchanges = new ReuseCache[Exchange]()
+      val subqueries = new ReuseCache[BaseSubqueryExec]()
+
+      def reuse(plan: SparkPlan): SparkPlan = plan.transformUp {
+        case exchange: Exchange if conf.exchangeReuseEnabled =>
+          val cached = exchanges.lookup(exchange)
+          if (cached.ne(exchange)) {
+            ReusedExchangeExec(exchange.output, cached)
+          } else {
+            exchange
+          }
+
+        case other => other.transformExpressionsUp {
+          case sub: ExecSubqueryExpression =>
+            val subquery = reuse(sub.plan).asInstanceOf[BaseSubqueryExec]
+            sub.withNewPlan(
+              if (conf.subqueryReuseEnabled) {

Review comment:
       `reuseOrElseAdd` is probably a better name, but this naming is getting hard :-) About the new behavior, sounds good to me, but probably if there is no performance benefit and it's only happening in the tests, I would have just wrap the same instance in a reuse node to also not bother about the second method and the equality check, but don't have a strong opinion on this.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r655142981



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/ReuseExchangeAndSubquery.scala
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.reuse
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern._
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.util.ReuseMap
+
+/**
+ * Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
+ * use the same exchange or subquery for all the references.
+ *
+ * Note that the Spark plan is a mutually recursive data structure:
+ * SparkPlan -> Expr -> Subquery -> SparkPlan -> Expr -> Subquery -> ...
+ * Therefore, in this rule, we recursively rewrite the exchanges and subqueries in a bottom-up way,
+ * in one go.
+ */
+case object ReuseExchangeAndSubquery extends Rule[SparkPlan] {

Review comment:
       `ReuseMap` has changed since the first version of this PR.
   Unfortunately, I rebased the PR already so only some discussion remained: https://github.com/apache/spark/pull/28885#discussion_r455692637 (about reverting 2nd to 3rd).
   
   The 1st version used a simple `Map[<canonicalized plan>, <plan>]` as `AdaptiveExecutionContext` does.
   The 2nd version was `Map[<schema>, Map[<canonicalized plan>, <plan>]]` with lazy initialization of the inner map to avoid canonicalization if there are no matching schemas but still provide quick lookup by canonicalized plans.
   This 3rd version reverted to the original `Map[<schema>, ArrayBuffer[<plan>]]` idea that `ReuseExchange` and `ReuseSubquery` had used.
   
   I can open a follow-up PR to improve `ReuseMap` to 2nd version if required, but I'm not sure that the improvement would be visible with TPCDS or real life queries.
   
   If we want to consolidate reuse map logic then I think we should also take into account that `ReuseAdaptiveSubquery` uses a concurrent, lock-free `TrieMap` map implementation which is not required by this non-AQE rule.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan closed pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
cloud-fan closed pull request #28885:
URL: https://github.com/apache/spark/pull/28885


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-766984435


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/39039/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-776230110


   **[Test build #135083 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135083/testReport)** for PR 28885 at commit [`c3cbdb2`](https://github.com/apache/spark/commit/c3cbdb2d6ff9c42b34fa1850645bb248cdc8fe6b).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r466351098



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala
##########
@@ -156,4 +158,46 @@ class ExchangeSuite extends SparkPlanTest with SharedSparkSession {
     val projection2 = cached.select("_1", "_3").queryExecution.executedPlan
     assert(!projection1.sameResult(projection2))
   }
+
+  test("Exchange reuse across the whole plan") {
+    val df = sql(
+      """
+        |SELECT
+        |  (SELECT max(a.key) FROM testData AS a JOIN testData AS b ON b.key = a.key),
+        |  a.key
+        |FROM testData AS a
+        |JOIN testData AS b ON b.key = a.key
+      """.stripMargin)
+
+    val plan = df.queryExecution.executedPlan
+
+    val exchangeIds = plan.collectWithSubqueries { case e: Exchange => e.id }
+    val reusedExchangeIds = plan.collectWithSubqueries {
+      case re: ReusedExchangeExec => re.child.id
+    }
+
+    assert(exchangeIds.size == 2, "Whole plan exchange reusing not working correctly")
+    assert(reusedExchangeIds.size == 3, "Whole plan exchange reusing not working correctly")
+    assert(reusedExchangeIds.forall(exchangeIds.contains(_)),
+      "ReusedExchangeExec should reuse an existing exchange")
+
+    val df2 = sql(

Review comment:
       Some of the rules already require specific ordering (please see batches in `Optimizer`) and we have one that requires to be the first (`InsertAdaptiveSparkPlan` in AQE) so I don't think that `ReuseExchangeAndSubquery` to be the last would be unique. Actually I suspect `ReuseExchange` and `ReuseSubquery` were added as last by design, but it was not documented.
   
   Do you think it is ok to note this requirement of the new rule in its scaladoc in this PR and maybe we can try to find a better mechanism in a follow-up PR?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-766425750


   **[Test build #134415 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/134415/testReport)** for PR 28885 at commit [`8f24add`](https://github.com/apache/spark/commit/8f24adda6fc72262cc50054e48b82c73a3cc213f).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-707154358


   **[Test build #129680 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129680/testReport)** for PR 28885 at commit [`b4bcbce`](https://github.com/apache/spark/commit/b4bcbce03b0d1bc763554cba31b9bbc2cb016a0b).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r655142981



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/ReuseExchangeAndSubquery.scala
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.reuse
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern._
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.util.ReuseMap
+
+/**
+ * Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
+ * use the same exchange or subquery for all the references.
+ *
+ * Note that the Spark plan is a mutually recursive data structure:
+ * SparkPlan -> Expr -> Subquery -> SparkPlan -> Expr -> Subquery -> ...
+ * Therefore, in this rule, we recursively rewrite the exchanges and subqueries in a bottom-up way,
+ * in one go.
+ */
+case object ReuseExchangeAndSubquery extends Rule[SparkPlan] {

Review comment:
       `ReuseMap` has changed since the first version of this PR.
   Unfortunately, I rebased the PR already so only some discussion remained: https://github.com/apache/spark/pull/28885#discussion_r455692637 (about reverting 2nd to 3rd).
   
   The 1st version used a simple `Map[<canonicalized plan>, <plan>]` as `AdaptiveExecutionContext` does.
   The 2nd version was `Map[<schema>, (<first plan with this schema>, Map[<canonicalized plan>, <plan>])]` with lazy initialization of the inner map to avoid canonicalization if there are no matching schemas but still provide quick lookup by canonicalized plans.
   This 3rd version reverted to the original `Map[<schema>, ArrayBuffer[<plan>]]` idea that `ReuseExchange` and `ReuseSubquery` had used.
   
   I can open a follow-up PR to improve `ReuseMap` to 2nd version if required, but I'm not sure that the improvement would be visible with TPCDS or real life queries.
   
   If we want to consolidate reuse map logic then I think we should also take into account that `ReuseAdaptiveSubquery` uses a concurrent, lock-free `TrieMap` map implementation which is not required by this non-AQE rule.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-683252321






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-766433954






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-705391128






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-763092763


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/134232/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [WIP][SPARK-29375][SPARK-28940][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r443198465



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
##########
@@ -336,12 +336,13 @@ object QueryPlan extends PredicateHelper {
    * `Attribute`, and replace it with `BoundReference` will cause error.
    */
   def normalizeExpressions[T <: Expression](e: T, input: AttributeSeq): T = {
+    type T2 = QueryPlan[_]
     e.transformUp {
-      case s: PlanExpression[QueryPlan[_] @unchecked] =>
+      case s: PlanExpression[T2 @unchecked] =>
         // Normalize the outer references in the subquery plan.
         val normalizedPlan = s.plan.transformAllExpressions {
           case OuterReference(r) => OuterReference(QueryPlan.normalizeExpressions(r, input))
-        }
+        }.canonicalized.asInstanceOf[T2]

Review comment:
       As we traverse bottom-up manner across the whole plan, we might run into a case when there is a reuse opportunity contains a subquery. If we don't canonicalize that subquery then we miss it.
   
   An example is `"join key with multiple references on the filtering plan"` test in `DynamicPartitionPruningSuiteBase where without this line of change only a "smaller" subquery reuse happens:
   ```
   == Physical Plan ==
   *(2) Project [id#2729L, a#2730, b#2731]
   +- *(2) BroadcastHashJoin [knownfloatingpointnormalized(normalizenanandzero((cast(b#2731 as double) + cast(a#2730 as double))))], [knownfloatingpointnormalized(normalizenanandzero((cast(y#2734 as double) + cast(z#2735 as double))))], Inner, BuildRight
      :- *(2) ColumnarToRow
      :  +- FileScan parquet default.fact[id#2729L,a#2730,b#2731] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [isnotnull(b#2731), isnotnull(a#2730), dynamicpruningexpression(knownfloatingpointnormalized(norm..., PushedFilters: [], ReadSchema: struct<id:bigint>
      :        +- SubqueryBroadcast dynamicpruning#2742, 0, [knownfloatingpointnormalized(normalizenanandzero((cast(y#2734 as double) + cast(z#2735 as double))))], [id=#274]
      :           +- BroadcastExchange HashedRelationBroadcastMode(List(knownfloatingpointnormalized(normalizenanandzero((cast(input[0, string, true] as double) + cast(input[1, string, true] as double)))))), [id=#273]
      :              +- *(1) Project [y#2734, z#2735]
      :                 +- *(1) Filter (((isnotnull(x#2733) AND (cast(x#2733 as double) = Subquery scalar-subquery#2728, [id=#262])) AND isnotnull(y#2734)) AND isnotnull(z#2735))
      :                    :  +- Subquery scalar-subquery#2728, [id=#262]
      :                    :     +- *(2) HashAggregate(keys=[], functions=[avg(cast(w#2736 as double))])
      :                    :        +- Exchange SinglePartition, true, [id=#258]
      :                    :           +- *(1) HashAggregate(keys=[], functions=[partial_avg(cast(w#2736 as double))])
      :                    :              +- *(1) ColumnarToRow
      :                    :                 +- FileScan parquet default.dim[w#2736] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct<w:string>
      :                    +- *(1) ColumnarToRow
      :                       +- FileScan parquet default.dim[x#2733,y#2734,z#2735] Batched: true, DataFilters: [isnotnull(x#2733), isnotnull(y#2734), isnotnull(z#2735)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [], PushedFilters: [IsNotNull(x), IsNotNull(y), IsNotNull(z)], ReadSchema: struct<x:string,y:string,z:string>
      +- BroadcastExchange HashedRelationBroadcastMode(List(knownfloatingpointnormalized(normalizenanandzero((cast(input[0, string, true] as double) + cast(input[1, string, true] as double)))))), [id=#359]
         +- *(1) Project [y#2734, z#2735]
            +- *(1) Filter (((isnotnull(x#2733) AND (cast(x#2733 as double) = ReusedSubquery Subquery scalar-subquery#2728, [id=#262])) AND isnotnull(y#2734)) AND isnotnull(z#2735))
               :  +- ReusedSubquery Subquery scalar-subquery#2728, [id=#262]
               +- *(1) ColumnarToRow
                  +- FileScan parquet default.dim[x#2733,y#2734,z#2735] Batched: true, DataFilters: [isnotnull(x#2733), isnotnull(y#2734), isnotnull(z#2735)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [], PushedFilters: [IsNotNull(x), IsNotNull(y), IsNotNull(z)], ReadSchema: struct<x:string,y:string,z:string>
   ```
   but with this line of change we get a "bigger" exchange reuse:
   ```
   *(2) Project [id#2729L, a#2730, b#2731]
   +- *(2) BroadcastHashJoin [knownfloatingpointnormalized(normalizenanandzero((cast(b#2731 as double) + cast(a#2730 as double))))], [knownfloatingpointnormalized(normalizenanandzero((cast(y#2734 as double) + cast(z#2735 as double))))], Inner, BuildRight
      :- *(2) ColumnarToRow
      :  +- FileScan parquet default.fact[id#2729L,a#2730,b#2731] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [isnotnull(b#2731), isnotnull(a#2730), dynamicpruningexpression(knownfloatingpointnormalized(norm..., PushedFilters: [], ReadSchema: struct<id:bigint>
      :        +- SubqueryBroadcast dynamicpruning#2742, 0, [knownfloatingpointnormalized(normalizenanandzero((cast(y#2734 as double) + cast(z#2735 as double))))], [id=#274]
      :           +- BroadcastExchange HashedRelationBroadcastMode(List(knownfloatingpointnormalized(normalizenanandzero((cast(input[0, string, true] as double) + cast(input[1, string, true] as double)))))), [id=#273]
      :              +- *(1) Project [y#2734, z#2735]
      :                 +- *(1) Filter (((isnotnull(x#2733) AND (cast(x#2733 as double) = Subquery scalar-subquery#2728, [id=#262])) AND isnotnull(y#2734)) AND isnotnull(z#2735))
      :                    :  +- Subquery scalar-subquery#2728, [id=#262]
      :                    :     +- *(2) HashAggregate(keys=[], functions=[avg(cast(w#2736 as double))])
      :                    :        +- Exchange SinglePartition, true, [id=#258]
      :                    :           +- *(1) HashAggregate(keys=[], functions=[partial_avg(cast(w#2736 as double))])
      :                    :              +- *(1) ColumnarToRow
      :                    :                 +- FileScan parquet default.dim[w#2736] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct<w:string>
      :                    +- *(1) ColumnarToRow
      :                       +- FileScan parquet default.dim[x#2733,y#2734,z#2735] Batched: true, DataFilters: [isnotnull(x#2733), isnotnull(y#2734), isnotnull(z#2735)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [], PushedFilters: [IsNotNull(x), IsNotNull(y), IsNotNull(z)], ReadSchema: struct<x:string,y:string,z:string>
      +- ReusedExchange [y#2734, z#2735], BroadcastExchange HashedRelationBroadcastMode(List(knownfloatingpointnormalized(normalizenanandzero((cast(input[0, string, true] as double) + cast(input[1, string, true] as double)))))), [id=#273]
   ```

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
##########
@@ -336,12 +336,13 @@ object QueryPlan extends PredicateHelper {
    * `Attribute`, and replace it with `BoundReference` will cause error.
    */
   def normalizeExpressions[T <: Expression](e: T, input: AttributeSeq): T = {
+    type T2 = QueryPlan[_]
     e.transformUp {
-      case s: PlanExpression[QueryPlan[_] @unchecked] =>
+      case s: PlanExpression[T2 @unchecked] =>
         // Normalize the outer references in the subquery plan.
         val normalizedPlan = s.plan.transformAllExpressions {
           case OuterReference(r) => OuterReference(QueryPlan.normalizeExpressions(r, input))
-        }
+        }.canonicalized.asInstanceOf[T2]

Review comment:
       As we traverse bottom-up manner across the whole plan, we might run into a case when there is a reuse opportunity contains a subquery. If we don't canonicalize that subquery then we miss it.
   
   An example is `"join key with multiple references on the filtering plan"` test in `DynamicPartitionPruningSuiteBase` where without this line of change only a "smaller" subquery reuse happens:
   ```
   == Physical Plan ==
   *(2) Project [id#2729L, a#2730, b#2731]
   +- *(2) BroadcastHashJoin [knownfloatingpointnormalized(normalizenanandzero((cast(b#2731 as double) + cast(a#2730 as double))))], [knownfloatingpointnormalized(normalizenanandzero((cast(y#2734 as double) + cast(z#2735 as double))))], Inner, BuildRight
      :- *(2) ColumnarToRow
      :  +- FileScan parquet default.fact[id#2729L,a#2730,b#2731] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [isnotnull(b#2731), isnotnull(a#2730), dynamicpruningexpression(knownfloatingpointnormalized(norm..., PushedFilters: [], ReadSchema: struct<id:bigint>
      :        +- SubqueryBroadcast dynamicpruning#2742, 0, [knownfloatingpointnormalized(normalizenanandzero((cast(y#2734 as double) + cast(z#2735 as double))))], [id=#274]
      :           +- BroadcastExchange HashedRelationBroadcastMode(List(knownfloatingpointnormalized(normalizenanandzero((cast(input[0, string, true] as double) + cast(input[1, string, true] as double)))))), [id=#273]
      :              +- *(1) Project [y#2734, z#2735]
      :                 +- *(1) Filter (((isnotnull(x#2733) AND (cast(x#2733 as double) = Subquery scalar-subquery#2728, [id=#262])) AND isnotnull(y#2734)) AND isnotnull(z#2735))
      :                    :  +- Subquery scalar-subquery#2728, [id=#262]
      :                    :     +- *(2) HashAggregate(keys=[], functions=[avg(cast(w#2736 as double))])
      :                    :        +- Exchange SinglePartition, true, [id=#258]
      :                    :           +- *(1) HashAggregate(keys=[], functions=[partial_avg(cast(w#2736 as double))])
      :                    :              +- *(1) ColumnarToRow
      :                    :                 +- FileScan parquet default.dim[w#2736] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct<w:string>
      :                    +- *(1) ColumnarToRow
      :                       +- FileScan parquet default.dim[x#2733,y#2734,z#2735] Batched: true, DataFilters: [isnotnull(x#2733), isnotnull(y#2734), isnotnull(z#2735)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [], PushedFilters: [IsNotNull(x), IsNotNull(y), IsNotNull(z)], ReadSchema: struct<x:string,y:string,z:string>
      +- BroadcastExchange HashedRelationBroadcastMode(List(knownfloatingpointnormalized(normalizenanandzero((cast(input[0, string, true] as double) + cast(input[1, string, true] as double)))))), [id=#359]
         +- *(1) Project [y#2734, z#2735]
            +- *(1) Filter (((isnotnull(x#2733) AND (cast(x#2733 as double) = ReusedSubquery Subquery scalar-subquery#2728, [id=#262])) AND isnotnull(y#2734)) AND isnotnull(z#2735))
               :  +- ReusedSubquery Subquery scalar-subquery#2728, [id=#262]
               +- *(1) ColumnarToRow
                  +- FileScan parquet default.dim[x#2733,y#2734,z#2735] Batched: true, DataFilters: [isnotnull(x#2733), isnotnull(y#2734), isnotnull(z#2735)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [], PushedFilters: [IsNotNull(x), IsNotNull(y), IsNotNull(z)], ReadSchema: struct<x:string,y:string,z:string>
   ```
   but with this line of change we get a "bigger" exchange reuse:
   ```
   *(2) Project [id#2729L, a#2730, b#2731]
   +- *(2) BroadcastHashJoin [knownfloatingpointnormalized(normalizenanandzero((cast(b#2731 as double) + cast(a#2730 as double))))], [knownfloatingpointnormalized(normalizenanandzero((cast(y#2734 as double) + cast(z#2735 as double))))], Inner, BuildRight
      :- *(2) ColumnarToRow
      :  +- FileScan parquet default.fact[id#2729L,a#2730,b#2731] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [isnotnull(b#2731), isnotnull(a#2730), dynamicpruningexpression(knownfloatingpointnormalized(norm..., PushedFilters: [], ReadSchema: struct<id:bigint>
      :        +- SubqueryBroadcast dynamicpruning#2742, 0, [knownfloatingpointnormalized(normalizenanandzero((cast(y#2734 as double) + cast(z#2735 as double))))], [id=#274]
      :           +- BroadcastExchange HashedRelationBroadcastMode(List(knownfloatingpointnormalized(normalizenanandzero((cast(input[0, string, true] as double) + cast(input[1, string, true] as double)))))), [id=#273]
      :              +- *(1) Project [y#2734, z#2735]
      :                 +- *(1) Filter (((isnotnull(x#2733) AND (cast(x#2733 as double) = Subquery scalar-subquery#2728, [id=#262])) AND isnotnull(y#2734)) AND isnotnull(z#2735))
      :                    :  +- Subquery scalar-subquery#2728, [id=#262]
      :                    :     +- *(2) HashAggregate(keys=[], functions=[avg(cast(w#2736 as double))])
      :                    :        +- Exchange SinglePartition, true, [id=#258]
      :                    :           +- *(1) HashAggregate(keys=[], functions=[partial_avg(cast(w#2736 as double))])
      :                    :              +- *(1) ColumnarToRow
      :                    :                 +- FileScan parquet default.dim[w#2736] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct<w:string>
      :                    +- *(1) ColumnarToRow
      :                       +- FileScan parquet default.dim[x#2733,y#2734,z#2735] Batched: true, DataFilters: [isnotnull(x#2733), isnotnull(y#2734), isnotnull(z#2735)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [], PushedFilters: [IsNotNull(x), IsNotNull(y), IsNotNull(z)], ReadSchema: struct<x:string,y:string,z:string>
      +- ReusedExchange [y#2734, z#2735], BroadcastExchange HashedRelationBroadcastMode(List(knownfloatingpointnormalized(normalizenanandzero((cast(input[0, string, true] as double) + cast(input[1, string, true] as double)))))), [id=#273]
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-707053020


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34285/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-683252321






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-773632699


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/134885/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r654293110



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/ExplainUtils.scala
##########
@@ -41,22 +38,15 @@ object ExplainUtils extends AdaptiveSparkPlanHelper {
    *
    * @param plan Input query plan to process
    * @param append function used to append the explain output
-   * @param startOperatorID The start value of operation id. The subsequent operations will
-   *                         be assigned higher value.
    *
    * @return The last generated operation id for this input plan. This is to ensure we

Review comment:
       Fixed in https://github.com/apache/spark/pull/28885/commits/7187ebd2e053570d92017100dba4a0738fa2f014.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/ReuseExchangeAndSubquery.scala
##########
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.reuse
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern._
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.util.ReuseMap
+
+/**
+ * Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
+ * use the same exchange or subquery for all the references.

Review comment:
       Ok, added the explanation in https://github.com/apache/spark/pull/28885/commits/7187ebd2e053570d92017100dba4a0738fa2f014

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/util/ReuseMap.scala
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.util
+
+import scala.collection.mutable.{ArrayBuffer, Map}
+
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Map of canonicalized plans that can be used to find reuse possibilities.
+ *
+ * To avoid costly canonicalization of a plan:
+ * - we use its schema first to check if it can be replaced to a reused one at all
+ * - we insert it into the map of canonicalized plans only when at least 2 have the same schema
+ *
+ * @tparam T the type of the node we want to reuse
+ * @tparam T2 the type of the canonicalized node
+ */
+class ReuseMap[T <: T2, T2 <: QueryPlan[T2]] {
+  private val map = Map[StructType, ArrayBuffer[T]]()
+
+  /**
+   * Find a matching plan with the same canonicalized form in the map or add the new plan to the
+   * map otherwise.
+   *
+   * @param plan the input plan
+   * @return the matching plan or the input plan
+   */
+  def lookupOrElseAdd(plan: T): T = {

Review comment:
       Fixed in https://github.com/apache/spark/pull/28885/commits/7187ebd2e053570d92017100dba4a0738fa2f014




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-776362027


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/135083/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-662961008






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-650623357


   **[Test build #124572 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124572/testReport)** for PR 28885 at commit [`c49a0f9`](https://github.com/apache/spark/commit/c49a0f99b25522dd7ed89e8b41d2c5c80dbaf170).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-706676195


   **[Test build #129637 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129637/testReport)** for PR 28885 at commit [`ab2f369`](https://github.com/apache/spark/commit/ab2f3699eb8313c72bcb29a4c4e3d7b142b00c5e).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-863497837






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r452926017



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/util/ReuseMap.scala
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.util
+
+import scala.collection.mutable.Map
+import scala.language.existentials
+
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Map of canonicalized plans that can be used to find reuse possibilities.
+ *
+ * To avoid costly canonicalization of a plan:
+ * - we use its schema first to check if it can be replaced to a reused one at all
+ * - we insert it into the map of canonicalized plans only when at least 2 have the same schema
+ */
+class ReuseMap[T <: QueryPlan[_]] {
+  // scalastyle:off structural.type
+  private val map = Map[StructType, (T, Map[T2 forSome { type T2 >: T }, T])]()

Review comment:
       If the existence type `T2`, which stands for the canonicalized type of `T`, is hard to read then we can move T2 to the class definition like this:
   ```
   class ReuseMap[T <: QueryPlan[_], T2 >: T] {
     private val map = Map[StructType, (T, Map[T2, T])]()
   ```
   and initialize the `ReuseMap`s as:
   ```
         val exchanges = new ReuseMap[Exchange, SparkPlan]()
         val subqueries = new ReuseMap[BaseSubqueryExec, SparkPlan]()
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-705179023






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-704870223


   **[Test build #129500 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129500/testReport)** for PR 28885 at commit [`e0f1ee1`](https://github.com/apache/spark/commit/e0f1ee1a3f132fce5801871fc2e39586aa6ee41d).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r446632088



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/Reuse.scala
##########
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.reuse
+
+import scala.collection.mutable.Map
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
+ * use the same exhange or subquery for all the references.
+ */
+case class WholePlanReuse(conf: SQLConf) extends Rule[SparkPlan] {
+
+  def apply(plan: SparkPlan): SparkPlan = {
+    if (conf.exchangeReuseEnabled || conf.subqueryReuseEnabled) {

Review comment:
       Hmm, I'm not sure why would anyone disable reuse subquery or reuse exchange at all. But those flags do exist and we shouldn't change their meaning that exchange or subquery reuse shouldn't happen when they are disabled.
   IMHO if we introduce this "whole plan reuse" with this new rule, then we should still respect the old flags.
   I also think this new rule can replace the old rules entirely with one whole plan traversal and I don't see any reason why would we keep the old rules. Actually, wouldn't be confusing if we had reuse related code at many places?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-773499553






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-705375055


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34148/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-650665761






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sririshindra commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
sririshindra commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r445859500



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/Reuse.scala
##########
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.reuse
+
+import scala.collection.mutable.Map
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
+ * use the same exhange or subquery for all the references.
+ */
+case class WholePlanReuse(conf: SQLConf) extends Rule[SparkPlan] {
+
+  def apply(plan: SparkPlan): SparkPlan = {
+    if (conf.exchangeReuseEnabled || conf.subqueryReuseEnabled) {

Review comment:
       Shouldn't this be `if (conf.wholePlanReuseEnabled) { `instead of `if (conf.exchangeReuseEnabled || conf.subqueryReuseEnabled) {`. 
   
   We could take the position that if wholePlanReuse is enabled then it means both exchangeReuseEnabled and subqueryReuseEnabled are set to true. So setting wholePlanReuseEnabled to true will override whatever user specified value for exchangeReuseEnabled and subqueryReuseEnabled is. if that is the case then in the current commit even `if (conf.wholePlanReuseEnabled) { ` will be redundant because this piece of code will only be triggered when wholePlanReuseEnabled is set to true.
   
   But if we want to say that the flags exchangeReuseEnabled and subqueryReuseEnabled are independent of the flag wholePlanReuseEnabled, then `if (conf.exchangeReuseEnabled || conf.subqueryReuseEnabled) {` is saying that if one flag is true then the other is true as well i.e if one flag is true then both exchange and subquery are being reused irrespective of the value set by the user for the other. This could be confusing to the user. Here we are basically enabling both exchangeReuse and subqueryReuse as along as wholePlanReuseEnabled is set to true.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-660037934


   **[Test build #126051 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126051/testReport)** for PR 28885 at commit [`100e5c2`](https://github.com/apache/spark/commit/100e5c2319866734facd8de6f86dac92f438402b).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-652578815






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dbaliafroozeh commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
dbaliafroozeh commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r446670286



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/Reuse.scala
##########
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.reuse
+
+import scala.collection.mutable.Map
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
+ * use the same exhange or subquery for all the references.
+ */
+case class WholePlanReuse(conf: SQLConf) extends Rule[SparkPlan] {
+
+  def apply(plan: SparkPlan): SparkPlan = {
+    if (conf.exchangeReuseEnabled || conf.subqueryReuseEnabled) {

Review comment:
       Ok, makes sense, if let's keep the old old behavior and just change the implementation.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-705347605






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-650623623






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HyukjinKwon commented on pull request #28885: [SPARK-29375][SPARK-28940][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-647239990


   cc @maryannxue too FYI


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-691525196






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-652721460






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-773499553


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/39471/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-773455991


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/39471/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-650665761






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r459255589



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/Exchange.scala
##########
@@ -95,46 +89,3 @@ case class ReusedExchangeExec(override val output: Seq[Attribute], child: Exchan
        |""".stripMargin
   }
 }
-
-/**
- * Find out duplicated exchanges in the spark plan, then use the same exchange for all the
- * references.
- */
-case class ReuseExchange(conf: SQLConf) extends Rule[SparkPlan] {
-
-  def apply(plan: SparkPlan): SparkPlan = {
-    if (!conf.exchangeReuseEnabled) {
-      return plan
-    }
-    // Build a hash map using schema of exchanges to avoid O(N*N) sameResult calls.
-    val exchanges = mutable.HashMap[StructType, ArrayBuffer[Exchange]]()
-
-    // Replace a Exchange duplicate with a ReusedExchange
-    def reuse: PartialFunction[Exchange, SparkPlan] = {
-      case exchange: Exchange =>
-        val sameSchema = exchanges.getOrElseUpdate(exchange.schema, ArrayBuffer[Exchange]())
-        val samePlan = sameSchema.find { e =>
-          exchange.sameResult(e)
-        }
-        if (samePlan.isDefined) {
-          // Keep the output of this exchange, the following plans require that to resolve
-          // attributes.
-          ReusedExchangeExec(exchange.output, samePlan.get)
-        } else {
-          sameSchema += exchange
-          exchange
-        }
-    }
-
-    plan transformUp {
-      case exchange: Exchange => reuse(exchange)
-    } transformAllExpressions {
-      // Lookup inside subqueries for duplicate exchanges
-      case in: InSubqueryExec =>
-        val newIn = in.plan.transformUp {
-          case exchange: Exchange => reuse(exchange)
-        }
-        in.copy(plan = newIn.asInstanceOf[BaseSubqueryExec])
-    }
-  }

Review comment:
       This is the issue described in 1. in the PR description and tested with the case `SPARK-32041: No reuse interference inside ReuseExchange` in the new `ReuseExchangeAndSubquerySuite`: https://github.com/apache/spark/pull/28885/files#diff-f6f54d5cfc4254d8ed9122013394351bR28
   
   Combining the 2 rules are required to fix 2. and tested with the case `SPARK-32041: No reuse interference between ReuseExchange and ReuseSubquery`: https://github.com/apache/spark/pull/28885/files#diff-f6f54d5cfc4254d8ed9122013394351bR67




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r466351098



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala
##########
@@ -156,4 +158,46 @@ class ExchangeSuite extends SparkPlanTest with SharedSparkSession {
     val projection2 = cached.select("_1", "_3").queryExecution.executedPlan
     assert(!projection1.sameResult(projection2))
   }
+
+  test("Exchange reuse across the whole plan") {
+    val df = sql(
+      """
+        |SELECT
+        |  (SELECT max(a.key) FROM testData AS a JOIN testData AS b ON b.key = a.key),
+        |  a.key
+        |FROM testData AS a
+        |JOIN testData AS b ON b.key = a.key
+      """.stripMargin)
+
+    val plan = df.queryExecution.executedPlan
+
+    val exchangeIds = plan.collectWithSubqueries { case e: Exchange => e.id }
+    val reusedExchangeIds = plan.collectWithSubqueries {
+      case re: ReusedExchangeExec => re.child.id
+    }
+
+    assert(exchangeIds.size == 2, "Whole plan exchange reusing not working correctly")
+    assert(reusedExchangeIds.size == 3, "Whole plan exchange reusing not working correctly")
+    assert(reusedExchangeIds.forall(exchangeIds.contains(_)),
+      "ReusedExchangeExec should reuse an existing exchange")
+
+    val df2 = sql(

Review comment:
       Some of the rules already require specific ordering (please see batches in `Optimizer`) and we have an other that require to be the first (`InsertAdaptiveSparkPlan` in AQE) so I don't think that `ReuseExchangeAndSubquery` to be the last would be unique. Actually I suspect `ReuseExchange` and `ReuseSubquery` were added as last by design, but it was not documented.
   
   Do you think it is ok to note this requirement of the new rule in its scaladoc in this PR and maybe we can try to find a better mechanism in a follow-up PR?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [WIP][SPARK-29375][SPARK-28940][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r443198465



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
##########
@@ -336,12 +336,13 @@ object QueryPlan extends PredicateHelper {
    * `Attribute`, and replace it with `BoundReference` will cause error.
    */
   def normalizeExpressions[T <: Expression](e: T, input: AttributeSeq): T = {
+    type T2 = QueryPlan[_]
     e.transformUp {
-      case s: PlanExpression[QueryPlan[_] @unchecked] =>
+      case s: PlanExpression[T2 @unchecked] =>
         // Normalize the outer references in the subquery plan.
         val normalizedPlan = s.plan.transformAllExpressions {
           case OuterReference(r) => OuterReference(QueryPlan.normalizeExpressions(r, input))
-        }
+        }.canonicalized.asInstanceOf[T2]

Review comment:
       As we traverse bottom-up manner across the whole plan, we might run into a case when there is a reuse opportunity contains a subquery. If we don't canonicalize that subquery then we miss it.
   
   An example is "join key with multiple references on the filtering plan" test in `DynamicPartitionPruningSuiteBase where without this line of change only a "smaller" subquery reuse happens:
   ```
   == Physical Plan ==
   *(2) Project [id#2729L, a#2730, b#2731]
   +- *(2) BroadcastHashJoin [knownfloatingpointnormalized(normalizenanandzero((cast(b#2731 as double) + cast(a#2730 as double))))], [knownfloatingpointnormalized(normalizenanandzero((cast(y#2734 as double) + cast(z#2735 as double))))], Inner, BuildRight
      :- *(2) ColumnarToRow
      :  +- FileScan parquet default.fact[id#2729L,a#2730,b#2731] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [isnotnull(b#2731), isnotnull(a#2730), dynamicpruningexpression(knownfloatingpointnormalized(norm..., PushedFilters: [], ReadSchema: struct<id:bigint>
      :        +- SubqueryBroadcast dynamicpruning#2742, 0, [knownfloatingpointnormalized(normalizenanandzero((cast(y#2734 as double) + cast(z#2735 as double))))], [id=#274]
      :           +- BroadcastExchange HashedRelationBroadcastMode(List(knownfloatingpointnormalized(normalizenanandzero((cast(input[0, string, true] as double) + cast(input[1, string, true] as double)))))), [id=#273]
      :              +- *(1) Project [y#2734, z#2735]
      :                 +- *(1) Filter (((isnotnull(x#2733) AND (cast(x#2733 as double) = Subquery scalar-subquery#2728, [id=#262])) AND isnotnull(y#2734)) AND isnotnull(z#2735))
      :                    :  +- Subquery scalar-subquery#2728, [id=#262]
      :                    :     +- *(2) HashAggregate(keys=[], functions=[avg(cast(w#2736 as double))])
      :                    :        +- Exchange SinglePartition, true, [id=#258]
      :                    :           +- *(1) HashAggregate(keys=[], functions=[partial_avg(cast(w#2736 as double))])
      :                    :              +- *(1) ColumnarToRow
      :                    :                 +- FileScan parquet default.dim[w#2736] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct<w:string>
      :                    +- *(1) ColumnarToRow
      :                       +- FileScan parquet default.dim[x#2733,y#2734,z#2735] Batched: true, DataFilters: [isnotnull(x#2733), isnotnull(y#2734), isnotnull(z#2735)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [], PushedFilters: [IsNotNull(x), IsNotNull(y), IsNotNull(z)], ReadSchema: struct<x:string,y:string,z:string>
      +- BroadcastExchange HashedRelationBroadcastMode(List(knownfloatingpointnormalized(normalizenanandzero((cast(input[0, string, true] as double) + cast(input[1, string, true] as double)))))), [id=#359]
         +- *(1) Project [y#2734, z#2735]
            +- *(1) Filter (((isnotnull(x#2733) AND (cast(x#2733 as double) = ReusedSubquery Subquery scalar-subquery#2728, [id=#262])) AND isnotnull(y#2734)) AND isnotnull(z#2735))
               :  +- ReusedSubquery Subquery scalar-subquery#2728, [id=#262]
               +- *(1) ColumnarToRow
                  +- FileScan parquet default.dim[x#2733,y#2734,z#2735] Batched: true, DataFilters: [isnotnull(x#2733), isnotnull(y#2734), isnotnull(z#2735)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [], PushedFilters: [IsNotNull(x), IsNotNull(y), IsNotNull(z)], ReadSchema: struct<x:string,y:string,z:string>
   ```
   but with this line of change we get a "bigger" exchange reuse:
   ```
   *(2) Project [id#2729L, a#2730, b#2731]
   +- *(2) BroadcastHashJoin [knownfloatingpointnormalized(normalizenanandzero((cast(b#2731 as double) + cast(a#2730 as double))))], [knownfloatingpointnormalized(normalizenanandzero((cast(y#2734 as double) + cast(z#2735 as double))))], Inner, BuildRight
      :- *(2) ColumnarToRow
      :  +- FileScan parquet default.fact[id#2729L,a#2730,b#2731] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [isnotnull(b#2731), isnotnull(a#2730), dynamicpruningexpression(knownfloatingpointnormalized(norm..., PushedFilters: [], ReadSchema: struct<id:bigint>
      :        +- SubqueryBroadcast dynamicpruning#2742, 0, [knownfloatingpointnormalized(normalizenanandzero((cast(y#2734 as double) + cast(z#2735 as double))))], [id=#274]
      :           +- BroadcastExchange HashedRelationBroadcastMode(List(knownfloatingpointnormalized(normalizenanandzero((cast(input[0, string, true] as double) + cast(input[1, string, true] as double)))))), [id=#273]
      :              +- *(1) Project [y#2734, z#2735]
      :                 +- *(1) Filter (((isnotnull(x#2733) AND (cast(x#2733 as double) = Subquery scalar-subquery#2728, [id=#262])) AND isnotnull(y#2734)) AND isnotnull(z#2735))
      :                    :  +- Subquery scalar-subquery#2728, [id=#262]
      :                    :     +- *(2) HashAggregate(keys=[], functions=[avg(cast(w#2736 as double))])
      :                    :        +- Exchange SinglePartition, true, [id=#258]
      :                    :           +- *(1) HashAggregate(keys=[], functions=[partial_avg(cast(w#2736 as double))])
      :                    :              +- *(1) ColumnarToRow
      :                    :                 +- FileScan parquet default.dim[w#2736] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct<w:string>
      :                    +- *(1) ColumnarToRow
      :                       +- FileScan parquet default.dim[x#2733,y#2734,z#2735] Batched: true, DataFilters: [isnotnull(x#2733), isnotnull(y#2734), isnotnull(z#2735)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [], PushedFilters: [IsNotNull(x), IsNotNull(y), IsNotNull(z)], ReadSchema: struct<x:string,y:string,z:string>
      +- ReusedExchange [y#2734, z#2735], BroadcastExchange HashedRelationBroadcastMode(List(knownfloatingpointnormalized(normalizenanandzero((cast(input[0, string, true] as double) + cast(input[1, string, true] as double)))))), [id=#273]
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-651701192


   > Changes look good to me, added 3 minor comments.
   
   Thanks @dbaliafroozeh.
   
   @cloud-fan, @maryannxue could you review this PR?
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on a change in pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on a change in pull request #28885:
URL: https://github.com/apache/spark/pull/28885#discussion_r460115721



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/util/ReuseMap.scala
##########
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.util
+
+import scala.collection.mutable.{ArrayBuffer, Map}
+
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Map of canonicalized plans that can be used to find reuse possibilities.
+ *
+ * To avoid costly canonicalization of a plan:
+ * - we use its schema first to check if it can be replaced to a reused one at all
+ * - we insert it into the map of canonicalized plans only when at least 2 have the same schema
+ *
+ * @tparam T the type of the node we want to reuse
+ */
+class ReuseMap[T <: T2, T2 <: QueryPlan[T2]] {

Review comment:
       It might look like `T2` is not required, but it is "silently" used at `sameSchema.find(plan.sameResult)` and Scala would complain without it.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-683252192


   **[Test build #128013 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128013/testReport)** for PR 28885 at commit [`cd064e4`](https://github.com/apache/spark/commit/cd064e460f90984d57397bac499058a261bc7205).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-706681607


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34241/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-776230110


   **[Test build #135083 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135083/testReport)** for PR 28885 at commit [`c3cbdb2`](https://github.com/apache/spark/commit/c3cbdb2d6ff9c42b34fa1850645bb248cdc8fe6b).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-663126801






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-707053044


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/34285/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-773617557


   **[Test build #134885 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/134885/testReport)** for PR 28885 at commit [`ec2324a`](https://github.com/apache/spark/commit/ec2324a25c9fb02272d84b29678155e240f5ce8f).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] peter-toth commented on pull request #28885: [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse

Posted by GitBox <gi...@apache.org>.
peter-toth commented on pull request #28885:
URL: https://github.com/apache/spark/pull/28885#issuecomment-864803292


   > thanks, merging to master!
   
   Thanks for the review!
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org