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 2021/04/22 14:08:25 UTC

[GitHub] [spark] peter-toth opened a new pull request #32298: [WIP][SPARK-34079][SQL] Merging non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


   ### What changes were proposed in this pull request?
   This PR:
   - Adds a new subquery type `MultiScalarSubquery` / `MultiScalarSubqueryExec` to compute multiple scalar values at the same time.
   - Adds a new optimizer rule `MergeScalarSubqueries` to merge similar non-correlated scalar subqueries into multi-column scalar subqueries and replaces the original scalar subquery expression to `GetStructField(MultiScalarSubquery(...))`.
   - Lets the `ReuseSubquery` / `ReuseAdaptiveSubquery` rules to replace multiple instances of the same `MultiScalarSubquery` to reuse references to make sure a `MultiScalarSubquery` runs only once.
   
   E.g. the following query:
   ```
   SELECT
     (SELECT avg(a) FROM t GROUP BY b),
     (SELECT sum(b) FROM t GROUP BY b)
   ```
   is optimized from:
   ```
   Project [scalar-subquery#231 [] AS scalarsubquery()#241, scalar-subquery#232 [] AS scalarsubquery()#242L]
   :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
   :  :  +- Relation default.t[a#233,b#234] parquet
   :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
   :     +- Project [b#240]
   :        +- Relation default.t[a#239,b#240] parquet
   +- OneRowRelation
   ```
   to:
   ```
   Project [multi-scalar-subquery#231.avg(a) AS scalarsubquery()#241, multi-scalar-subquery#232.sum(b) AS scalarsubquery()#242L]
   :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
   :  :  +- Project [a#233, b#234]
   :  :     +- Relation default.t[a#233,b#234] parquet
   :  +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
   :     +- Project [a#233, b#234]
   :        +- Relation default.t[a#233,b#234] parquet
   +- OneRowRelation
   ```
   
   ### Why are the changes needed?
   Performance improvement.
   ```
   TPCDS Snappy:                                    Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
   -------------------------------------------------------------------------------------------------------------------------------
   q9 - spark.sql.scalarSubqueyMerge.enabled=false          45892          47172        1220          0.0      Infinity       1.0X
   q9 - spark.sql.scalarSubqueyMerge.enabled=true           16769          16863         124          0.0      Infinity       2.7X
   ```
   
   ### Does this PR introduce _any_ user-facing change?
   No.
   
   ### How was this patch tested?
   Existing UTs. I will add new ones later...
   


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


   **[Test build #138112 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138112/testReport)** for PR 32298 at commit [`e63111d`](https://github.com/apache/spark/commit/e63111d2e26b92a983c5561b90608d80e0b4e7ee).


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


   **[Test build #138842 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138842/testReport)** for PR 32298 at commit [`a3e84a4`](https://github.com/apache/spark/commit/a3e84a4d729bae4b708bce89a19dd355d660262c).


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala
##########
@@ -107,6 +107,53 @@ case class ScalarSubquery(
   }
 }
 
+/**
+ * A subquery that is capable to return multiple scalar values.
+ */
+case class MultiScalarSubqueryExec(

Review comment:
       This is not needed after https://github.com/apache/spark/pull/32298/commits/1f2f75c9c52f7494db4e3bb15368c87d9b8e17c7.




-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LeafNode, LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{MULTI_SCALAR_SUBQUERY, SCALAR_SUBQUERY}
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s into a
+ * [[MultiScalarSubquery]] to compute multiple scalar values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ * - The original [[ScalarSubquery]] expression is replaced to a reference pointing to its cached
+ *   version in this form: `GetStructField(MultiScalarSubquery(SubqueryReference(...)))`.
+ * - A second traversal checks if a [[SubqueryReference]] is pointing to a subquery plan that
+ *   returns multiple values and either replaces only [[SubqueryReference]] to the cached plan or
+ *   restores the whole expression to its original [[ScalarSubquery]] form.
+ * - [[ReuseSubquery]] rule makes sure that merged subqueries are computed once.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *   scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet

Review comment:
       I think that is smart rewrite way but let's check a more complex example: E.g. can we rewrite 
   ```
   SELECT *
   FROM r
   JOIN r2 ON r2.x = r.x
   WHERE r.y = (SELECT sum(b) FROM t) AND r2.y = (SELECT avg(b) FROM t)
   ```
   ? Maybe
   ```
   SELECT *
   FROM (
     SELECT (
       SELECT STRUCT(sum(b) AS sum_b, avg(b) AS avg_b) FROM t) AS st, x, y 
       FROM r
     ) AS r
   )
   JOIN r2 ON r2.x = r.x
   WHERE r.y = r.st.sum_b AND r2.y = r.st.avg_b
   ```
   ? Does this work with outer joins? And isn't this more complex than the reuse way in this PR?
   
   I was also thinking about "whole plan subquery merge" (similar to my "whole plan reuse" suggestion: https://github.com/apache/spark/pull/28885) where subqueries at "different level" could be merged (and reused) as a possible improvement to this PR in the future.
   
   BTW, the `ReuseExchangeAndSubquery` rule you mentioned is suggested in my "whole plan reuse" PR, which got stuck a bit due to lack of reviews. Do you also have a similar rule in production or you just saw my PR? If you have some time, any feedback is appreciated there as well. :)
   
   I didn't check how correlated subqueries could benefit from rewriting the query (this PR focuses on uncorrelated ones), but I think at this point in the optimizer those have been transformed to joins.
   Can you please elaborate on the "reading from tables v.s. reading from an array column" part?




-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


   **[Test build #137860 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137860/testReport)** for PR 32298 at commit [`0cff7b2`](https://github.com/apache/spark/commit/0cff7b20a0e0e10b37bb422d0dd1a13e59e810f8).
    * This patch passes all tests.
    * This patch **does not merge 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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


   **[Test build #140386 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/140386/testReport)** for PR 32298 at commit [`d081885`](https://github.com/apache/spark/commit/d0818854541b4a9dcf3d728cf1152625a54a73df).


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LeafNode, LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{MULTI_SCALAR_SUBQUERY, SCALAR_SUBQUERY}
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s into a
+ * [[MultiScalarSubquery]] to compute multiple scalar values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ * - The original [[ScalarSubquery]] expression is replaced to a reference pointing to its cached
+ *   version in this form: `GetStructField(MultiScalarSubquery(SubqueryReference(...)))`.
+ * - A second traversal checks if a [[SubqueryReference]] is pointing to a subquery plan that
+ *   returns multiple values and either replaces only [[SubqueryReference]] to the cached plan or
+ *   restores the whole expression to its original [[ScalarSubquery]] form.
+ * - [[ReuseSubquery]] rule makes sure that merged subqueries are computed once.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *   scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * Project [multi-scalar-subquery#231.avg(a) AS scalarsubquery()#241,
+ *   multi-scalar-subquery#232.sum(b) AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :  :  +- Project [a#233, b#234]
+ * :  :     +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :     +- Project [a#233, b#234]
+ * :        +- Relation default.t[a#233,b#234] parquet
+ * +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    if (conf.scalarSubqueryMergeEabled && conf.subqueryReuseEnabled) {
+      val mergedSubqueries = ArrayBuffer.empty[LogicalPlan]
+      removeReferences(mergeAndInsertReferences(plan, mergedSubqueries), mergedSubqueries)
+    } else {
+      plan
+    }
+  }
+
+  private def mergeAndInsertReferences(
+      plan: LogicalPlan,
+      mergedSubqueries: ArrayBuffer[LogicalPlan]): LogicalPlan = {
+    plan.transformAllExpressionsWithPruning(_.containsAnyPattern(SCALAR_SUBQUERY), ruleId) {
+      case s: ScalarSubquery if s.children.isEmpty =>
+        val (mergedPlan, ordinal) = mergeAndGetReference(s.plan, mergedSubqueries)
+        GetStructField(MultiScalarSubquery(mergedPlan, s.exprId), ordinal)
+    }
+  }
+
+  case class SubqueryReference(
+      index: Int,
+      mergedSubqueries: ArrayBuffer[LogicalPlan]) extends LeafNode {
+    override def stringArgs: Iterator[Any] = Iterator(index)
+
+    override def output: Seq[Attribute] = mergedSubqueries(index).output
+  }
+
+  private def mergeAndGetReference(
+      plan: LogicalPlan,
+      mergedSubqueries: ArrayBuffer[LogicalPlan]): (SubqueryReference, Int) = {
+    mergedSubqueries.zipWithIndex.collectFirst {
+      Function.unlift { case (s, i) => mergePlans(plan, s).map(_ -> i) }
+    }.map { case ((mergedPlan, outputMap), i) =>
+      mergedSubqueries(i) = mergedPlan
+      SubqueryReference(i, mergedSubqueries) ->
+        mergedPlan.output.indexOf(outputMap(plan.output.head))
+    }.getOrElse {
+      mergedSubqueries += plan
+      SubqueryReference(mergedSubqueries.length - 1, mergedSubqueries) -> 0
+    }
+  }
+
+  private def mergePlans(
+      newPlan: LogicalPlan,
+      existingPlan: LogicalPlan): Option[(LogicalPlan, AttributeMap[Attribute])] = {
+    (newPlan, existingPlan) match {
+      case (np, ep) if np.canonicalized == ep.canonicalized =>
+        Some(ep -> AttributeMap(np.output.zip(ep.output)))
+      case (np: Project, ep: Project) =>
+        mergePlans(np.child, ep.child).map { case (mergedChild, outputMap) =>
+          val newProjectList = replaceAttributes(np.projectList, outputMap)
+          val newOutputMap = createOutputMap(np.projectList, newProjectList)
+          Project(distinctExpressions(ep.projectList ++ newProjectList), mergedChild) ->
+            newOutputMap
+        }
+      case (np, ep: Project) =>
+        mergePlans(np, ep.child).map { case (mergedChild, outputMap) =>
+          Project(distinctExpressions(ep.projectList ++ outputMap.values), mergedChild) -> outputMap
+        }
+      case (np: Project, ep) =>
+        mergePlans(np.child, ep).map { case (mergedChild, outputMap) =>
+          val newProjectList = replaceAttributes(np.projectList, outputMap)
+          val newOutputMap = createOutputMap(np.projectList, newProjectList)
+          Project(distinctExpressions(ep.output ++ newProjectList), mergedChild) -> newOutputMap
+        }
+      case (np: Aggregate, ep: Aggregate) =>
+        mergePlans(np.child, ep.child).flatMap { case (mergedChild, outputMap) =>
+          val newGroupingExpression = replaceAttributes(np.groupingExpressions, outputMap)
+          if (ExpressionSet(newGroupingExpression) == ExpressionSet(ep.groupingExpressions)) {
+            val newAggregateExpressions = replaceAttributes(np.aggregateExpressions, outputMap)
+            val newOutputMap = createOutputMap(np.aggregateExpressions, newAggregateExpressions)

Review comment:
       Sure, added a new test in https://github.com/apache/spark/pull/32298/commits/6134fa9741398438aff70f8657c61175db48ac9d to cover 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] SparkQA commented on pull request #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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] zinking commented on a change in pull request #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,413 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CommonScalarSubqueries, Filter, Join, LogicalPlan, Project, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s to compute multiple scalar
+ * values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ *   During this first traversal each [[ScalarSubquery]] expression is replaced to a
+ *   [[ScalarSubqueryReference]] pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a results of merging 2 or more
+ *   plans, or it is a plan that was seen only once.
+ *   Merged plans in the cache get a "header" that is is basically
+ *   `CreateNamedStructure(name1, attribute1, name2, attribute2, ...)`
+ *   expression in new root [[Project]] node. This expression ensures that the merged plan is a
+ *   valid scalar subquery that returns only one value.
+ * - A second traversal checks if a [[ScalarSubqueryReference]] is pointing to a merged subquery
+ *   plan or not and either keeps the reference or restores the original [[ScalarSubquery]].
+ *   If there are [[ScalarSubqueryReference]] nodes remained a [[CommonScalarSubqueries]] root node
+ *   is added to the plan with the referenced scalar subqueries.
+ * - [[PlanSubqueries]] or [[PlanAdaptiveSubqueries]] rule does the physical planning of scalar
+ *   subqueries including the ones under [[CommonScalarSubqueriesExec]] node and replaces
+ *   each [[ScalarSubqueryReference]] to their referenced physical plan in
+ *   `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() header))` form.
+ *   It is important that references pointing to the same merged subquery are replaced to the same
+ *   planned instance to make sure that each merged subquery runs only once (even without a wrapping
+ *   [[ReuseSubquery]] node).
+ *   Finally, the [[CommonScalarSubqueriesExec]] node is removed from the physical plan.
+ * - The [[ReuseExchangeAndSubquery]] rule wraps the second, third, ... instances of the same
+ *   subquery into a [[ReuseSubquery]] node, but this just a cosmetic change in the plan.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *          scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * CommonScalarSubqueries [scalar-subquery#250 []]
+ * :  +- Project [named_struct(avg(a), avg(a)#236, sum(b), sum(b)#238L) AS mergedValue#249]
+ * :     +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :        +- Project [a#233, b#234]
+ * :           +- Relation default.t[a#233,b#234] parquet
+ * +- Project [scalarsubqueryreference(0, 0, DoubleType, 231) AS scalarsubquery()#241,
+ *             scalarsubqueryreference(0, 1, LongType, 232) AS scalarsubquery()#242L]
+ *    +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    if (conf.subqueryReuseEnabled) {
+      plan match {
+        case Subquery(_: CommonScalarSubqueries, _) => plan
+        case s: Subquery => s.copy(child = extractCommonScalarSubqueries(s.child))
+        case _: CommonScalarSubqueries => plan
+        case _ => extractCommonScalarSubqueries(plan)
+      }
+    } else {
+      plan
+    }
+  }
+
+  private def extractCommonScalarSubqueries(plan: LogicalPlan) = {
+    // Plan of subqueries and a flag is the plan is merged
+    val cache = ListBuffer.empty[(Project, Boolean)]

Review comment:
       is it worthwhile to alias Project to some different type name that hint this is the merged header of scalar query ?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LeafNode, LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{MULTI_SCALAR_SUBQUERY, SCALAR_SUBQUERY}
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s into a
+ * [[MultiScalarSubquery]] to compute multiple scalar values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ * - The original [[ScalarSubquery]] expression is replaced to a reference pointing to its cached
+ *   version in this form: `GetStructField(MultiScalarSubquery(SubqueryReference(...)))`.
+ * - A second traversal checks if a [[SubqueryReference]] is pointing to a subquery plan that
+ *   returns multiple values and either replaces only [[SubqueryReference]] to the cached plan or
+ *   restores the whole expression to its original [[ScalarSubquery]] form.
+ * - [[ReuseSubquery]] rule makes sure that merged subqueries are computed once.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *   scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * Project [multi-scalar-subquery#231.avg(a) AS scalarsubquery()#241,
+ *   multi-scalar-subquery#232.sum(b) AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :  :  +- Project [a#233, b#234]
+ * :  :     +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :     +- Project [a#233, b#234]
+ * :        +- Relation default.t[a#233,b#234] parquet
+ * +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    if (conf.scalarSubqueryMergeEabled && conf.subqueryReuseEnabled) {
+      val mergedSubqueries = ArrayBuffer.empty[LogicalPlan]
+      removeReferences(mergeAndInsertReferences(plan, mergedSubqueries), mergedSubqueries)
+    } else {
+      plan
+    }
+  }
+
+  private def mergeAndInsertReferences(
+      plan: LogicalPlan,
+      mergedSubqueries: ArrayBuffer[LogicalPlan]): LogicalPlan = {
+    plan.transformAllExpressionsWithPruning(_.containsAnyPattern(SCALAR_SUBQUERY), ruleId) {
+      case s: ScalarSubquery if s.children.isEmpty =>
+        val (mergedPlan, ordinal) = mergeAndGetReference(s.plan, mergedSubqueries)
+        GetStructField(MultiScalarSubquery(mergedPlan, s.exprId), ordinal)
+    }
+  }
+
+  case class SubqueryReference(
+      index: Int,
+      mergedSubqueries: ArrayBuffer[LogicalPlan]) extends LeafNode {
+    override def stringArgs: Iterator[Any] = Iterator(index)
+
+    override def output: Seq[Attribute] = mergedSubqueries(index).output
+  }
+
+  private def mergeAndGetReference(
+      plan: LogicalPlan,
+      mergedSubqueries: ArrayBuffer[LogicalPlan]): (SubqueryReference, Int) = {
+    mergedSubqueries.zipWithIndex.collectFirst {
+      Function.unlift { case (s, i) => mergePlans(plan, s).map(_ -> i) }
+    }.map { case ((mergedPlan, outputMap), i) =>
+      mergedSubqueries(i) = mergedPlan
+      SubqueryReference(i, mergedSubqueries) ->
+        mergedPlan.output.indexOf(outputMap(plan.output.head))
+    }.getOrElse {
+      mergedSubqueries += plan
+      SubqueryReference(mergedSubqueries.length - 1, mergedSubqueries) -> 0
+    }
+  }
+
+  private def mergePlans(

Review comment:
       Ok, fixed in https://github.com/apache/spark/pull/32298/commits/17fd6667d127ba32ffb34b41bb58f114a82941e4




-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


   > Since we already have [WithCTE](https://github.com/apache/spark/blob/efe43306fcab18f076f755c81c0406ebc1a5fee9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala#L703-L710) and [CTERelationRef](https://github.com/apache/spark/blob/efe43306fcab18f076f755c81c0406ebc1a5fee9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala#L678-L686), the rewrite looks similar to what you want to achieve, while do not need to add yet-another Logical/Exec node?
   
   `WithCTE` and `CTERelationRef` nodes, that remained in logical plan (because of not inlined CTEs), look to serve only one purpose, that is to handle queries with multiple references to non-deterministic CTEs. That's why they are planned with an extra shuffle exchange in [WithCTEStrategy](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala#L681-L706). That extra exchange is needed for `ReuseExchangeAndSubquery` to kick in and ensure that the CTE is executed only once.
   But I think that an extra shuffle could mean performance degradation in case of scalar subqueries (CTEs returning only one row).


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] sigmod commented on pull request #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


   > It never replaces a Project. I think it is more similar to `WithCTE` in that regard.
   
   Sorry, I misunderstood it. Yes, it's very similar to WithCTE. My thoughts are:
   - since we've already had `WithCTE` and `CTERef`,  we probably don't want to re-invent a similar wheel for a narrower case (i.e., shared evaluation of subqueries). I suspect the additional performance gain is marginal, compared to the gain of shared scan and aggregate computation.
   - after this PR, subquery related rules also may have to think about SubqueryReferences;
   - what would happen if a `WithCTE` co-exists with `CommonSubqueries`;
   - `ColumnPruning` may also need to consider SubqueryReferences and CommonSubqueries`.
   


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] tgravescs commented on a change in pull request #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,422 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CommonScalarSubqueries, Filter, Join, LogicalPlan, Project, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s to compute multiple scalar
+ * values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ *   During this first traversal each [[ScalarSubquery]] expression is replaced to a
+ *   [[ScalarSubqueryReference]] pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a results of merging 2 or more
+ *   plans, or it is a plan that was seen only once.
+ *   Merged plans in the cache get a "header", that is is basically
+ *   `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in new root
+ *   [[Project]] node. This expression ensures that the merged plan is a valid scalar subquery that
+ *   returns only one value.
+ * - A second traversal checks if a [[ScalarSubqueryReference]] is pointing to a merged subquery
+ *   plan or not and either keeps the reference or restores the original [[ScalarSubquery]].
+ *   If there are [[ScalarSubqueryReference]] nodes remained a [[CommonScalarSubqueries]] root node
+ *   is added to the plan with the referenced scalar subqueries.
+ * - [[PlanSubqueries]] or [[PlanAdaptiveSubqueries]] rule does the physical planning of scalar
+ *   subqueries including the ones under [[CommonScalarSubqueriesExec]] node and replaces
+ *   each [[ScalarSubqueryReference]] to their referenced physical plan in
+ *   `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() header))` form.
+ *   It is important that references pointing to the same merged subquery are replaced to the same
+ *   planned instance to make sure that each merged subquery runs only once (even without a wrapping
+ *   [[ReuseSubquery]] node).
+ *   Finally, the [[CommonScalarSubqueriesExec]] node is removed from the physical plan.
+ * - The [[ReuseExchangeAndSubquery]] rule wraps the second, third, ... instances of the same
+ *   subquery into a [[ReuseSubquery]] node, but this just a cosmetic change in the plan.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *          scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * CommonScalarSubqueries [scalar-subquery#250 []]
+ * :  +- Project [named_struct(avg(a), avg(a)#236, sum(b), sum(b)#238L) AS mergedValue#249]
+ * :     +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :        +- Project [a#233, b#234]
+ * :           +- Relation default.t[a#233,b#234] parquet
+ * +- Project [scalarsubqueryreference(0, 0, DoubleType, 231) AS scalarsubquery()#241,
+ *             scalarsubqueryreference(0, 1, LongType, 232) AS scalarsubquery()#242L]
+ *    +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    plan match {
+      case Subquery(_: CommonScalarSubqueries, _) => plan
+      case s: Subquery => s.copy(child = extractCommonScalarSubqueries(s.child))
+      case _: CommonScalarSubqueries => plan
+      case _ => extractCommonScalarSubqueries(plan)
+    }
+  }
+
+  /**
+   * An item in the cache of merged scalar subqueries.
+   *
+   * @param elements  List of attributes that form the scalar return value of a merged subquery
+   * @param plan      The plan of a merged scalar subquery
+   * @param merged    A flag to identify if this item is the result of merging subqueries.
+   *                  Please note that `elements.size == 1` doesn't always mean that the plan is not
+   *                  merged as there can be subqueries that are different ([[checkIdenticalPlans]]
+   *                  is false) due to an extra [[Project]] node in one of them. In that case
+   *                  `elements.size` remains 1 after merging, but the merged flag becomes true.
+   */
+  case class Header(elements: Seq[(String, Attribute)], plan: LogicalPlan, merged: Boolean)
+
+  private def extractCommonScalarSubqueries(plan: LogicalPlan) = {
+    val cache = ListBuffer.empty[Header]
+    val newPlan = removeReferences(insertReferences(plan, cache), cache)
+    if (cache.nonEmpty) {
+      val scalarSubqueries = cache.map {
+        case Header(elements, child, _) => ScalarSubquery(createProject(elements, child))
+      }.toSeq
+      CommonScalarSubqueries(scalarSubqueries, newPlan)
+    } else {
+      newPlan
+    }
+  }
+
+  // First traversal builds up the cache and inserts `ScalarSubqueryReference`s to the plan.
+  private def insertReferences(plan: LogicalPlan, cache: ListBuffer[Header]): LogicalPlan = {
+    plan.transformAllExpressionsWithPruning(_.containsAnyPattern(SCALAR_SUBQUERY)) {
+      case s: ScalarSubquery if s.children.isEmpty =>
+        val (subqueryIndex, headerIndex) = cacheSubquery(s.plan, cache)
+        ScalarSubqueryReference(subqueryIndex, headerIndex, s.dataType, s.exprId)
+    }
+  }
+
+  // Caching returns the index of the subquery in the cache and the index of scalar member in the
+  // "header", that `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in a
+  // [[Project]] node.
+  private def cacheSubquery(plan: LogicalPlan, cache: ListBuffer[Header]): (Int, Int) = {
+    val output = plan.output.head
+    cache.zipWithIndex.collectFirst(Function.unlift { case (header, subqueryIndex) =>

Review comment:
       it does take a second to figure out but I'm ok with it.

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,421 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CommonScalarSubqueries, Filter, Join, LogicalPlan, Project, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s to compute multiple scalar
+ * values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ *   During this first traversal each [[ScalarSubquery]] expression is replaced to a
+ *   [[ScalarSubqueryReference]] pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a results of merging 2 or more
+ *   plans, or it is a plan that was seen only once.
+ *   Merged plans in the cache get a "header", that is is basically
+ *   `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in new root
+ *   [[Project]] node. This expression ensures that the merged plan is a valid scalar subquery that
+ *   returns only one value.
+ * - A second traversal checks if a [[ScalarSubqueryReference]] is pointing to a merged subquery
+ *   plan or not and either keeps the reference or restores the original [[ScalarSubquery]].
+ *   If there are [[ScalarSubqueryReference]] nodes remained a [[CommonScalarSubqueries]] root node
+ *   is added to the plan with the referenced scalar subqueries.
+ * - [[PlanSubqueries]] or [[PlanAdaptiveSubqueries]] rule does the physical planning of scalar
+ *   subqueries including the ones under [[CommonScalarSubqueriesExec]] node and replaces
+ *   each [[ScalarSubqueryReference]] to their referenced physical plan in
+ *   `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() header))` form.
+ *   It is important that references pointing to the same merged subquery are replaced to the same
+ *   planned instance to make sure that each merged subquery runs only once (even without a wrapping
+ *   [[ReuseSubquery]] node).
+ *   Finally, the [[CommonScalarSubqueriesExec]] node is removed from the physical plan.
+ * - The [[ReuseExchangeAndSubquery]] rule wraps the second, third, ... instances of the same
+ *   subquery into a [[ReuseSubquery]] node, but this just a cosmetic change in the plan.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *          scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * CommonScalarSubqueries [scalar-subquery#250 []]
+ * :  +- Project [named_struct(avg(a), avg(a)#236, sum(b), sum(b)#238L) AS mergedValue#249]
+ * :     +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :        +- Project [a#233, b#234]
+ * :           +- Relation default.t[a#233,b#234] parquet
+ * +- Project [scalarsubqueryreference(0, 0, DoubleType, 231) AS scalarsubquery()#241,
+ *             scalarsubqueryreference(0, 1, LongType, 232) AS scalarsubquery()#242L]
+ *    +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    plan match {
+      case Subquery(_: CommonScalarSubqueries, _) => plan
+      case s: Subquery => s.copy(child = extractCommonScalarSubqueries(s.child))
+      case _: CommonScalarSubqueries => plan
+      case _ => extractCommonScalarSubqueries(plan)
+    }
+  }
+
+  /**
+   * An item in the cache of merged scalar subqueries.
+   *
+   * @param elements  List of attributes that form the scalar return value of a merged subquery
+   * @param plan      The plan of a merged scalar subquery
+   * @param merged    A flag to identify if this item is the result of merging subqueries.
+   *                  Please note that `elements.size == 1` doesn't always mean that the plan is not
+   *                  merged as there can be subqueries that are different ([[checkIdenticalPlans]]
+   *                  is false) due to an extra [[Project]] node in one of them. In that case
+   *                  `elements.size` remains 1 after merging, but the merged flag becomes true.
+   */
+  case class Header(elements: Seq[(String, Attribute)], plan: LogicalPlan, merged: Boolean)
+
+  private def extractCommonScalarSubqueries(plan: LogicalPlan) = {
+    val cache = ListBuffer.empty[Header]
+    val newPlan = removeReferences(insertReferences(plan, cache), cache)
+    if (cache.nonEmpty) {
+      val scalarSubqueries = cache.map {
+        case Header(elements, child, _) => ScalarSubquery(createProject(elements, child))
+      }.toSeq
+      CommonScalarSubqueries(scalarSubqueries, newPlan)
+    } else {
+      newPlan
+    }
+  }
+
+  // First traversal builds up the cache and inserts `ScalarSubqueryReference`s to the plan.
+  private def insertReferences(plan: LogicalPlan, cache: ListBuffer[Header]): LogicalPlan = {
+    plan.transformAllExpressionsWithPruning(_.containsAnyPattern(SCALAR_SUBQUERY)) {
+      case s: ScalarSubquery if !s.isCorrelated && s.deterministic =>
+        val (subqueryIndex, headerIndex) = cacheSubquery(s.plan, cache)
+        ScalarSubqueryReference(subqueryIndex, headerIndex, s.dataType, s.exprId)
+    }
+  }
+
+  // Caching returns the index of the subquery in the cache and the index of scalar member in the
+  // "header", that `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in a
+  // [[Project]] node.
+  private def cacheSubquery(plan: LogicalPlan, cache: ListBuffer[Header]): (Int, Int) = {
+    val output = plan.output.head
+    cache.zipWithIndex.collectFirst(Function.unlift { case (header, subqueryIndex) =>
+      checkIdenticalPlans(plan, header.plan).map { outputMap =>
+        val mappedOutput = mapAttributes(output, outputMap)
+        val headerIndex = header.elements.indexWhere {
+          case (_, attribute) => attribute.exprId == mappedOutput.exprId
+        }
+        subqueryIndex -> headerIndex
+      }.orElse(tryMergePlans(plan, header.plan).map {
+        case (mergedPlan, outputMap) =>
+          val mappedOutput = mapAttributes(output, outputMap)
+          var headerIndex = header.elements.indexWhere {
+            case (_, attribute) => attribute.exprId == mappedOutput.exprId
+          }
+          val newHeaderElements = if (headerIndex == -1) {
+            headerIndex = header.elements.size
+            header.elements :+ (output.name -> mappedOutput)
+          } else {
+            header.elements
+          }
+          cache(subqueryIndex) = Header(newHeaderElements, mergedPlan, true)
+          subqueryIndex -> headerIndex
+      })
+    }).getOrElse {
+      cache += Header(Seq(output.name -> output), plan, false)
+      cache.length - 1 -> 0
+    }
+  }
+
+  // If 2 plans are identical return the attribute mapping from the new to the cached version.
+  private def checkIdenticalPlans(
+      newPlan: LogicalPlan,
+      cachedPlan: LogicalPlan): Option[AttributeMap[Attribute]] = {
+    if (newPlan.canonicalized == cachedPlan.canonicalized) {
+      Some(AttributeMap(newPlan.output.zip(cachedPlan.output)))
+    } else {
+      None
+    }
+  }
+
+  // Recursively traverse down and try merging 2 plans. If merge is possible then return the merged
+  // plan with the attribute mapping from the new to the merged version.
+  // Please note that merging arbitrary plans can be complicated, the current version supports only
+  // some of the most important nodes.
+  private def tryMergePlans(
+      newPlan: LogicalPlan,
+      cachedPlan: LogicalPlan): Option[(LogicalPlan, AttributeMap[Attribute])] = {
+    checkIdenticalPlans(newPlan, cachedPlan).map(cachedPlan -> _).orElse(
+      (newPlan, cachedPlan) match {
+        case (np: Project, cp: Project) =>
+          tryMergePlans(np.child, cp.child).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.projectList, outputMap, cp.projectList)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np, cp: Project) =>
+          tryMergePlans(np, cp.child).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.output, outputMap, cp.projectList)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np: Project, cp) =>
+          tryMergePlans(np.child, cp).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.projectList, outputMap, cp.output)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np: Aggregate, cp: Aggregate) if supportedAggregateMerge(np, cp) =>
+          tryMergePlans(np.child, cp.child).flatMap { case (mergedChild, outputMap) =>
+            val mappedNewGroupingExpression =
+              np.groupingExpressions.map(mapAttributes(_, outputMap))
+            // Order of grouping expression doesn't matter so we can compare sets
+            if (ExpressionSet(mappedNewGroupingExpression) ==
+              ExpressionSet(cp.groupingExpressions)) {
+              val (mergedAggregateExpressions, newOutputMap) =
+                mergeNamedExpressions(np.aggregateExpressions, outputMap, cp.aggregateExpressions)
+              val mergedPlan =
+                Aggregate(cp.groupingExpressions, mergedAggregateExpressions, mergedChild)
+              Some(mergedPlan -> newOutputMap)
+            } else {
+              None
+            }
+          }
+
+        // Merging general nodes is complicated and this implementation supports only those nodes in
+        // which the order and the number of output attributes are not relevant (see
+        // `supportedMerge()` whitelist).
+        // Also, this implementation supports only those nodes in which children can be merged in
+        // the same order.
+        case (np, cp) if supportedMerge(np) && np.getClass == cp.getClass &&
+            np.children.size == cp.children.size &&
+            np.expressions.size == cp.expressions.size &&
+            // Fields that don't contain any children or expressions should match
+            np.productIterator.filterNot(np.children.contains)
+              .filter(QueryPlan.extractExpressions(_).isEmpty).toSeq ==
+              cp.productIterator.filterNot(cp.children.contains)
+                .filter(QueryPlan.extractExpressions(_).isEmpty).toSeq =>
+          val merged = np.children.zip(cp.children).map {
+            case (npChild, cpChild) => tryMergePlans(npChild, cpChild)
+          }
+          if (merged.forall(_.isDefined)) {
+            val (mergedChildren, outputMaps) = merged.map(_.get).unzip
+            val outputMap = AttributeMap(outputMaps.map(_.iterator).reduce(_ ++ _).toSeq)
+            // We know that fields that don't contain any children or expressions do match and
+            // children can be merged so we need to test expressions only
+            if (np.expressions.map(mapAttributes(_, outputMap).canonicalized) ==
+              cp.expressions.map(_.canonicalized)) {
+              val mergedPlan = cp.withNewChildren(mergedChildren)
+              Some(mergedPlan -> outputMap)
+            } else {
+              None
+            }
+          } else {
+            None
+          }
+
+        // As a follow-up, it would be possible to merge `CommonScalarSubqueries` nodes, which would
+        // allow merging subqueries with mergee subqueries.
+        // E.g. this query:
+        //
+        // SELECT
+        //   (
+        //     SELECT
+        //       (SELECT avg(a) FROM t GROUP BY b) +
+        //       (SELECT sum(b) FROM t GROUP BY b)
+        //   ),
+        //   (
+        //     SELECT
+        //       (SELECT max(a) FROM t GROUP BY b) +
+        //       (SELECT min(b) FROM t GROUP BY b)
+        //   )
+        //
+        // is currently optimized to:
+        //
+        // == Optimized Logical Plan ==
+        // Project [scalar-subquery#233 [] AS scalarsubquery()#255,
+        //          scalar-subquery#236 [] AS scalarsubquery()#256]
+        // :  :- CommonScalarSubqueries [scalar-subquery#264 []]
+        // :  :  :  +- Aggregate [b#238], [named_struct(avg(a), avg(a#237), sum(b), sum(b#238))
+        //                                 AS mergedValue#263]
+        // :  :  :     +- Relation default.t[a#237,b#238] parquet
+        // :  :  +- Project [(scalarsubqueryreference(0, 0, DoubleType, 231) +
+        //                   cast(scalarsubqueryreference(0, 1, LongType, 232) as double))
+        //                   AS (scalarsubquery() + scalarsubquery())#245]
+        // :  :     +- OneRowRelation
+        // :  +- CommonScalarSubqueries [scalar-subquery#269 []]
+        // :     :  +- Aggregate [b#254], [named_struct(min(a), min(a#253), max(b), max(b#254))
+        //                                 AS mergedValue#268]
+        // :     :     +- Relation default.t[a#253,b#254] parquet
+        // :     +- Project [(scalarsubqueryreference(0, 0, IntegerType, 234) +
+        //                   scalarsubqueryreference(0, 1, IntegerType, 235))
+        //                   AS (scalarsubquery() + scalarsubquery())#252]
+        // :        +- OneRowRelation
+        // +- OneRowRelation
+        //
+        // but if we implemented merging `CommonScalarSubqueries` nodes then the plan could be
+        // transformed further and all leaf subqueries could be merged.
+
+        // Otherwise merging is not possible.
+        case _ => None
+      })
+  }
+
+  private def createProject(elements: Seq[(String, Attribute)], plan: LogicalPlan): Project = {
+    Project(
+      Seq(Alias(
+        CreateNamedStruct(elements.flatMap {
+          case (name, attribute) => Seq(Literal(name), attribute)
+        }),
+        "mergedValue")()),
+      plan)
+  }
+
+  private def mapAttributes[T <: Expression](expr: T, outputMap: AttributeMap[Attribute]) = {
+    expr.transform {
+      case a: Attribute => outputMap.getOrElse(a, a)
+    }.asInstanceOf[T]
+  }
+
+  // Applies `outputMap` attribute mapping on elements of `newExpressions` and merges them into
+  // `cachedExpressions`. Returns the merged expressions and the attribute mapping from the new to
+  // the merged version that can be propagated up during merging nodes.
+  private def mergeNamedExpressions(
+      newExpressions: Seq[NamedExpression],
+      outputMap: AttributeMap[Attribute],
+      cachedExpressions: Seq[NamedExpression]) = {
+    val mergedExpressions = ListBuffer[NamedExpression](cachedExpressions: _*)
+    val newOutputMap = AttributeMap(newExpressions.map { ne =>
+      val mapped = mapAttributes(ne, outputMap)
+      val withoutAlias = mapped match {
+        case Alias(child, _) => child
+        case e => e
+      }
+      ne.toAttribute -> mergedExpressions.find {
+        case Alias(child, _) => child semanticEquals withoutAlias
+        case e => e semanticEquals withoutAlias
+      }.getOrElse {
+        mergedExpressions += mapped
+        mapped
+      }.toAttribute
+    })
+    (mergedExpressions.toSeq, newOutputMap)
+  }
+
+  // Merging different aggregate implementations could cause performance regression

Review comment:
       nit - perhaps enhance by just saying "Only allow aggregates of the same implementation because merging different...

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,421 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CommonScalarSubqueries, Filter, Join, LogicalPlan, Project, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s to compute multiple scalar
+ * values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ *   During this first traversal each [[ScalarSubquery]] expression is replaced to a
+ *   [[ScalarSubqueryReference]] pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a results of merging 2 or more
+ *   plans, or it is a plan that was seen only once.
+ *   Merged plans in the cache get a "header", that is is basically
+ *   `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in new root
+ *   [[Project]] node. This expression ensures that the merged plan is a valid scalar subquery that
+ *   returns only one value.
+ * - A second traversal checks if a [[ScalarSubqueryReference]] is pointing to a merged subquery
+ *   plan or not and either keeps the reference or restores the original [[ScalarSubquery]].
+ *   If there are [[ScalarSubqueryReference]] nodes remained a [[CommonScalarSubqueries]] root node
+ *   is added to the plan with the referenced scalar subqueries.
+ * - [[PlanSubqueries]] or [[PlanAdaptiveSubqueries]] rule does the physical planning of scalar
+ *   subqueries including the ones under [[CommonScalarSubqueriesExec]] node and replaces
+ *   each [[ScalarSubqueryReference]] to their referenced physical plan in
+ *   `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() header))` form.
+ *   It is important that references pointing to the same merged subquery are replaced to the same
+ *   planned instance to make sure that each merged subquery runs only once (even without a wrapping
+ *   [[ReuseSubquery]] node).
+ *   Finally, the [[CommonScalarSubqueriesExec]] node is removed from the physical plan.
+ * - The [[ReuseExchangeAndSubquery]] rule wraps the second, third, ... instances of the same
+ *   subquery into a [[ReuseSubquery]] node, but this just a cosmetic change in the plan.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *          scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * CommonScalarSubqueries [scalar-subquery#250 []]
+ * :  +- Project [named_struct(avg(a), avg(a)#236, sum(b), sum(b)#238L) AS mergedValue#249]
+ * :     +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :        +- Project [a#233, b#234]
+ * :           +- Relation default.t[a#233,b#234] parquet
+ * +- Project [scalarsubqueryreference(0, 0, DoubleType, 231) AS scalarsubquery()#241,
+ *             scalarsubqueryreference(0, 1, LongType, 232) AS scalarsubquery()#242L]
+ *    +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    plan match {
+      case Subquery(_: CommonScalarSubqueries, _) => plan
+      case s: Subquery => s.copy(child = extractCommonScalarSubqueries(s.child))
+      case _: CommonScalarSubqueries => plan
+      case _ => extractCommonScalarSubqueries(plan)
+    }
+  }
+
+  /**
+   * An item in the cache of merged scalar subqueries.
+   *
+   * @param elements  List of attributes that form the scalar return value of a merged subquery
+   * @param plan      The plan of a merged scalar subquery
+   * @param merged    A flag to identify if this item is the result of merging subqueries.
+   *                  Please note that `elements.size == 1` doesn't always mean that the plan is not
+   *                  merged as there can be subqueries that are different ([[checkIdenticalPlans]]
+   *                  is false) due to an extra [[Project]] node in one of them. In that case
+   *                  `elements.size` remains 1 after merging, but the merged flag becomes true.
+   */
+  case class Header(elements: Seq[(String, Attribute)], plan: LogicalPlan, merged: Boolean)
+
+  private def extractCommonScalarSubqueries(plan: LogicalPlan) = {
+    val cache = ListBuffer.empty[Header]
+    val newPlan = removeReferences(insertReferences(plan, cache), cache)
+    if (cache.nonEmpty) {
+      val scalarSubqueries = cache.map {
+        case Header(elements, child, _) => ScalarSubquery(createProject(elements, child))
+      }.toSeq
+      CommonScalarSubqueries(scalarSubqueries, newPlan)
+    } else {
+      newPlan
+    }
+  }
+
+  // First traversal builds up the cache and inserts `ScalarSubqueryReference`s to the plan.
+  private def insertReferences(plan: LogicalPlan, cache: ListBuffer[Header]): LogicalPlan = {
+    plan.transformAllExpressionsWithPruning(_.containsAnyPattern(SCALAR_SUBQUERY)) {
+      case s: ScalarSubquery if !s.isCorrelated && s.deterministic =>
+        val (subqueryIndex, headerIndex) = cacheSubquery(s.plan, cache)
+        ScalarSubqueryReference(subqueryIndex, headerIndex, s.dataType, s.exprId)
+    }
+  }
+
+  // Caching returns the index of the subquery in the cache and the index of scalar member in the
+  // "header", that `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in a
+  // [[Project]] node.
+  private def cacheSubquery(plan: LogicalPlan, cache: ListBuffer[Header]): (Int, Int) = {
+    val output = plan.output.head
+    cache.zipWithIndex.collectFirst(Function.unlift { case (header, subqueryIndex) =>
+      checkIdenticalPlans(plan, header.plan).map { outputMap =>
+        val mappedOutput = mapAttributes(output, outputMap)
+        val headerIndex = header.elements.indexWhere {
+          case (_, attribute) => attribute.exprId == mappedOutput.exprId
+        }
+        subqueryIndex -> headerIndex
+      }.orElse(tryMergePlans(plan, header.plan).map {
+        case (mergedPlan, outputMap) =>
+          val mappedOutput = mapAttributes(output, outputMap)
+          var headerIndex = header.elements.indexWhere {
+            case (_, attribute) => attribute.exprId == mappedOutput.exprId
+          }
+          val newHeaderElements = if (headerIndex == -1) {
+            headerIndex = header.elements.size
+            header.elements :+ (output.name -> mappedOutput)
+          } else {
+            header.elements
+          }
+          cache(subqueryIndex) = Header(newHeaderElements, mergedPlan, true)
+          subqueryIndex -> headerIndex
+      })
+    }).getOrElse {
+      cache += Header(Seq(output.name -> output), plan, false)
+      cache.length - 1 -> 0
+    }
+  }
+
+  // If 2 plans are identical return the attribute mapping from the new to the cached version.
+  private def checkIdenticalPlans(
+      newPlan: LogicalPlan,
+      cachedPlan: LogicalPlan): Option[AttributeMap[Attribute]] = {
+    if (newPlan.canonicalized == cachedPlan.canonicalized) {
+      Some(AttributeMap(newPlan.output.zip(cachedPlan.output)))
+    } else {
+      None
+    }
+  }
+
+  // Recursively traverse down and try merging 2 plans. If merge is possible then return the merged
+  // plan with the attribute mapping from the new to the merged version.
+  // Please note that merging arbitrary plans can be complicated, the current version supports only
+  // some of the most important nodes.
+  private def tryMergePlans(
+      newPlan: LogicalPlan,
+      cachedPlan: LogicalPlan): Option[(LogicalPlan, AttributeMap[Attribute])] = {
+    checkIdenticalPlans(newPlan, cachedPlan).map(cachedPlan -> _).orElse(
+      (newPlan, cachedPlan) match {
+        case (np: Project, cp: Project) =>
+          tryMergePlans(np.child, cp.child).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.projectList, outputMap, cp.projectList)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np, cp: Project) =>
+          tryMergePlans(np, cp.child).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.output, outputMap, cp.projectList)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np: Project, cp) =>
+          tryMergePlans(np.child, cp).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.projectList, outputMap, cp.output)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np: Aggregate, cp: Aggregate) if supportedAggregateMerge(np, cp) =>
+          tryMergePlans(np.child, cp.child).flatMap { case (mergedChild, outputMap) =>
+            val mappedNewGroupingExpression =
+              np.groupingExpressions.map(mapAttributes(_, outputMap))
+            // Order of grouping expression doesn't matter so we can compare sets
+            if (ExpressionSet(mappedNewGroupingExpression) ==
+              ExpressionSet(cp.groupingExpressions)) {
+              val (mergedAggregateExpressions, newOutputMap) =
+                mergeNamedExpressions(np.aggregateExpressions, outputMap, cp.aggregateExpressions)
+              val mergedPlan =
+                Aggregate(cp.groupingExpressions, mergedAggregateExpressions, mergedChild)
+              Some(mergedPlan -> newOutputMap)
+            } else {
+              None
+            }
+          }
+
+        // Merging general nodes is complicated and this implementation supports only those nodes in
+        // which the order and the number of output attributes are not relevant (see
+        // `supportedMerge()` whitelist).
+        // Also, this implementation supports only those nodes in which children can be merged in
+        // the same order.
+        case (np, cp) if supportedMerge(np) && np.getClass == cp.getClass &&
+            np.children.size == cp.children.size &&
+            np.expressions.size == cp.expressions.size &&
+            // Fields that don't contain any children or expressions should match
+            np.productIterator.filterNot(np.children.contains)
+              .filter(QueryPlan.extractExpressions(_).isEmpty).toSeq ==
+              cp.productIterator.filterNot(cp.children.contains)
+                .filter(QueryPlan.extractExpressions(_).isEmpty).toSeq =>
+          val merged = np.children.zip(cp.children).map {
+            case (npChild, cpChild) => tryMergePlans(npChild, cpChild)
+          }
+          if (merged.forall(_.isDefined)) {
+            val (mergedChildren, outputMaps) = merged.map(_.get).unzip
+            val outputMap = AttributeMap(outputMaps.map(_.iterator).reduce(_ ++ _).toSeq)
+            // We know that fields that don't contain any children or expressions do match and
+            // children can be merged so we need to test expressions only
+            if (np.expressions.map(mapAttributes(_, outputMap).canonicalized) ==
+              cp.expressions.map(_.canonicalized)) {
+              val mergedPlan = cp.withNewChildren(mergedChildren)
+              Some(mergedPlan -> outputMap)
+            } else {
+              None
+            }
+          } else {
+            None
+          }
+
+        // As a follow-up, it would be possible to merge `CommonScalarSubqueries` nodes, which would
+        // allow merging subqueries with mergee subqueries.

Review comment:
       nit "mergee" mispelled




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


   There are a few relevant UT failures, I will look into them...


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


   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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LeafNode, LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{MULTI_SCALAR_SUBQUERY, SCALAR_SUBQUERY}
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s into a
+ * [[MultiScalarSubquery]] to compute multiple scalar values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ * - The original [[ScalarSubquery]] expression is replaced to a reference pointing to its cached
+ *   version in this form: `GetStructField(MultiScalarSubquery(SubqueryReference(...)))`.
+ * - A second traversal checks if a [[SubqueryReference]] is pointing to a subquery plan that
+ *   returns multiple values and either replaces only [[SubqueryReference]] to the cached plan or
+ *   restores the whole expression to its original [[ScalarSubquery]] form.
+ * - [[ReuseSubquery]] rule makes sure that merged subqueries are computed once.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *   scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * Project [multi-scalar-subquery#231.avg(a) AS scalarsubquery()#241,
+ *   multi-scalar-subquery#232.sum(b) AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :  :  +- Project [a#233, b#234]
+ * :  :     +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :     +- Project [a#233, b#234]
+ * :        +- Relation default.t[a#233,b#234] parquet
+ * +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    if (conf.scalarSubqueryMergeEabled && conf.subqueryReuseEnabled) {
+      val mergedSubqueries = ArrayBuffer.empty[LogicalPlan]
+      removeReferences(mergeAndInsertReferences(plan, mergedSubqueries), mergedSubqueries)
+    } else {
+      plan
+    }
+  }
+
+  private def mergeAndInsertReferences(
+      plan: LogicalPlan,
+      mergedSubqueries: ArrayBuffer[LogicalPlan]): LogicalPlan = {
+    plan.transformAllExpressionsWithPruning(_.containsAnyPattern(SCALAR_SUBQUERY), ruleId) {
+      case s: ScalarSubquery if s.children.isEmpty =>
+        val (mergedPlan, ordinal) = mergeAndGetReference(s.plan, mergedSubqueries)
+        GetStructField(MultiScalarSubquery(mergedPlan, s.exprId), ordinal)
+    }
+  }
+
+  case class SubqueryReference(
+      index: Int,
+      mergedSubqueries: ArrayBuffer[LogicalPlan]) extends LeafNode {
+    override def stringArgs: Iterator[Any] = Iterator(index)
+
+    override def output: Seq[Attribute] = mergedSubqueries(index).output
+  }
+
+  private def mergeAndGetReference(
+      plan: LogicalPlan,
+      mergedSubqueries: ArrayBuffer[LogicalPlan]): (SubqueryReference, Int) = {
+    mergedSubqueries.zipWithIndex.collectFirst {
+      Function.unlift { case (s, i) => mergePlans(plan, s).map(_ -> i) }
+    }.map { case ((mergedPlan, outputMap), i) =>
+      mergedSubqueries(i) = mergedPlan
+      SubqueryReference(i, mergedSubqueries) ->
+        mergedPlan.output.indexOf(outputMap(plan.output.head))
+    }.getOrElse {
+      mergedSubqueries += plan
+      SubqueryReference(mergedSubqueries.length - 1, mergedSubqueries) -> 0
+    }
+  }
+
+  private def mergePlans(
+      newPlan: LogicalPlan,
+      existingPlan: LogicalPlan): Option[(LogicalPlan, AttributeMap[Attribute])] = {
+    (newPlan, existingPlan) match {
+      case (np, ep) if np.canonicalized == ep.canonicalized =>
+        Some(ep -> AttributeMap(np.output.zip(ep.output)))
+      case (np: Project, ep: Project) =>
+        mergePlans(np.child, ep.child).map { case (mergedChild, outputMap) =>
+          val newProjectList = replaceAttributes(np.projectList, outputMap)
+          val newOutputMap = createOutputMap(np.projectList, newProjectList)
+          Project(distinctExpressions(ep.projectList ++ newProjectList), mergedChild) ->
+            newOutputMap
+        }
+      case (np, ep: Project) =>
+        mergePlans(np, ep.child).map { case (mergedChild, outputMap) =>
+          Project(distinctExpressions(ep.projectList ++ outputMap.values), mergedChild) -> outputMap
+        }
+      case (np: Project, ep) =>
+        mergePlans(np.child, ep).map { case (mergedChild, outputMap) =>
+          val newProjectList = replaceAttributes(np.projectList, outputMap)
+          val newOutputMap = createOutputMap(np.projectList, newProjectList)
+          Project(distinctExpressions(ep.output ++ newProjectList), mergedChild) -> newOutputMap
+        }
+      case (np: Aggregate, ep: Aggregate) =>

Review comment:
       > We can always assume merging two (or more) aggregates makes performance better? For example, we have two aggregates in a plan, one side is a hash-aggregate and the other side is an object hash-aggregate. In this case, the merged plan node seems to be an object-hash aggregate. If this is true, this rewrite can easily cause high memory pressure.
   
   Thanks, this is a very good question, let me look into this...
   
   > IMHO, since this rewrite itself is not an optimization, but a pre-process to reuse sub-queries, so it might be better to implement this logic inside the ReuseSubquery side, and merge them if physical plans are the same.
   
   The reason why I implemented this feature as an `Optimizer` rule is that merging `LogicalPlans` seems much easier than merging physical ones. The example in the description has the following physical subquery plans:
   ``` 
   *(1) Project [Subquery scalar-subquery#231, [id=#110] AS scalarsubquery()#241, Subquery scalar-subquery#232, [id=#132] AS scalarsubquery()#242L]
   :  :- Subquery scalar-subquery#231, [id=#110]
   :  :  +- *(2) HashAggregate(keys=[b#234], functions=[avg(a#233)], output=[avg(a)#236])
   :  :     +- Exchange hashpartitioning(b#234, 5), ENSURE_REQUIREMENTS, [id=#106]
   :  :        +- *(1) HashAggregate(keys=[b#234], functions=[partial_avg(a#233)], output=[b#234, sum#247, count#248L])
   :  :           +- *(1) ColumnarToRow
   :  :              +- FileScan parquet default.t[a#233,b#234] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex(1 paths)[file:/Users/petertoth/git/apache/spark/spark-warehouse/org.apache.spar..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct<a:int,b:int>
   :  +- Subquery scalar-subquery#232, [id=#132]
   :     +- *(2) HashAggregate(keys=[b#240], functions=[sum(b#240)], output=[sum(b)#238L])
   :        +- Exchange hashpartitioning(b#240, 5), ENSURE_REQUIREMENTS, [id=#128]
   :           +- *(1) HashAggregate(keys=[b#240], functions=[partial_sum(b#240)], output=[b#240, sum#250L])
   :              +- *(1) ColumnarToRow
   :                 +- FileScan parquet default.t[b#240] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex(1 paths)[file:/Users/petertoth/git/apache/spark/spark-warehouse/org.apache.spar..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct<b:int>
   ```
   Merging these 2 physical subqueries would require much more complex `mergePlans()` function that can handle `Exchange` and `Scan` nodes.
   
    




-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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






-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


   **[Test build #139069 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/139069/testReport)** for PR 32298 at commit [`0f61394`](https://github.com/apache/spark/commit/0f613948dc6fd09e0ef81846ea5501c147bb4a5f).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `case class CommonScalarSubqueries(scalarSubqueries: Seq[ScalarSubquery], child: LogicalPlan)`
     * `case class CommonScalarSubqueriesExec(scalarSubqueries: Seq[Expression], child: 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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


   **[Test build #139121 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/139121/testReport)** for PR 32298 at commit [`100cb9c`](https://github.com/apache/spark/commit/100cb9c9cd8cbf79bcd34c563d02d381cc93dec8).


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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






-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


   **[Test build #140073 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/140073/testReport)** for PR 32298 at commit [`41c0f0a`](https://github.com/apache/spark/commit/41c0f0aa9a1ab0b19a538e20b586807e39059eec).
    * This patch **fails Spark unit tests**.
    * This patch **does not merge 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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


   **[Test build #139058 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/139058/testReport)** for PR 32298 at commit [`3cb1f38`](https://github.com/apache/spark/commit/3cb1f38c123b76db539d13c0b94164d51318c8af).


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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






-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LeafNode, LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{MULTI_SCALAR_SUBQUERY, SCALAR_SUBQUERY}
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s into a
+ * [[MultiScalarSubquery]] to compute multiple scalar values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ * - The original [[ScalarSubquery]] expression is replaced to a reference pointing to its cached
+ *   version in this form: `GetStructField(MultiScalarSubquery(SubqueryReference(...)))`.
+ * - A second traversal checks if a [[SubqueryReference]] is pointing to a subquery plan that
+ *   returns multiple values and either replaces only [[SubqueryReference]] to the cached plan or
+ *   restores the whole expression to its original [[ScalarSubquery]] form.
+ * - [[ReuseSubquery]] rule makes sure that merged subqueries are computed once.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *   scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * Project [multi-scalar-subquery#231.avg(a) AS scalarsubquery()#241,
+ *   multi-scalar-subquery#232.sum(b) AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :  :  +- Project [a#233, b#234]
+ * :  :     +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :     +- Project [a#233, b#234]
+ * :        +- Relation default.t[a#233,b#234] parquet
+ * +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    if (conf.scalarSubqueryMergeEabled && conf.subqueryReuseEnabled) {
+      val mergedSubqueries = ArrayBuffer.empty[LogicalPlan]
+      removeReferences(mergeAndInsertReferences(plan, mergedSubqueries), mergedSubqueries)
+    } else {
+      plan
+    }
+  }
+
+  private def mergeAndInsertReferences(
+      plan: LogicalPlan,
+      mergedSubqueries: ArrayBuffer[LogicalPlan]): LogicalPlan = {
+    plan.transformAllExpressionsWithPruning(_.containsAnyPattern(SCALAR_SUBQUERY), ruleId) {
+      case s: ScalarSubquery if s.children.isEmpty =>
+        val (mergedPlan, ordinal) = mergeAndGetReference(s.plan, mergedSubqueries)
+        GetStructField(MultiScalarSubquery(mergedPlan, s.exprId), ordinal)
+    }
+  }
+
+  case class SubqueryReference(
+      index: Int,
+      mergedSubqueries: ArrayBuffer[LogicalPlan]) extends LeafNode {
+    override def stringArgs: Iterator[Any] = Iterator(index)
+
+    override def output: Seq[Attribute] = mergedSubqueries(index).output
+  }
+
+  private def mergeAndGetReference(
+      plan: LogicalPlan,
+      mergedSubqueries: ArrayBuffer[LogicalPlan]): (SubqueryReference, Int) = {
+    mergedSubqueries.zipWithIndex.collectFirst {
+      Function.unlift { case (s, i) => mergePlans(plan, s).map(_ -> i) }
+    }.map { case ((mergedPlan, outputMap), i) =>
+      mergedSubqueries(i) = mergedPlan
+      SubqueryReference(i, mergedSubqueries) ->
+        mergedPlan.output.indexOf(outputMap(plan.output.head))
+    }.getOrElse {
+      mergedSubqueries += plan
+      SubqueryReference(mergedSubqueries.length - 1, mergedSubqueries) -> 0
+    }
+  }
+
+  private def mergePlans(
+      newPlan: LogicalPlan,
+      existingPlan: LogicalPlan): Option[(LogicalPlan, AttributeMap[Attribute])] = {
+    (newPlan, existingPlan) match {
+      case (np, ep) if np.canonicalized == ep.canonicalized =>
+        Some(ep -> AttributeMap(np.output.zip(ep.output)))
+      case (np: Project, ep: Project) =>
+        mergePlans(np.child, ep.child).map { case (mergedChild, outputMap) =>
+          val newProjectList = replaceAttributes(np.projectList, outputMap)
+          val newOutputMap = createOutputMap(np.projectList, newProjectList)
+          Project(distinctExpressions(ep.projectList ++ newProjectList), mergedChild) ->
+            newOutputMap
+        }
+      case (np, ep: Project) =>
+        mergePlans(np, ep.child).map { case (mergedChild, outputMap) =>
+          Project(distinctExpressions(ep.projectList ++ outputMap.values), mergedChild) -> outputMap
+        }
+      case (np: Project, ep) =>
+        mergePlans(np.child, ep).map { case (mergedChild, outputMap) =>
+          val newProjectList = replaceAttributes(np.projectList, outputMap)
+          val newOutputMap = createOutputMap(np.projectList, newProjectList)
+          Project(distinctExpressions(ep.output ++ newProjectList), mergedChild) -> newOutputMap
+        }
+      case (np: Aggregate, ep: Aggregate) =>

Review comment:
       Fixed the hash/objecthash/sorted aggregate merge issue in https://github.com/apache/spark/pull/32298/commits/282834590a4ca8ccae1f96fd2d107c07af6f14ba and I also added a test.




-- 
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] sigmod commented on a change in pull request #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LeafNode, LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{MULTI_SCALAR_SUBQUERY, SCALAR_SUBQUERY}
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s into a
+ * [[MultiScalarSubquery]] to compute multiple scalar values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ * - The original [[ScalarSubquery]] expression is replaced to a reference pointing to its cached
+ *   version in this form: `GetStructField(MultiScalarSubquery(SubqueryReference(...)))`.
+ * - A second traversal checks if a [[SubqueryReference]] is pointing to a subquery plan that
+ *   returns multiple values and either replaces only [[SubqueryReference]] to the cached plan or
+ *   restores the whole expression to its original [[ScalarSubquery]] form.
+ * - [[ReuseSubquery]] rule makes sure that merged subqueries are computed once.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *   scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet

Review comment:
       >> In this PR the new MergeScalarSubqueries rule runs in a separate batch 
   >> after column pruning, close to the end of optimization. 
   >> This is by design to make sure no subsequent rule changes the structure 
   
   I don't see how we can guarantee that. I think the hidden, inter-rule dependency can add complexities for future development and maintenance. 
   
   For instance, someone could implement a new Strategy that internally calls ColumnPruning after exploring one logical plan alternative. By the time such a Strategy is implemented, the authors wouldn't be aware of the fact that ColumnPruning should *not* be called after MergeScalarSubqueries. 
   - First of all, such issues can hardly be detected by the author's new unit/query tests, as an effective test has to have both effective patterns to trigger the Strategy and for MergeScalarSubqueries. However, such a case can happen in prod traffic;
   - Second, if they do find that's an issue, they would then have to either (a) add some hacks in the Aggregate to mark that MergeScalarSubqueries has been applied and hence ColumnPruning should not go through it, or (b) re-implement the rule per my proposal (1).
     
   I'm wondering whether we can pursue (2) for now, if it meets your need. It's less ambitious but may address most of your issue?  If you indeed have to extract subqueries over the entire tree, I don't have a clean approach in mind other than (1).
   
   >> Add a performance improvement to 1. so as to physical plan a merged 
   >> subquery only once. This is your (1) basically.
   
   The performance was not my initial concern, but rather, we'd better make MergeScalarSubqueries self-contained and does not depend on *an assumption that could be changed*.
   
   >>  But I don't think we need lateral views
   
   Sub-querying over arrays are important use cases, for which we don't want to de-correlate. In this case, a subquery is more like an ordinary expression and should be evaluated within the operator 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] SparkQA removed a comment on pull request #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


   **[Test build #139635 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/139635/testReport)** for PR 32298 at commit [`f83f22b`](https://github.com/apache/spark/commit/f83f22bedf4247baea0c48906c4a7399996cd6d8).


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LeafNode, LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{MULTI_SCALAR_SUBQUERY, SCALAR_SUBQUERY}
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s into a
+ * [[MultiScalarSubquery]] to compute multiple scalar values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ * - The original [[ScalarSubquery]] expression is replaced to a reference pointing to its cached
+ *   version in this form: `GetStructField(MultiScalarSubquery(SubqueryReference(...)))`.
+ * - A second traversal checks if a [[SubqueryReference]] is pointing to a subquery plan that
+ *   returns multiple values and either replaces only [[SubqueryReference]] to the cached plan or
+ *   restores the whole expression to its original [[ScalarSubquery]] form.
+ * - [[ReuseSubquery]] rule makes sure that merged subqueries are computed once.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *   scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet

Review comment:
       > For instance, someone could implement a new Strategy that internally calls ColumnPruning after exploring one logical plan alternative. By the time such a Strategy is implemented, the authors wouldn't be aware of the fact that ColumnPruning should not be called after MergeScalarSubqueries.
   
   I see, thanks. I've never seen such transformations in `SparkStrategy`s. 
   
   But, if we followed (2) with non-correlated subqueries like this example:
   ```
   SELECT t1.*
   FROM t as t1
   JOIN t as t2 ON t2.a = t1.a
   WHERE
     t1.b = (SELECT sum(a) FROM t)
     AND t2.b = (SELECT count(a) FROM t)
   ```
   and if I get your (2) right the rewritten query is:
   ```
   SELECT t12.a, t12.b
   FROM (
     SELECT
       t1.*,
       (SELECT STRUCT(sum(a) AS sum_a, count(a) AS count_a) FROM t) AS st,
       t1.b AS t1_b,
       t2.b AS t2_b
     FROM t as t1
     JOIN t as t2 ON t2.a = t1.a
   ) t12
   WHERE
     t1_b = st.sum_a
     AND t2_b = st.count_a
   ```
   so we basically add an extra project node under `Filter`. The analyzed plan is:
   ```
   Project [a#237, b#238]
   +- Filter ((cast(t1_b#235 as bigint) = st#234.sum_a) AND (cast(t2_b#236 as bigint) = st#234.count_a))
      +- SubqueryAlias t12
         +- Project [a#237, b#238, scalar-subquery#233 [] AS st#234, b#238 AS t1_b#235, b#240 AS t2_b#236]
            :  +- Aggregate [struct(sum_a, sum(a#244), count_a, count(a#244)) AS struct(sum(a) AS sum_a, count(a) AS count_a)#243]
            :     +- SubqueryAlias spark_catalog.default.t
            :        +- Relation default.t[a#244,b#245] parquet
            +- Join Inner, (a#239 = a#237)
               :- SubqueryAlias t1
               :  +- SubqueryAlias spark_catalog.default.t
               :     +- Relation default.t[a#237,b#238] parquet
               +- SubqueryAlias t2
                  +- SubqueryAlias spark_catalog.default.t
                     +- Relation default.t[a#239,b#240] parquet
   ```
   The optimzer (`PushDownPredicates`) would duplicate and push down the subquery under both sides of the join:
   ```
   Project [a#237, b#238]
   +- Join Inner, (a#239 = a#237)
      :- Filter ((isnotnull(b#238) AND (cast(b#238 as bigint) = scalar-subquery#233 [].sum_a)) AND isnotnull(a#237))
      :  :  +- Aggregate [struct(sum_a, sum(a#244), count_a, count(a#244)) AS struct(sum(a) AS sum_a, count(a) AS count_a)#243]
      :  :     +- Project [a#244]
      :  :        +- Relation default.t[a#244,b#245] parquet
      :  +- Relation default.t[a#237,b#238] parquet
      +- Project [a#239]
         +- Filter ((isnotnull(b#240) AND (cast(b#240 as bigint) = scalar-subquery#233 [].count_a)) AND isnotnull(a#239))
            :  +- Aggregate [struct(sum_a, sum(a#244), count_a, count(a#244)) AS struct(sum(a) AS sum_a, count(a) AS count_a)#243]
            :     +- Project [a#244]
            :        +- Relation default.t[a#244,b#245] parquet
            +- Relation default.t[a#239,b#240] parquet
   ```
   Doesn't that mean that (2) also assumes that:
   - no subsequent transformation changes the 2 instances differently and 
   - `ReuseSubquery` does the dedup?
   




-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] zinking commented on a change in pull request #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,413 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CommonScalarSubqueries, Filter, Join, LogicalPlan, Project, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s to compute multiple scalar
+ * values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ *   During this first traversal each [[ScalarSubquery]] expression is replaced to a
+ *   [[ScalarSubqueryReference]] pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a results of merging 2 or more
+ *   plans, or it is a plan that was seen only once.
+ *   Merged plans in the cache get a "header" that is is basically
+ *   `CreateNamedStructure(name1, attribute1, name2, attribute2, ...)`
+ *   expression in new root [[Project]] node. This expression ensures that the merged plan is a
+ *   valid scalar subquery that returns only one value.
+ * - A second traversal checks if a [[ScalarSubqueryReference]] is pointing to a merged subquery
+ *   plan or not and either keeps the reference or restores the original [[ScalarSubquery]].
+ *   If there are [[ScalarSubqueryReference]] nodes remained a [[CommonScalarSubqueries]] root node
+ *   is added to the plan with the referenced scalar subqueries.
+ * - [[PlanSubqueries]] or [[PlanAdaptiveSubqueries]] rule does the physical planning of scalar
+ *   subqueries including the ones under [[CommonScalarSubqueriesExec]] node and replaces
+ *   each [[ScalarSubqueryReference]] to their referenced physical plan in
+ *   `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() header))` form.
+ *   It is important that references pointing to the same merged subquery are replaced to the same
+ *   planned instance to make sure that each merged subquery runs only once (even without a wrapping
+ *   [[ReuseSubquery]] node).
+ *   Finally, the [[CommonScalarSubqueriesExec]] node is removed from the physical plan.
+ * - The [[ReuseExchangeAndSubquery]] rule wraps the second, third, ... instances of the same
+ *   subquery into a [[ReuseSubquery]] node, but this just a cosmetic change in the plan.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *          scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * CommonScalarSubqueries [scalar-subquery#250 []]
+ * :  +- Project [named_struct(avg(a), avg(a)#236, sum(b), sum(b)#238L) AS mergedValue#249]
+ * :     +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :        +- Project [a#233, b#234]
+ * :           +- Relation default.t[a#233,b#234] parquet
+ * +- Project [scalarsubqueryreference(0, 0, DoubleType, 231) AS scalarsubquery()#241,
+ *             scalarsubqueryreference(0, 1, LongType, 232) AS scalarsubquery()#242L]
+ *    +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    if (conf.subqueryReuseEnabled) {
+      plan match {
+        case Subquery(_: CommonScalarSubqueries, _) => plan
+        case s: Subquery => s.copy(child = extractCommonScalarSubqueries(s.child))
+        case _: CommonScalarSubqueries => plan
+        case _ => extractCommonScalarSubqueries(plan)
+      }
+    } else {
+      plan
+    }
+  }
+
+  private def extractCommonScalarSubqueries(plan: LogicalPlan) = {
+    // Plan of subqueries and a flag is the plan is merged
+    val cache = ListBuffer.empty[(Project, Boolean)]
+    val newPlan = removeReferences(insertReferences(plan, cache), cache)
+    if (cache.nonEmpty) {
+      val scalarSubqueries = cache.map { case (header, _) => ScalarSubquery(header) }.toSeq
+      CommonScalarSubqueries(scalarSubqueries, newPlan)
+    } else {
+      newPlan
+    }
+  }
+
+  // First traversal builds up the cache and inserts `ScalarSubqueryReference`s to the plan.
+  private def insertReferences(
+      plan: LogicalPlan,
+      cache: ListBuffer[(Project, Boolean)]): LogicalPlan = {
+    plan.transformAllExpressionsWithPruning(_.containsAnyPattern(SCALAR_SUBQUERY)) {
+      case s: ScalarSubquery if s.children.isEmpty =>
+        val (subqueryIndex, headerIndex) = cacheSubquery(s.plan, cache)
+        ScalarSubqueryReference(subqueryIndex, headerIndex, s.dataType, s.exprId)
+    }
+  }
+
+  // Caching returns the index of the subquery in the cache and the index of scalar member in the
+  // `CreateNamedStruct` header.
+  private def cacheSubquery(
+      plan: LogicalPlan,
+      cache: ListBuffer[(Project, Boolean)]): (Int, Int) = {
+    val firstOutput = plan.output.head
+    cache.zipWithIndex.collectFirst(Function.unlift { case ((header, merged), subqueryIndex) =>
+      checkIdenticalPlans(plan, header.child)
+        .map((subqueryIndex, header, header.child, _, merged))
+        .orElse(tryMergePlans(plan, header.child).map {
+          case (mergedPlan, outputMap) => (subqueryIndex, header, mergedPlan, outputMap, true)
+        })
+    }).map { case (subqueryIndex, header, mergedPlan, outputMap, merged) =>
+      val mappedFirstOutput = mapAttributes(firstOutput, outputMap)
+      val headerElements = getHeaderElements(header)
+      var headerIndex = headerElements.indexWhere {
+        case (_, attribute) => attribute.exprId == mappedFirstOutput.exprId
+      }
+      if (headerIndex == -1) {
+        val newHeaderElements = headerElements :+ (Literal(firstOutput.name) -> mappedFirstOutput)
+        cache(subqueryIndex) = createHeader(newHeaderElements, mergedPlan) -> merged
+        headerIndex = headerElements.size
+      }
+      subqueryIndex -> headerIndex
+    }.getOrElse {
+      cache += createHeader(Seq(Literal(firstOutput.name) -> firstOutput), plan) -> false
+      cache.length - 1 -> 0
+    }
+  }
+
+  // If 2 plans are identical return the attribute mapping from the new to the cached version.
+  private def checkIdenticalPlans(newPlan: LogicalPlan, cachedPlan: LogicalPlan) = {
+    if (newPlan.canonicalized == cachedPlan.canonicalized) {
+      Some(AttributeMap(newPlan.output.zip(cachedPlan.output)))
+    } else {
+      None
+    }
+  }
+
+  // Recursively traverse down and try merging 2 plans. If merge is possible then return the merged
+  // plan with the attribute mapping from the new to the merged version.
+  // Please note that merging arbitrary plans can be complicated, the current version supports only
+  // some of the most important nodes.
+  private def tryMergePlans(
+      newPlan: LogicalPlan,
+      cachedPlan: LogicalPlan): Option[(LogicalPlan, AttributeMap[Attribute])] = {
+    checkIdenticalPlans(newPlan, cachedPlan).map(cachedPlan -> _).orElse(
+      (newPlan, cachedPlan) match {
+        case (np: Project, cp: Project) =>
+          tryMergePlans(np.child, cp.child).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.projectList, outputMap, cp.projectList)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np, cp: Project) =>
+          tryMergePlans(np, cp.child).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.output, outputMap, cp.projectList)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np: Project, cp) =>
+          tryMergePlans(np.child, cp).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.projectList, outputMap, cp.output)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np: Aggregate, cp: Aggregate) if supportedAggregateMerge(np, cp) =>
+          tryMergePlans(np.child, cp.child).flatMap { case (mergedChild, outputMap) =>
+            val mappedNewGroupingExpression =
+              np.groupingExpressions.map(mapAttributes(_, outputMap))
+            if (ExpressionSet(mappedNewGroupingExpression) ==
+              ExpressionSet(cp.groupingExpressions)) {
+              val (mergedAggregateExpressions, newOutputMap) =
+                mergeNamedExpressions(np.aggregateExpressions, outputMap, cp.aggregateExpressions)
+              val mergedPlan =
+                Aggregate(cp.groupingExpressions, mergedAggregateExpressions, mergedChild)
+              Some(mergedPlan -> newOutputMap)
+            } else {
+              None
+            }
+          }
+
+        // Merging general nodes is complicated and this implementation supports only those nodes in
+        // which the order and the number of output attributes are not relevant (see
+        // `supportedMerge()` whitelist).
+        // Also, this implementation supports only those nodes in which children can be merged in
+        // the same order.
+        case (np, cp) if supportedMerge(np) && np.getClass == cp.getClass &&
+          np.children.size == cp.children.size =>
+          val merged = np.children.zip(cp.children).map {
+            case (npChild, cpChild) => tryMergePlans(npChild, cpChild)
+          }
+          if (merged.forall(_.isDefined)) {
+            val (mergedChildren, outputMaps) = merged.map(_.get).unzip
+            val outputMap = AttributeMap(outputMaps.map(_.iterator).reduce(_ ++ _).toSeq)
+            val mappedNewPlan = mapAttributes(np.withNewChildren(mergedChildren), outputMap)
+            val mergedPlan = cp.withNewChildren(mergedChildren)
+            if (mappedNewPlan.canonicalized == mergedPlan.canonicalized) {
+              Some(mergedPlan -> outputMap)
+            } else {
+              None
+            }
+          } else {
+            None
+          }
+
+        // As a follow-up, it would be possible to merge `CommonScalarSubqueries` nodes, which would

Review comment:
       perhaps I am not catch all. wouldn't it be straight forward if you put the `Common` under the root node and merge reference plans there ? 




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] tgravescs commented on pull request #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


   If no other comments by tomorrow, I'm going to merge 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] sigmod edited a comment on pull request #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

Posted by GitBox <gi...@apache.org>.
sigmod edited a comment on pull request #32298:
URL: https://github.com/apache/spark/pull/32298#issuecomment-1075524713


   > But I think that an extra shuffle could mean performance degradation
   > in case of scalar subqueries (CTEs returning only one row).
   
   Is it still way better than running the scalar subqueries over the same table multiple times?
   I'm more worried about the complexities (i.e., pattern matching cognitive overhead) with new plan nodes like CommonSubqueries and CommonSubqueriesExec. Many rules have been implemented as pattern matching, e.g., a rule that matches a `Project` is supposed to also match `CommonSubqueries` (in theory)?


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


   >Is it still way better than running the scalar subqueries over the same table multiple times?
   
   If the scalar subqueries to be merged are quick then I'm not sure.
   
   > I'm more worried about the complexities (i.e., pattern matching cognitive overhead) with new plan nodes like CommonSubqueries and CommonSubqueriesExec. Many rules have been implemented as pattern matching, e.g., a rule that matches a Project is supposed to also match CommonSubqueries (in theory)?
   
   Hmm, `CommonScalarSubqueries` (and `CommonScalarSubqueriesExec`) can appear only as a root node, it is't exactly the same as `Project`. It never replaces Project. I think it is more similar to `WithCTE` in that regard. Can you give an example where we need to handle these new nodes as `Project`?


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] tgravescs edited a comment on pull request #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

Posted by GitBox <gi...@apache.org>.
tgravescs edited a comment on pull request #32298:
URL: https://github.com/apache/spark/pull/32298#issuecomment-1069593711


   @peter-toth would you mind rekicking the tests on this?  
   It sounds like you have done quite a bit of manual testing on this with tpcds and possibly other queries? 
   We are interested since we were trying something similar with @sperlingxx PRs.  I'm reviewing in more detail and testing it out.
   I think it would be great to get this into Spark 3.3 if no objections.
   It would also be great to get feedback from others who already commented @sigmod 


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,421 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CommonScalarSubqueries, Filter, Join, LogicalPlan, Project, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s to compute multiple scalar
+ * values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ *   During this first traversal each [[ScalarSubquery]] expression is replaced to a
+ *   [[ScalarSubqueryReference]] pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a results of merging 2 or more
+ *   plans, or it is a plan that was seen only once.
+ *   Merged plans in the cache get a "header", that is is basically
+ *   `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in new root
+ *   [[Project]] node. This expression ensures that the merged plan is a valid scalar subquery that
+ *   returns only one value.
+ * - A second traversal checks if a [[ScalarSubqueryReference]] is pointing to a merged subquery
+ *   plan or not and either keeps the reference or restores the original [[ScalarSubquery]].
+ *   If there are [[ScalarSubqueryReference]] nodes remained a [[CommonScalarSubqueries]] root node
+ *   is added to the plan with the referenced scalar subqueries.
+ * - [[PlanSubqueries]] or [[PlanAdaptiveSubqueries]] rule does the physical planning of scalar
+ *   subqueries including the ones under [[CommonScalarSubqueriesExec]] node and replaces
+ *   each [[ScalarSubqueryReference]] to their referenced physical plan in
+ *   `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() header))` form.
+ *   It is important that references pointing to the same merged subquery are replaced to the same
+ *   planned instance to make sure that each merged subquery runs only once (even without a wrapping
+ *   [[ReuseSubquery]] node).
+ *   Finally, the [[CommonScalarSubqueriesExec]] node is removed from the physical plan.
+ * - The [[ReuseExchangeAndSubquery]] rule wraps the second, third, ... instances of the same
+ *   subquery into a [[ReuseSubquery]] node, but this just a cosmetic change in the plan.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *          scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * CommonScalarSubqueries [scalar-subquery#250 []]
+ * :  +- Project [named_struct(avg(a), avg(a)#236, sum(b), sum(b)#238L) AS mergedValue#249]
+ * :     +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :        +- Project [a#233, b#234]
+ * :           +- Relation default.t[a#233,b#234] parquet
+ * +- Project [scalarsubqueryreference(0, 0, DoubleType, 231) AS scalarsubquery()#241,
+ *             scalarsubqueryreference(0, 1, LongType, 232) AS scalarsubquery()#242L]
+ *    +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    plan match {
+      case Subquery(_: CommonScalarSubqueries, _) => plan
+      case s: Subquery => s.copy(child = extractCommonScalarSubqueries(s.child))
+      case _: CommonScalarSubqueries => plan
+      case _ => extractCommonScalarSubqueries(plan)
+    }
+  }
+
+  /**
+   * An item in the cache of merged scalar subqueries.
+   *
+   * @param elements  List of attributes that form the scalar return value of a merged subquery
+   * @param plan      The plan of a merged scalar subquery
+   * @param merged    A flag to identify if this item is the result of merging subqueries.
+   *                  Please note that `elements.size == 1` doesn't always mean that the plan is not
+   *                  merged as there can be subqueries that are different ([[checkIdenticalPlans]]
+   *                  is false) due to an extra [[Project]] node in one of them. In that case
+   *                  `elements.size` remains 1 after merging, but the merged flag becomes true.
+   */
+  case class Header(elements: Seq[(String, Attribute)], plan: LogicalPlan, merged: Boolean)
+
+  private def extractCommonScalarSubqueries(plan: LogicalPlan) = {
+    val cache = ListBuffer.empty[Header]
+    val newPlan = removeReferences(insertReferences(plan, cache), cache)
+    if (cache.nonEmpty) {
+      val scalarSubqueries = cache.map {
+        case Header(elements, child, _) => ScalarSubquery(createProject(elements, child))
+      }.toSeq
+      CommonScalarSubqueries(scalarSubqueries, newPlan)
+    } else {
+      newPlan
+    }
+  }
+
+  // First traversal builds up the cache and inserts `ScalarSubqueryReference`s to the plan.
+  private def insertReferences(plan: LogicalPlan, cache: ListBuffer[Header]): LogicalPlan = {
+    plan.transformAllExpressionsWithPruning(_.containsAnyPattern(SCALAR_SUBQUERY)) {
+      case s: ScalarSubquery if !s.isCorrelated && s.deterministic =>
+        val (subqueryIndex, headerIndex) = cacheSubquery(s.plan, cache)
+        ScalarSubqueryReference(subqueryIndex, headerIndex, s.dataType, s.exprId)
+    }
+  }
+
+  // Caching returns the index of the subquery in the cache and the index of scalar member in the
+  // "header", that `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in a
+  // [[Project]] node.
+  private def cacheSubquery(plan: LogicalPlan, cache: ListBuffer[Header]): (Int, Int) = {
+    val output = plan.output.head
+    cache.zipWithIndex.collectFirst(Function.unlift { case (header, subqueryIndex) =>
+      checkIdenticalPlans(plan, header.plan).map { outputMap =>
+        val mappedOutput = mapAttributes(output, outputMap)
+        val headerIndex = header.elements.indexWhere {
+          case (_, attribute) => attribute.exprId == mappedOutput.exprId
+        }
+        subqueryIndex -> headerIndex
+      }.orElse(tryMergePlans(plan, header.plan).map {
+        case (mergedPlan, outputMap) =>
+          val mappedOutput = mapAttributes(output, outputMap)
+          var headerIndex = header.elements.indexWhere {
+            case (_, attribute) => attribute.exprId == mappedOutput.exprId
+          }
+          val newHeaderElements = if (headerIndex == -1) {
+            headerIndex = header.elements.size
+            header.elements :+ (output.name -> mappedOutput)
+          } else {
+            header.elements
+          }
+          cache(subqueryIndex) = Header(newHeaderElements, mergedPlan, true)
+          subqueryIndex -> headerIndex
+      })
+    }).getOrElse {
+      cache += Header(Seq(output.name -> output), plan, false)
+      cache.length - 1 -> 0
+    }
+  }
+
+  // If 2 plans are identical return the attribute mapping from the new to the cached version.
+  private def checkIdenticalPlans(
+      newPlan: LogicalPlan,
+      cachedPlan: LogicalPlan): Option[AttributeMap[Attribute]] = {
+    if (newPlan.canonicalized == cachedPlan.canonicalized) {
+      Some(AttributeMap(newPlan.output.zip(cachedPlan.output)))
+    } else {
+      None
+    }
+  }
+
+  // Recursively traverse down and try merging 2 plans. If merge is possible then return the merged
+  // plan with the attribute mapping from the new to the merged version.
+  // Please note that merging arbitrary plans can be complicated, the current version supports only
+  // some of the most important nodes.
+  private def tryMergePlans(
+      newPlan: LogicalPlan,
+      cachedPlan: LogicalPlan): Option[(LogicalPlan, AttributeMap[Attribute])] = {
+    checkIdenticalPlans(newPlan, cachedPlan).map(cachedPlan -> _).orElse(
+      (newPlan, cachedPlan) match {
+        case (np: Project, cp: Project) =>
+          tryMergePlans(np.child, cp.child).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.projectList, outputMap, cp.projectList)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np, cp: Project) =>
+          tryMergePlans(np, cp.child).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.output, outputMap, cp.projectList)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np: Project, cp) =>
+          tryMergePlans(np.child, cp).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.projectList, outputMap, cp.output)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np: Aggregate, cp: Aggregate) if supportedAggregateMerge(np, cp) =>
+          tryMergePlans(np.child, cp.child).flatMap { case (mergedChild, outputMap) =>
+            val mappedNewGroupingExpression =
+              np.groupingExpressions.map(mapAttributes(_, outputMap))
+            // Order of grouping expression doesn't matter so we can compare sets
+            if (ExpressionSet(mappedNewGroupingExpression) ==
+              ExpressionSet(cp.groupingExpressions)) {
+              val (mergedAggregateExpressions, newOutputMap) =
+                mergeNamedExpressions(np.aggregateExpressions, outputMap, cp.aggregateExpressions)
+              val mergedPlan =
+                Aggregate(cp.groupingExpressions, mergedAggregateExpressions, mergedChild)
+              Some(mergedPlan -> newOutputMap)
+            } else {
+              None
+            }
+          }
+
+        // Merging general nodes is complicated and this implementation supports only those nodes in
+        // which the order and the number of output attributes are not relevant (see
+        // `supportedMerge()` whitelist).
+        // Also, this implementation supports only those nodes in which children can be merged in
+        // the same order.
+        case (np, cp) if supportedMerge(np) && np.getClass == cp.getClass &&
+            np.children.size == cp.children.size &&
+            np.expressions.size == cp.expressions.size &&
+            // Fields that don't contain any children or expressions should match
+            np.productIterator.filterNot(np.children.contains)
+              .filter(QueryPlan.extractExpressions(_).isEmpty).toSeq ==
+              cp.productIterator.filterNot(cp.children.contains)
+                .filter(QueryPlan.extractExpressions(_).isEmpty).toSeq =>
+          val merged = np.children.zip(cp.children).map {
+            case (npChild, cpChild) => tryMergePlans(npChild, cpChild)
+          }
+          if (merged.forall(_.isDefined)) {
+            val (mergedChildren, outputMaps) = merged.map(_.get).unzip
+            val outputMap = AttributeMap(outputMaps.map(_.iterator).reduce(_ ++ _).toSeq)
+            // We know that fields that don't contain any children or expressions do match and
+            // children can be merged so we need to test expressions only
+            if (np.expressions.map(mapAttributes(_, outputMap).canonicalized) ==
+              cp.expressions.map(_.canonicalized)) {
+              val mergedPlan = cp.withNewChildren(mergedChildren)
+              Some(mergedPlan -> outputMap)
+            } else {
+              None
+            }
+          } else {
+            None
+          }
+
+        // As a follow-up, it would be possible to merge `CommonScalarSubqueries` nodes, which would
+        // allow merging subqueries with mergee subqueries.
+        // E.g. this query:
+        //
+        // SELECT
+        //   (
+        //     SELECT
+        //       (SELECT avg(a) FROM t GROUP BY b) +
+        //       (SELECT sum(b) FROM t GROUP BY b)
+        //   ),
+        //   (
+        //     SELECT
+        //       (SELECT max(a) FROM t GROUP BY b) +
+        //       (SELECT min(b) FROM t GROUP BY b)
+        //   )
+        //
+        // is currently optimized to:
+        //
+        // == Optimized Logical Plan ==
+        // Project [scalar-subquery#233 [] AS scalarsubquery()#255,
+        //          scalar-subquery#236 [] AS scalarsubquery()#256]
+        // :  :- CommonScalarSubqueries [scalar-subquery#264 []]
+        // :  :  :  +- Aggregate [b#238], [named_struct(avg(a), avg(a#237), sum(b), sum(b#238))
+        //                                 AS mergedValue#263]
+        // :  :  :     +- Relation default.t[a#237,b#238] parquet
+        // :  :  +- Project [(scalarsubqueryreference(0, 0, DoubleType, 231) +
+        //                   cast(scalarsubqueryreference(0, 1, LongType, 232) as double))
+        //                   AS (scalarsubquery() + scalarsubquery())#245]
+        // :  :     +- OneRowRelation
+        // :  +- CommonScalarSubqueries [scalar-subquery#269 []]
+        // :     :  +- Aggregate [b#254], [named_struct(min(a), min(a#253), max(b), max(b#254))
+        //                                 AS mergedValue#268]
+        // :     :     +- Relation default.t[a#253,b#254] parquet
+        // :     +- Project [(scalarsubqueryreference(0, 0, IntegerType, 234) +
+        //                   scalarsubqueryreference(0, 1, IntegerType, 235))
+        //                   AS (scalarsubquery() + scalarsubquery())#252]
+        // :        +- OneRowRelation
+        // +- OneRowRelation
+        //
+        // but if we implemented merging `CommonScalarSubqueries` nodes then the plan could be
+        // transformed further and all leaf subqueries could be merged.
+
+        // Otherwise merging is not possible.
+        case _ => None
+      })
+  }
+
+  private def createProject(elements: Seq[(String, Attribute)], plan: LogicalPlan): Project = {
+    Project(
+      Seq(Alias(
+        CreateNamedStruct(elements.flatMap {
+          case (name, attribute) => Seq(Literal(name), attribute)
+        }),
+        "mergedValue")()),
+      plan)
+  }
+
+  private def mapAttributes[T <: Expression](expr: T, outputMap: AttributeMap[Attribute]) = {
+    expr.transform {
+      case a: Attribute => outputMap.getOrElse(a, a)
+    }.asInstanceOf[T]
+  }
+
+  // Applies `outputMap` attribute mapping on elements of `newExpressions` and merges them into
+  // `cachedExpressions`. Returns the merged expressions and the attribute mapping from the new to
+  // the merged version that can be propagated up during merging nodes.
+  private def mergeNamedExpressions(
+      newExpressions: Seq[NamedExpression],
+      outputMap: AttributeMap[Attribute],
+      cachedExpressions: Seq[NamedExpression]) = {
+    val mergedExpressions = ListBuffer[NamedExpression](cachedExpressions: _*)
+    val newOutputMap = AttributeMap(newExpressions.map { ne =>
+      val mapped = mapAttributes(ne, outputMap)
+      val withoutAlias = mapped match {
+        case Alias(child, _) => child
+        case e => e
+      }
+      ne.toAttribute -> mergedExpressions.find {
+        case Alias(child, _) => child semanticEquals withoutAlias
+        case e => e semanticEquals withoutAlias
+      }.getOrElse {
+        mergedExpressions += mapped
+        mapped
+      }.toAttribute
+    })
+    (mergedExpressions.toSeq, newOutputMap)
+  }
+
+  // Merging different aggregate implementations could cause performance regression

Review comment:
       Done in https://github.com/apache/spark/pull/32298/commits/87ba289589c41a9c83e39c493e4dabf90b95c14a




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/InjectRuntimeFilterSuite.scala
##########
@@ -253,20 +253,14 @@ class InjectRuntimeFilterSuite extends QueryTest with SQLTestUtils with SharedSp
   }
 
   def getNumBloomFilters(plan: LogicalPlan): Integer = {
-    val numBloomFilterAggs = plan.collect {
-      case Filter(condition, _) => condition.collect {
-        case subquery: org.apache.spark.sql.catalyst.expressions.ScalarSubquery
-        => subquery.plan.collect {
-          case Aggregate(_, aggregateExpressions, _) =>
-            aggregateExpressions.map {
-              case Alias(AggregateExpression(bfAgg : BloomFilterAggregate, _, _, _, _),
-              _) =>
-                assert(bfAgg.estimatedNumItemsExpression.isInstanceOf[Literal])
-                assert(bfAgg.numBitsExpression.isInstanceOf[Literal])
-                1
-            }.sum
+    val numBloomFilterAggs = plan.collectWithSubqueries {

Review comment:
       cc @somani 




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


   **[Test build #139635 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/139635/testReport)** for PR 32298 at commit [`f83f22b`](https://github.com/apache/spark/commit/f83f22bedf4247baea0c48906c4a7399996cd6d8).


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


   **[Test build #138112 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138112/testReport)** for PR 32298 at commit [`e63111d`](https://github.com/apache/spark/commit/e63111d2e26b92a983c5561b90608d80e0b4e7ee).


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala
##########
@@ -267,6 +268,33 @@ object ScalarSubquery {
   }
 }
 
+/**
+ * A subquery that is capable to return multiple scalar values.
+ */
+case class MultiScalarSubquery(
+    plan: LogicalPlan,
+    exprId: ExprId = NamedExpression.newExprId)
+  extends SubqueryExpression(plan, Seq.empty, exprId) with LeafLike[Expression] with Unevaluable {
+  override def dataType: DataType = {
+    assert(plan.schema.nonEmpty, "Multi-column scalar subquery should have columns")

Review comment:
       This is not relevant in latest 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 removed a comment on pull request #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


   @cloud-fan, @maropu, @sigmod, I've updated this PR with the (1) suggestion: https://github.com/apache/spark/pull/32298#discussion_r627800694 and also added tests and comments.
   
   I've just realized that I need to update the PR description too, will do it 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] peter-toth commented on pull request #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


   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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


   **[Test build #140073 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/140073/testReport)** for PR 32298 at commit [`41c0f0a`](https://github.com/apache/spark/commit/41c0f0aa9a1ab0b19a538e20b586807e39059eec).


-- 
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] zinking commented on a change in pull request #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,413 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CommonScalarSubqueries, Filter, Join, LogicalPlan, Project, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s to compute multiple scalar
+ * values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ *   During this first traversal each [[ScalarSubquery]] expression is replaced to a
+ *   [[ScalarSubqueryReference]] pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a results of merging 2 or more
+ *   plans, or it is a plan that was seen only once.
+ *   Merged plans in the cache get a "header" that is is basically
+ *   `CreateNamedStructure(name1, attribute1, name2, attribute2, ...)`
+ *   expression in new root [[Project]] node. This expression ensures that the merged plan is a
+ *   valid scalar subquery that returns only one value.
+ * - A second traversal checks if a [[ScalarSubqueryReference]] is pointing to a merged subquery
+ *   plan or not and either keeps the reference or restores the original [[ScalarSubquery]].
+ *   If there are [[ScalarSubqueryReference]] nodes remained a [[CommonScalarSubqueries]] root node
+ *   is added to the plan with the referenced scalar subqueries.
+ * - [[PlanSubqueries]] or [[PlanAdaptiveSubqueries]] rule does the physical planning of scalar
+ *   subqueries including the ones under [[CommonScalarSubqueriesExec]] node and replaces
+ *   each [[ScalarSubqueryReference]] to their referenced physical plan in
+ *   `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() header))` form.
+ *   It is important that references pointing to the same merged subquery are replaced to the same
+ *   planned instance to make sure that each merged subquery runs only once (even without a wrapping
+ *   [[ReuseSubquery]] node).
+ *   Finally, the [[CommonScalarSubqueriesExec]] node is removed from the physical plan.
+ * - The [[ReuseExchangeAndSubquery]] rule wraps the second, third, ... instances of the same
+ *   subquery into a [[ReuseSubquery]] node, but this just a cosmetic change in the plan.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *          scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * CommonScalarSubqueries [scalar-subquery#250 []]
+ * :  +- Project [named_struct(avg(a), avg(a)#236, sum(b), sum(b)#238L) AS mergedValue#249]
+ * :     +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :        +- Project [a#233, b#234]
+ * :           +- Relation default.t[a#233,b#234] parquet
+ * +- Project [scalarsubqueryreference(0, 0, DoubleType, 231) AS scalarsubquery()#241,
+ *             scalarsubqueryreference(0, 1, LongType, 232) AS scalarsubquery()#242L]
+ *    +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    if (conf.subqueryReuseEnabled) {
+      plan match {
+        case Subquery(_: CommonScalarSubqueries, _) => plan
+        case s: Subquery => s.copy(child = extractCommonScalarSubqueries(s.child))
+        case _: CommonScalarSubqueries => plan
+        case _ => extractCommonScalarSubqueries(plan)
+      }
+    } else {
+      plan
+    }
+  }
+
+  private def extractCommonScalarSubqueries(plan: LogicalPlan) = {
+    // Plan of subqueries and a flag is the plan is merged
+    val cache = ListBuffer.empty[(Project, Boolean)]
+    val newPlan = removeReferences(insertReferences(plan, cache), cache)
+    if (cache.nonEmpty) {
+      val scalarSubqueries = cache.map { case (header, _) => ScalarSubquery(header) }.toSeq
+      CommonScalarSubqueries(scalarSubqueries, newPlan)
+    } else {
+      newPlan
+    }
+  }
+
+  // First traversal builds up the cache and inserts `ScalarSubqueryReference`s to the plan.
+  private def insertReferences(
+      plan: LogicalPlan,
+      cache: ListBuffer[(Project, Boolean)]): LogicalPlan = {
+    plan.transformAllExpressionsWithPruning(_.containsAnyPattern(SCALAR_SUBQUERY)) {
+      case s: ScalarSubquery if s.children.isEmpty =>
+        val (subqueryIndex, headerIndex) = cacheSubquery(s.plan, cache)
+        ScalarSubqueryReference(subqueryIndex, headerIndex, s.dataType, s.exprId)
+    }
+  }
+
+  // Caching returns the index of the subquery in the cache and the index of scalar member in the
+  // `CreateNamedStruct` header.
+  private def cacheSubquery(
+      plan: LogicalPlan,
+      cache: ListBuffer[(Project, Boolean)]): (Int, Int) = {
+    val firstOutput = plan.output.head
+    cache.zipWithIndex.collectFirst(Function.unlift { case ((header, merged), subqueryIndex) =>
+      checkIdenticalPlans(plan, header.child)
+        .map((subqueryIndex, header, header.child, _, merged))
+        .orElse(tryMergePlans(plan, header.child).map {
+          case (mergedPlan, outputMap) => (subqueryIndex, header, mergedPlan, outputMap, true)
+        })
+    }).map { case (subqueryIndex, header, mergedPlan, outputMap, merged) =>
+      val mappedFirstOutput = mapAttributes(firstOutput, outputMap)
+      val headerElements = getHeaderElements(header)
+      var headerIndex = headerElements.indexWhere {
+        case (_, attribute) => attribute.exprId == mappedFirstOutput.exprId
+      }
+      if (headerIndex == -1) {
+        val newHeaderElements = headerElements :+ (Literal(firstOutput.name) -> mappedFirstOutput)
+        cache(subqueryIndex) = createHeader(newHeaderElements, mergedPlan) -> merged
+        headerIndex = headerElements.size
+      }
+      subqueryIndex -> headerIndex
+    }.getOrElse {
+      cache += createHeader(Seq(Literal(firstOutput.name) -> firstOutput), plan) -> false
+      cache.length - 1 -> 0
+    }
+  }
+
+  // If 2 plans are identical return the attribute mapping from the new to the cached version.
+  private def checkIdenticalPlans(newPlan: LogicalPlan, cachedPlan: LogicalPlan) = {
+    if (newPlan.canonicalized == cachedPlan.canonicalized) {
+      Some(AttributeMap(newPlan.output.zip(cachedPlan.output)))
+    } else {
+      None
+    }
+  }
+
+  // Recursively traverse down and try merging 2 plans. If merge is possible then return the merged
+  // plan with the attribute mapping from the new to the merged version.
+  // Please note that merging arbitrary plans can be complicated, the current version supports only
+  // some of the most important nodes.
+  private def tryMergePlans(
+      newPlan: LogicalPlan,
+      cachedPlan: LogicalPlan): Option[(LogicalPlan, AttributeMap[Attribute])] = {
+    checkIdenticalPlans(newPlan, cachedPlan).map(cachedPlan -> _).orElse(
+      (newPlan, cachedPlan) match {
+        case (np: Project, cp: Project) =>
+          tryMergePlans(np.child, cp.child).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.projectList, outputMap, cp.projectList)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np, cp: Project) =>
+          tryMergePlans(np, cp.child).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.output, outputMap, cp.projectList)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np: Project, cp) =>
+          tryMergePlans(np.child, cp).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.projectList, outputMap, cp.output)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np: Aggregate, cp: Aggregate) if supportedAggregateMerge(np, cp) =>
+          tryMergePlans(np.child, cp.child).flatMap { case (mergedChild, outputMap) =>
+            val mappedNewGroupingExpression =
+              np.groupingExpressions.map(mapAttributes(_, outputMap))
+            if (ExpressionSet(mappedNewGroupingExpression) ==
+              ExpressionSet(cp.groupingExpressions)) {
+              val (mergedAggregateExpressions, newOutputMap) =
+                mergeNamedExpressions(np.aggregateExpressions, outputMap, cp.aggregateExpressions)
+              val mergedPlan =
+                Aggregate(cp.groupingExpressions, mergedAggregateExpressions, mergedChild)
+              Some(mergedPlan -> newOutputMap)
+            } else {
+              None
+            }
+          }
+
+        // Merging general nodes is complicated and this implementation supports only those nodes in
+        // which the order and the number of output attributes are not relevant (see
+        // `supportedMerge()` whitelist).
+        // Also, this implementation supports only those nodes in which children can be merged in
+        // the same order.
+        case (np, cp) if supportedMerge(np) && np.getClass == cp.getClass &&
+          np.children.size == cp.children.size =>
+          val merged = np.children.zip(cp.children).map {
+            case (npChild, cpChild) => tryMergePlans(npChild, cpChild)
+          }
+          if (merged.forall(_.isDefined)) {
+            val (mergedChildren, outputMaps) = merged.map(_.get).unzip
+            val outputMap = AttributeMap(outputMaps.map(_.iterator).reduce(_ ++ _).toSeq)
+            val mappedNewPlan = mapAttributes(np.withNewChildren(mergedChildren), outputMap)
+            val mergedPlan = cp.withNewChildren(mergedChildren)
+            if (mappedNewPlan.canonicalized == mergedPlan.canonicalized) {
+              Some(mergedPlan -> outputMap)
+            } else {
+              None
+            }
+          } else {
+            None
+          }
+
+        // As a follow-up, it would be possible to merge `CommonScalarSubqueries` nodes, which would

Review comment:
       guess it involves more execution work though




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


   @cloud-fan, @maropu, @sigmod, I've updated this PR with the (1) suggestion: https://github.com/apache/spark/pull/32298#discussion_r627800694 and also added tests and comments.
   


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


   **[Test build #145957 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/145957/testReport)** for PR 32298 at commit [`fa5e786`](https://github.com/apache/spark/commit/fa5e7860d05e224f42d8005f0ecb2cf89e8d27b0).
    * 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] maryannxue commented on pull request #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


   @peter-toth I'm going to push a PR to defer CTE inlining to after query analysis and avoid inlining of non-deterministic CTEs. It'll include some new CTE nodes that might be useful for your PR here.


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


   @tgravescs, thanks for your feedback and interest in in this PR. I reran the tests, they seem to be fine now.
   This change adds basic subquery merging logic and affects only `q9` from TPCDS, but it brings considerable performance improvement to it.
   The change has been ready for review for a while, but haven't got much attention. Your and others review would be much appreciated.
   
   I also have some follow-up PRs/plans:
   - to add more advanced logic to support merging aggregate subqueries with different `Filter` nodes
   - support DSv2
   - try merging correlated subqueries
   
   but this PR is complex enough so I would add these in subsequent PRs only.
   
   We can target 3.3, but AFAIK the branch has been cut so probably we need some kind of approval on the dev list?
   


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


   @sigmod,
   - https://github.com/apache/spark/pull/32298/commits/96d0cabef001960ace0b193b01863a8377f78b6f changes the implementation to use CTE nodes.
   - https://github.com/apache/spark/pull/32298/commits/a57ed32588b8df6e3d5bafbba950c96fa48181a9 adds the flag to `CTERelationRef` and changes `WithCTEStrategy` to avoid extra shuffles if the CTE is used as a subquery.
   
   I updated the PR description and benchmark results 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


   > * since we've already had `WithCTE` and `CTERef`,  we probably don't want to re-invent a similar wheel for a narrower case (i.e., shared evaluation of subqueries). I suspect the additional performance gain is marginal, compared to the gain of shared scan and aggregate computation.
   
   I agree that `WithCTE` is similar to `CommonSubqueries` as they are both root nodes and both host common definitions.
   But, I'm a bit reluctant to accept that we should handle common scalar subqueries as CTEs as we know that `CTERef` are invented for 1 special purpose and always do an extra shuffle. I think the difference between a CTE, that can return many-many rows and a subquery, that always returns max. one value, is huge. I don't see why we should force using `CTERef` if we can easily give a better alternative. 
   
   > * after this PR, subquery related rules also may have to think about `SubqueryReference` and `CommonSubqueries`;
   
   I'm not sure about this. `CommonSubqueries` does nothing, just like `WithCTE`. The point of `SubqueryReference` is exactly to move away the (merged) subquery from an arbitrary node to a common place to avoid any rule changing it at its original place. Once the reference is inserted, subsequent rules should handle it as they handle a literal and no longer as a subquery.
   
   > * what would happen if a `WithCTE` co-exists with `CommonSubqueries`;
   
   Actually this is a good question and this PR doesn't have any test to cover that rare scenario. As we agree that both are definition hosting root nodes, how about combining them into one node (`CommonDefinitions`?) that can host CTEs and scalar subqueries as well?
   
   > * `ColumnPruning` may also need to consider `SubqueryReference` and `CommonSubqueries`.
   
   I'm not sure I get this. Why `ColumnPruning` should consider these new nodes?
   


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


   @sigmod, how about doing this kind of transformation?
   
   ```
   SELECT
     (SELECT avg(a) FROM t GROUP BY b),
     (SELECT sum(b) FROM t GROUP BY b)
   FROM T
   ```
   => 
   ```
   WITH cte AS (SELECT named_struct(a, avg(a), b, sum(b)) mergedValue FROM t GROUP BY b)
   SELECT (SELECT mergedValue FROM cte).a, (SELECT mergedValue FROM cte).b
   FROM T
   ```
   and adding a flag to `cte` `CTERelationDef` that it hosts a scalar query + changing `WithCTEStrategy` a bit to avoid extra shuffles in those cases as `ReuseExchangeAndSubquery` can insert `ReusedSubqueryExec` nodes (no need to insert  `ReusedExchangeExec`). 


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


   **[Test build #146071 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/146071/testReport)** for PR 32298 at commit [`96a502d`](https://github.com/apache/spark/commit/96a502d6f4ed44ed56a92f636f793d365dfa69d3).


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,416 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CommonScalarSubqueries, Filter, Join, LogicalPlan, Project, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s to compute multiple scalar
+ * values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ *   During this first traversal each [[ScalarSubquery]] expression is replaced to a
+ *   [[ScalarSubqueryReference]] pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a results of merging 2 or more
+ *   plans, or it is a plan that was seen only once.
+ *   Merged plans in the cache get a "header", that is is basically
+ *   `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in new root
+ *   [[Project]] node. This expression ensures that the merged plan is a valid scalar subquery that
+ *   returns only one value.
+ * - A second traversal checks if a [[ScalarSubqueryReference]] is pointing to a merged subquery
+ *   plan or not and either keeps the reference or restores the original [[ScalarSubquery]].
+ *   If there are [[ScalarSubqueryReference]] nodes remained a [[CommonScalarSubqueries]] root node
+ *   is added to the plan with the referenced scalar subqueries.
+ * - [[PlanSubqueries]] or [[PlanAdaptiveSubqueries]] rule does the physical planning of scalar
+ *   subqueries including the ones under [[CommonScalarSubqueriesExec]] node and replaces
+ *   each [[ScalarSubqueryReference]] to their referenced physical plan in
+ *   `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() header))` form.
+ *   It is important that references pointing to the same merged subquery are replaced to the same
+ *   planned instance to make sure that each merged subquery runs only once (even without a wrapping
+ *   [[ReuseSubquery]] node).
+ *   Finally, the [[CommonScalarSubqueriesExec]] node is removed from the physical plan.
+ * - The [[ReuseExchangeAndSubquery]] rule wraps the second, third, ... instances of the same
+ *   subquery into a [[ReuseSubquery]] node, but this just a cosmetic change in the plan.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *          scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * CommonScalarSubqueries [scalar-subquery#250 []]
+ * :  +- Project [named_struct(avg(a), avg(a)#236, sum(b), sum(b)#238L) AS mergedValue#249]
+ * :     +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :        +- Project [a#233, b#234]
+ * :           +- Relation default.t[a#233,b#234] parquet
+ * +- Project [scalarsubqueryreference(0, 0, DoubleType, 231) AS scalarsubquery()#241,
+ *             scalarsubqueryreference(0, 1, LongType, 232) AS scalarsubquery()#242L]
+ *    +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    if (conf.subqueryReuseEnabled) {
+      plan match {
+        case Subquery(_: CommonScalarSubqueries, _) => plan
+        case s: Subquery => s.copy(child = extractCommonScalarSubqueries(s.child))
+        case _: CommonScalarSubqueries => plan
+        case _ => extractCommonScalarSubqueries(plan)
+      }
+    } else {
+      plan
+    }
+  }
+
+  type Header = Project

Review comment:
       This is the PR against your PR with the refactor:
   https://github.com/peter-toth/spark/pull/4
   
   
   




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LeafNode, LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{MULTI_SCALAR_SUBQUERY, SCALAR_SUBQUERY}
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s into a
+ * [[MultiScalarSubquery]] to compute multiple scalar values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ * - The original [[ScalarSubquery]] expression is replaced to a reference pointing to its cached
+ *   version in this form: `GetStructField(MultiScalarSubquery(SubqueryReference(...)))`.
+ * - A second traversal checks if a [[SubqueryReference]] is pointing to a subquery plan that
+ *   returns multiple values and either replaces only [[SubqueryReference]] to the cached plan or
+ *   restores the whole expression to its original [[ScalarSubquery]] form.
+ * - [[ReuseSubquery]] rule makes sure that merged subqueries are computed once.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *   scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet

Review comment:
       > The proposed rule augments two subqueries, makes them look identical, and hopes (a) column-pruning doesn't prune too aggressively and (b) physical de-dup could dedup them. In case (a) changes later and the two aggregate trees are not deduped in the physical plan, there could potentially be regressions -- each aggregation then becomes more expensive.
   
   In this PR the new `MergeScalarSubqueries` rule runs in a separate batch after column pruning, close to the end of optimization. This is by design to make sure no subsequent rule changes the structure of different instances of a merged subquery plan at different places in the logical plan differently. So the physical planing creates the same physical plan for these instances and there shouldn't be any dedup issues.
   Update: I need to recheck this part as the current PR might not do what I wanted to.
   
   I think probably the downside of my current PR is that the physical planning of merged subqueries happen multiple times (as many times as they they appear in the logical plan) and physical dedup comes only after that. This could be improved if we had subquery references in logical plan as well (something like `ReuseSubqueryExec`). But I think that's what your (1) is about. Move the merged subqueries to a special top logical plan node and add subquery references at places where they are actually used.
   
   > SELECT y
   FROM LATERAL VIEW explode(ARRAY(ARRAY(1), ARRAY(1, 2), ARRAY(1, 2, 3))) AS y
   WHERE
   ( SELECT COUNT(*) FROM LATERAL VIEW explode(y) AS element ) > 1
   AND
   ( SELECT SUM(element) FROM LATERAL VIEW explode(y) AS element ) > 3
   I noticed that such subqueries do not work for now. But they align with the language spec and has well defined semantics. Once we support them, we want your proposed rule to be able to speedup them as well.
   
   Ah ok, but what should be the optimized plan of that query? This looks like we have 2 correlated subqueries and (2) makes perfect sense to merge them. But I don't think we need lateral views, just take the following query:
   ```
   SELECT
     (SELECT avg(a) FROM t WHERE t.a = outer.a),
     (SELECT sum(b) FROM t WHERE t.a = outer.a)
   FROM t AS outer
   ```
   which is
   ```
   Project [scalar-subquery#231 [a#233] AS scalarsubquery(a)#243, scalar-subquery#232 [a#233] AS scalarsubquery(a)#244L]
   :  :- Aggregate [avg(a#239) AS avg(a)#236]
   :  :  +- Filter (a#239 = outer(a#233))
   :  :     +- SubqueryAlias spark_catalog.default.t
   :  :        +- Relation default.t[a#239,b#240] parquet
   :  +- Aggregate [sum(b#242) AS sum(b)#238L]
   :     +- Filter (a#241 = outer(a#233))
   :        +- SubqueryAlias spark_catalog.default.t
   :           +- Relation default.t[a#241,b#242] parquet
   +- SubqueryAlias outer
      +- SubqueryAlias spark_catalog.default.t
         +- Relation default.t[a#233,b#234] parquet
   ```
   / 
   ```
   Project [avg(a)#236 AS scalarsubquery(a)#243, sum(b)#238L AS scalarsubquery(a)#244L]
   +- Join LeftOuter, (a#241 = a#233)
      :- Project [a#233, avg(a)#236]
      :  +- Join LeftOuter, (a#239 = a#233)
      :     :- Project [a#233]
      :     :  +- Relation default.t[a#233,b#234] parquet
      :     +- Aggregate [a#239], [avg(a#239) AS avg(a)#236, a#239]
      :        +- Project [a#239]
      :           +- Filter isnotnull(a#239)
      :              +- Relation default.t[a#239,b#240] parquet
      +- Aggregate [a#241], [sum(b#242) AS sum(b)#238L, a#241]
         +- Filter isnotnull(a#241)
            +- Relation default.t[a#241,b#242] parquet
   ```
   now, and this PR doesn't help at all, but it could be optimized using your (2).
   
   I wonder the following steps (tickets/PRs) would make sense:
   
   1. Finish this PR and support only non-correlated subqueries. Mainly focus on merging plans and keep the physical reuse dependency for simplicity. This supports subquery merging within a plan regardless they are in the same logical node.
   2. Add a performance improvement to 1. so as to physical plan a merged subquery only once. This is your (1) basically. Move the merged subqueries to a top node and introduce subquery references in logical plan.
   3. Add support for correlated subqueries using your (2). As you mentioned this will only support subqueries within the same logical node.
   
   Probably we should implement separate rules for 1. + 2. and 3. but the plan merging logic can be common.




-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


   **[Test build #140073 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/140073/testReport)** for PR 32298 at commit [`41c0f0a`](https://github.com/apache/spark/commit/41c0f0aa9a1ab0b19a538e20b586807e39059eec).


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


   @cloud-fan, @maropu, @sigmod, I've updated this PR with the (1) suggestion: https://github.com/apache/spark/pull/32298#discussion_r627800694 and also added tests and comments.


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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






-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


   **[Test build #139121 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/139121/testReport)** for PR 32298 at commit [`100cb9c`](https://github.com/apache/spark/commit/100cb9c9cd8cbf79bcd34c563d02d381cc93dec8).


-- 
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] sigmod commented on a change in pull request #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LeafNode, LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{MULTI_SCALAR_SUBQUERY, SCALAR_SUBQUERY}
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s into a
+ * [[MultiScalarSubquery]] to compute multiple scalar values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ * - The original [[ScalarSubquery]] expression is replaced to a reference pointing to its cached
+ *   version in this form: `GetStructField(MultiScalarSubquery(SubqueryReference(...)))`.
+ * - A second traversal checks if a [[SubqueryReference]] is pointing to a subquery plan that
+ *   returns multiple values and either replaces only [[SubqueryReference]] to the cached plan or
+ *   restores the whole expression to its original [[ScalarSubquery]] form.
+ * - [[ReuseSubquery]] rule makes sure that merged subqueries are computed once.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *   scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet

Review comment:
       Is it possible to rewrite them into a single scalar subquery using struct, e.g.,
   
   SELECT
    (SELECT avg(a) FROM t ),
    (SELECT sum(b) FROM t )
   FROM R
   
   =>
   
   SELECT st.avg_a, st.sum_b
   FROM (
     SELECT (SELECT STRUCT(avg(a) AS avg_a, sum(b) AS sum_b) FROM t) AS st
     FROM R
   )
   
   This way, 
   - (1) you don't need to rely on ReuseExchangeAndSubquery;
   - (2) the rewrite can then work for any subqueries, regardless of uncorrelated v.s. correlated, reading from tables or reading from an array column?
   




-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


   **[Test build #139058 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/139058/testReport)** for PR 32298 at commit [`3cb1f38`](https://github.com/apache/spark/commit/3cb1f38c123b76db539d13c0b94164d51318c8af).


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


   **[Test build #139061 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/139061/testReport)** for PR 32298 at commit [`dca324e`](https://github.com/apache/spark/commit/dca324e173a639952172ff87efab762c665cdb8e).


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


   **[Test build #137809 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137809/testReport)** for PR 32298 at commit [`e0e39d5`](https://github.com/apache/spark/commit/e0e39d56acf0ac7d85461f338543569db022d86d).


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


   **[Test build #140206 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/140206/testReport)** for PR 32298 at commit [`d10a8be`](https://github.com/apache/spark/commit/d10a8bef691e052b85611f75164f7d21ec4ddc55).


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,422 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CommonScalarSubqueries, Filter, Join, LogicalPlan, Project, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s to compute multiple scalar
+ * values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ *   During this first traversal each [[ScalarSubquery]] expression is replaced to a
+ *   [[ScalarSubqueryReference]] pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a results of merging 2 or more
+ *   plans, or it is a plan that was seen only once.
+ *   Merged plans in the cache get a "header", that is is basically
+ *   `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in new root
+ *   [[Project]] node. This expression ensures that the merged plan is a valid scalar subquery that
+ *   returns only one value.
+ * - A second traversal checks if a [[ScalarSubqueryReference]] is pointing to a merged subquery
+ *   plan or not and either keeps the reference or restores the original [[ScalarSubquery]].
+ *   If there are [[ScalarSubqueryReference]] nodes remained a [[CommonScalarSubqueries]] root node
+ *   is added to the plan with the referenced scalar subqueries.
+ * - [[PlanSubqueries]] or [[PlanAdaptiveSubqueries]] rule does the physical planning of scalar
+ *   subqueries including the ones under [[CommonScalarSubqueriesExec]] node and replaces
+ *   each [[ScalarSubqueryReference]] to their referenced physical plan in
+ *   `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() header))` form.
+ *   It is important that references pointing to the same merged subquery are replaced to the same
+ *   planned instance to make sure that each merged subquery runs only once (even without a wrapping
+ *   [[ReuseSubquery]] node).
+ *   Finally, the [[CommonScalarSubqueriesExec]] node is removed from the physical plan.
+ * - The [[ReuseExchangeAndSubquery]] rule wraps the second, third, ... instances of the same
+ *   subquery into a [[ReuseSubquery]] node, but this just a cosmetic change in the plan.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *          scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * CommonScalarSubqueries [scalar-subquery#250 []]
+ * :  +- Project [named_struct(avg(a), avg(a)#236, sum(b), sum(b)#238L) AS mergedValue#249]
+ * :     +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :        +- Project [a#233, b#234]
+ * :           +- Relation default.t[a#233,b#234] parquet
+ * +- Project [scalarsubqueryreference(0, 0, DoubleType, 231) AS scalarsubquery()#241,
+ *             scalarsubqueryreference(0, 1, LongType, 232) AS scalarsubquery()#242L]
+ *    +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    plan match {
+      case Subquery(_: CommonScalarSubqueries, _) => plan
+      case s: Subquery => s.copy(child = extractCommonScalarSubqueries(s.child))
+      case _: CommonScalarSubqueries => plan
+      case _ => extractCommonScalarSubqueries(plan)
+    }
+  }
+
+  /**
+   * An item in the cache of merged scalar subqueries.
+   *
+   * @param elements  List of attributes that form the scalar return value of a merged subquery
+   * @param plan      The plan of a merged scalar subquery
+   * @param merged    A flag to identify if this item is the result of merging subqueries.
+   *                  Please note that `elements.size == 1` doesn't always mean that the plan is not
+   *                  merged as there can be subqueries that are different ([[checkIdenticalPlans]]
+   *                  is false) due to an extra [[Project]] node in one of them. In that case
+   *                  `elements.size` remains 1 after merging, but the merged flag becomes true.
+   */
+  case class Header(elements: Seq[(String, Attribute)], plan: LogicalPlan, merged: Boolean)
+
+  private def extractCommonScalarSubqueries(plan: LogicalPlan) = {
+    val cache = ListBuffer.empty[Header]
+    val newPlan = removeReferences(insertReferences(plan, cache), cache)
+    if (cache.nonEmpty) {
+      val scalarSubqueries = cache.map {
+        case Header(elements, child, _) => ScalarSubquery(createProject(elements, child))
+      }.toSeq
+      CommonScalarSubqueries(scalarSubqueries, newPlan)
+    } else {
+      newPlan
+    }
+  }
+
+  // First traversal builds up the cache and inserts `ScalarSubqueryReference`s to the plan.
+  private def insertReferences(plan: LogicalPlan, cache: ListBuffer[Header]): LogicalPlan = {
+    plan.transformAllExpressionsWithPruning(_.containsAnyPattern(SCALAR_SUBQUERY)) {
+      case s: ScalarSubquery if s.children.isEmpty =>
+        val (subqueryIndex, headerIndex) = cacheSubquery(s.plan, cache)
+        ScalarSubqueryReference(subqueryIndex, headerIndex, s.dataType, s.exprId)
+    }
+  }
+
+  // Caching returns the index of the subquery in the cache and the index of scalar member in the
+  // "header", that `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in a
+  // [[Project]] node.
+  private def cacheSubquery(plan: LogicalPlan, cache: ListBuffer[Header]): (Int, Int) = {
+    val output = plan.output.head
+    cache.zipWithIndex.collectFirst(Function.unlift { case (header, subqueryIndex) =>

Review comment:
       If `cache.zipWithIndex.collectFirst(Function.unlift {` is hard to read I can covert it to a loop.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


   @cloud-fan, @maropu, @sigmod, I wonder how we can move forward with 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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






-- 
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] maryannxue edited a comment on pull request #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

Posted by GitBox <gi...@apache.org>.
maryannxue edited a comment on pull request #32298:
URL: https://github.com/apache/spark/pull/32298#issuecomment-890115306


   @peter-toth I'm going to push a PR sometime soon to defer CTE inlining to after query analysis and avoid inlining of non-deterministic CTEs. It'll include some new CTE nodes that might be useful for your PR here.


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LeafNode, LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{MULTI_SCALAR_SUBQUERY, SCALAR_SUBQUERY}
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s into a
+ * [[MultiScalarSubquery]] to compute multiple scalar values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ * - The original [[ScalarSubquery]] expression is replaced to a reference pointing to its cached
+ *   version in this form: `GetStructField(MultiScalarSubquery(SubqueryReference(...)))`.
+ * - A second traversal checks if a [[SubqueryReference]] is pointing to a subquery plan that
+ *   returns multiple values and either replaces only [[SubqueryReference]] to the cached plan or
+ *   restores the whole expression to its original [[ScalarSubquery]] form.
+ * - [[ReuseSubquery]] rule makes sure that merged subqueries are computed once.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *   scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet

Review comment:
       > The proposed rule augments two subqueries, makes them look identical, and hopes (a) column-pruning doesn't prune too aggressively and (b) physical de-dup could dedup them. In case (a) changes later and the two aggregate trees are not deduped in the physical plan, there could potentially be regressions -- each aggregation then becomes more expensive.
   
   In this PR the new `MergeScalarSubqueries` rule runs in a separate batch after column pruning, close to the end of optimization. This is by design to make sure no subsequent rule changes the structure of different instances of a merged subquery plan at different places in the logical plan differently. So the physical planing creates the same physical plan for these instances and there shouldn't be any dedup issues.
   **Update: I need to recheck this part as the current PR might not do what I wanted to.**
   
   I think probably the downside of my current PR is that the physical planning of merged subqueries happen multiple times (as many times as they they appear in the logical plan) and physical dedup comes only after that. This could be improved if we had subquery references in logical plan as well (something like `ReuseSubqueryExec`). But I think that's what your (1) is about. Move the merged subqueries to a special top logical plan node and add subquery references at places where they are actually used.
   
   > SELECT y
   FROM LATERAL VIEW explode(ARRAY(ARRAY(1), ARRAY(1, 2), ARRAY(1, 2, 3))) AS y
   WHERE
   ( SELECT COUNT(*) FROM LATERAL VIEW explode(y) AS element ) > 1
   AND
   ( SELECT SUM(element) FROM LATERAL VIEW explode(y) AS element ) > 3
   I noticed that such subqueries do not work for now. But they align with the language spec and has well defined semantics. Once we support them, we want your proposed rule to be able to speedup them as well.
   
   Ah ok, but what should be the optimized plan of that query? This looks like we have 2 correlated subqueries and (2) makes perfect sense to merge them. But I don't think we need lateral views, just take the following query:
   ```
   SELECT
     (SELECT avg(a) FROM t WHERE t.a = outer.a),
     (SELECT sum(b) FROM t WHERE t.a = outer.a)
   FROM t AS outer
   ```
   which is
   ```
   Project [scalar-subquery#231 [a#233] AS scalarsubquery(a)#243, scalar-subquery#232 [a#233] AS scalarsubquery(a)#244L]
   :  :- Aggregate [avg(a#239) AS avg(a)#236]
   :  :  +- Filter (a#239 = outer(a#233))
   :  :     +- SubqueryAlias spark_catalog.default.t
   :  :        +- Relation default.t[a#239,b#240] parquet
   :  +- Aggregate [sum(b#242) AS sum(b)#238L]
   :     +- Filter (a#241 = outer(a#233))
   :        +- SubqueryAlias spark_catalog.default.t
   :           +- Relation default.t[a#241,b#242] parquet
   +- SubqueryAlias outer
      +- SubqueryAlias spark_catalog.default.t
         +- Relation default.t[a#233,b#234] parquet
   ```
   / 
   ```
   Project [avg(a)#236 AS scalarsubquery(a)#243, sum(b)#238L AS scalarsubquery(a)#244L]
   +- Join LeftOuter, (a#241 = a#233)
      :- Project [a#233, avg(a)#236]
      :  +- Join LeftOuter, (a#239 = a#233)
      :     :- Project [a#233]
      :     :  +- Relation default.t[a#233,b#234] parquet
      :     +- Aggregate [a#239], [avg(a#239) AS avg(a)#236, a#239]
      :        +- Project [a#239]
      :           +- Filter isnotnull(a#239)
      :              +- Relation default.t[a#239,b#240] parquet
      +- Aggregate [a#241], [sum(b#242) AS sum(b)#238L, a#241]
         +- Filter isnotnull(a#241)
            +- Relation default.t[a#241,b#242] parquet
   ```
   now, and this PR doesn't help at all, but it could be optimized using your (2).
   
   I wonder the following steps (tickets/PRs) would make sense:
   
   1. Finish this PR and support only non-correlated subqueries. Mainly focus on merging plans and keep the physical reuse dependency for simplicity. This supports subquery merging within a plan regardless they are in the same logical node.
   2. Add a performance improvement to 1. so as to physical plan a merged subquery only once. This is your (1) basically. Move the merged subqueries to a top node and introduce subquery references in logical plan.
   3. Add support for correlated subqueries using your (2). As you mentioned this will only support subqueries within the same logical node.
   
   Probably we should implement separate rules for 1. + 2. and 3. but the plan merging logic can be common.




-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


   **[Test build #145822 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/145822/testReport)** for PR 32298 at commit [`3e8f7fa`](https://github.com/apache/spark/commit/3e8f7faa79272d2149cb29ce62700b942c83d864).
    * 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


   @tgravescs, thanks for your interesting in in this PR. I reran the tests, they seem to be fine now.
   This change adds basic query merging logic and affects only `q9` from TPCDS, but it brings considerable performance improvement to it.
   This PR has been ready for review for a while, but haven't got much attention. Your and others review would be much appreciated.
   
   I also have some follow-up PRs/plans:
   - to add more advanced logic to support merging aggregate subqueries with different `Filter` nodes
   - support DSv2
   - try merging correlated subqueries
   
   but this PR is complex enough so I would add these in subsequent PRs only.
   
   We can target 3.3, but AFAIK the branch has been cut so probably we need some kind of approval on the dev list?
   


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] tgravescs commented on a change in pull request #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,422 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CommonScalarSubqueries, Filter, Join, LogicalPlan, Project, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s to compute multiple scalar
+ * values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ *   During this first traversal each [[ScalarSubquery]] expression is replaced to a
+ *   [[ScalarSubqueryReference]] pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a results of merging 2 or more
+ *   plans, or it is a plan that was seen only once.
+ *   Merged plans in the cache get a "header", that is is basically
+ *   `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in new root
+ *   [[Project]] node. This expression ensures that the merged plan is a valid scalar subquery that
+ *   returns only one value.
+ * - A second traversal checks if a [[ScalarSubqueryReference]] is pointing to a merged subquery
+ *   plan or not and either keeps the reference or restores the original [[ScalarSubquery]].
+ *   If there are [[ScalarSubqueryReference]] nodes remained a [[CommonScalarSubqueries]] root node
+ *   is added to the plan with the referenced scalar subqueries.
+ * - [[PlanSubqueries]] or [[PlanAdaptiveSubqueries]] rule does the physical planning of scalar
+ *   subqueries including the ones under [[CommonScalarSubqueriesExec]] node and replaces
+ *   each [[ScalarSubqueryReference]] to their referenced physical plan in
+ *   `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() header))` form.
+ *   It is important that references pointing to the same merged subquery are replaced to the same
+ *   planned instance to make sure that each merged subquery runs only once (even without a wrapping
+ *   [[ReuseSubquery]] node).
+ *   Finally, the [[CommonScalarSubqueriesExec]] node is removed from the physical plan.
+ * - The [[ReuseExchangeAndSubquery]] rule wraps the second, third, ... instances of the same
+ *   subquery into a [[ReuseSubquery]] node, but this just a cosmetic change in the plan.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *          scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * CommonScalarSubqueries [scalar-subquery#250 []]
+ * :  +- Project [named_struct(avg(a), avg(a)#236, sum(b), sum(b)#238L) AS mergedValue#249]
+ * :     +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :        +- Project [a#233, b#234]
+ * :           +- Relation default.t[a#233,b#234] parquet
+ * +- Project [scalarsubqueryreference(0, 0, DoubleType, 231) AS scalarsubquery()#241,
+ *             scalarsubqueryreference(0, 1, LongType, 232) AS scalarsubquery()#242L]
+ *    +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    plan match {
+      case Subquery(_: CommonScalarSubqueries, _) => plan
+      case s: Subquery => s.copy(child = extractCommonScalarSubqueries(s.child))
+      case _: CommonScalarSubqueries => plan
+      case _ => extractCommonScalarSubqueries(plan)
+    }
+  }
+
+  /**
+   * An item in the cache of merged scalar subqueries.
+   *
+   * @param elements  List of attributes that form the scalar return value of a merged subquery
+   * @param plan      The plan of a merged scalar subquery
+   * @param merged    A flag to identify if this item is the result of merging subqueries.
+   *                  Please note that `elements.size == 1` doesn't always mean that the plan is not
+   *                  merged as there can be subqueries that are different ([[checkIdenticalPlans]]
+   *                  is false) due to an extra [[Project]] node in one of them. In that case
+   *                  `elements.size` remains 1 after merging, but the merged flag becomes true.
+   */
+  case class Header(elements: Seq[(String, Attribute)], plan: LogicalPlan, merged: Boolean)
+
+  private def extractCommonScalarSubqueries(plan: LogicalPlan) = {
+    val cache = ListBuffer.empty[Header]
+    val newPlan = removeReferences(insertReferences(plan, cache), cache)
+    if (cache.nonEmpty) {
+      val scalarSubqueries = cache.map {
+        case Header(elements, child, _) => ScalarSubquery(createProject(elements, child))
+      }.toSeq
+      CommonScalarSubqueries(scalarSubqueries, newPlan)
+    } else {
+      newPlan
+    }
+  }
+
+  // First traversal builds up the cache and inserts `ScalarSubqueryReference`s to the plan.
+  private def insertReferences(plan: LogicalPlan, cache: ListBuffer[Header]): LogicalPlan = {
+    plan.transformAllExpressionsWithPruning(_.containsAnyPattern(SCALAR_SUBQUERY)) {
+      case s: ScalarSubquery if s.children.isEmpty =>
+        val (subqueryIndex, headerIndex) = cacheSubquery(s.plan, cache)
+        ScalarSubqueryReference(subqueryIndex, headerIndex, s.dataType, s.exprId)
+    }
+  }
+
+  // Caching returns the index of the subquery in the cache and the index of scalar member in the
+  // "header", that `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in a
+  // [[Project]] node.
+  private def cacheSubquery(plan: LogicalPlan, cache: ListBuffer[Header]): (Int, Int) = {
+    val output = plan.output.head
+    cache.zipWithIndex.collectFirst(Function.unlift { case (header, subqueryIndex) =>

Review comment:
       it does take a second to figure out but I'm ok with it.

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,421 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CommonScalarSubqueries, Filter, Join, LogicalPlan, Project, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s to compute multiple scalar
+ * values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ *   During this first traversal each [[ScalarSubquery]] expression is replaced to a
+ *   [[ScalarSubqueryReference]] pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a results of merging 2 or more
+ *   plans, or it is a plan that was seen only once.
+ *   Merged plans in the cache get a "header", that is is basically
+ *   `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in new root
+ *   [[Project]] node. This expression ensures that the merged plan is a valid scalar subquery that
+ *   returns only one value.
+ * - A second traversal checks if a [[ScalarSubqueryReference]] is pointing to a merged subquery
+ *   plan or not and either keeps the reference or restores the original [[ScalarSubquery]].
+ *   If there are [[ScalarSubqueryReference]] nodes remained a [[CommonScalarSubqueries]] root node
+ *   is added to the plan with the referenced scalar subqueries.
+ * - [[PlanSubqueries]] or [[PlanAdaptiveSubqueries]] rule does the physical planning of scalar
+ *   subqueries including the ones under [[CommonScalarSubqueriesExec]] node and replaces
+ *   each [[ScalarSubqueryReference]] to their referenced physical plan in
+ *   `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() header))` form.
+ *   It is important that references pointing to the same merged subquery are replaced to the same
+ *   planned instance to make sure that each merged subquery runs only once (even without a wrapping
+ *   [[ReuseSubquery]] node).
+ *   Finally, the [[CommonScalarSubqueriesExec]] node is removed from the physical plan.
+ * - The [[ReuseExchangeAndSubquery]] rule wraps the second, third, ... instances of the same
+ *   subquery into a [[ReuseSubquery]] node, but this just a cosmetic change in the plan.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *          scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * CommonScalarSubqueries [scalar-subquery#250 []]
+ * :  +- Project [named_struct(avg(a), avg(a)#236, sum(b), sum(b)#238L) AS mergedValue#249]
+ * :     +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :        +- Project [a#233, b#234]
+ * :           +- Relation default.t[a#233,b#234] parquet
+ * +- Project [scalarsubqueryreference(0, 0, DoubleType, 231) AS scalarsubquery()#241,
+ *             scalarsubqueryreference(0, 1, LongType, 232) AS scalarsubquery()#242L]
+ *    +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    plan match {
+      case Subquery(_: CommonScalarSubqueries, _) => plan
+      case s: Subquery => s.copy(child = extractCommonScalarSubqueries(s.child))
+      case _: CommonScalarSubqueries => plan
+      case _ => extractCommonScalarSubqueries(plan)
+    }
+  }
+
+  /**
+   * An item in the cache of merged scalar subqueries.
+   *
+   * @param elements  List of attributes that form the scalar return value of a merged subquery
+   * @param plan      The plan of a merged scalar subquery
+   * @param merged    A flag to identify if this item is the result of merging subqueries.
+   *                  Please note that `elements.size == 1` doesn't always mean that the plan is not
+   *                  merged as there can be subqueries that are different ([[checkIdenticalPlans]]
+   *                  is false) due to an extra [[Project]] node in one of them. In that case
+   *                  `elements.size` remains 1 after merging, but the merged flag becomes true.
+   */
+  case class Header(elements: Seq[(String, Attribute)], plan: LogicalPlan, merged: Boolean)
+
+  private def extractCommonScalarSubqueries(plan: LogicalPlan) = {
+    val cache = ListBuffer.empty[Header]
+    val newPlan = removeReferences(insertReferences(plan, cache), cache)
+    if (cache.nonEmpty) {
+      val scalarSubqueries = cache.map {
+        case Header(elements, child, _) => ScalarSubquery(createProject(elements, child))
+      }.toSeq
+      CommonScalarSubqueries(scalarSubqueries, newPlan)
+    } else {
+      newPlan
+    }
+  }
+
+  // First traversal builds up the cache and inserts `ScalarSubqueryReference`s to the plan.
+  private def insertReferences(plan: LogicalPlan, cache: ListBuffer[Header]): LogicalPlan = {
+    plan.transformAllExpressionsWithPruning(_.containsAnyPattern(SCALAR_SUBQUERY)) {
+      case s: ScalarSubquery if !s.isCorrelated && s.deterministic =>
+        val (subqueryIndex, headerIndex) = cacheSubquery(s.plan, cache)
+        ScalarSubqueryReference(subqueryIndex, headerIndex, s.dataType, s.exprId)
+    }
+  }
+
+  // Caching returns the index of the subquery in the cache and the index of scalar member in the
+  // "header", that `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in a
+  // [[Project]] node.
+  private def cacheSubquery(plan: LogicalPlan, cache: ListBuffer[Header]): (Int, Int) = {
+    val output = plan.output.head
+    cache.zipWithIndex.collectFirst(Function.unlift { case (header, subqueryIndex) =>
+      checkIdenticalPlans(plan, header.plan).map { outputMap =>
+        val mappedOutput = mapAttributes(output, outputMap)
+        val headerIndex = header.elements.indexWhere {
+          case (_, attribute) => attribute.exprId == mappedOutput.exprId
+        }
+        subqueryIndex -> headerIndex
+      }.orElse(tryMergePlans(plan, header.plan).map {
+        case (mergedPlan, outputMap) =>
+          val mappedOutput = mapAttributes(output, outputMap)
+          var headerIndex = header.elements.indexWhere {
+            case (_, attribute) => attribute.exprId == mappedOutput.exprId
+          }
+          val newHeaderElements = if (headerIndex == -1) {
+            headerIndex = header.elements.size
+            header.elements :+ (output.name -> mappedOutput)
+          } else {
+            header.elements
+          }
+          cache(subqueryIndex) = Header(newHeaderElements, mergedPlan, true)
+          subqueryIndex -> headerIndex
+      })
+    }).getOrElse {
+      cache += Header(Seq(output.name -> output), plan, false)
+      cache.length - 1 -> 0
+    }
+  }
+
+  // If 2 plans are identical return the attribute mapping from the new to the cached version.
+  private def checkIdenticalPlans(
+      newPlan: LogicalPlan,
+      cachedPlan: LogicalPlan): Option[AttributeMap[Attribute]] = {
+    if (newPlan.canonicalized == cachedPlan.canonicalized) {
+      Some(AttributeMap(newPlan.output.zip(cachedPlan.output)))
+    } else {
+      None
+    }
+  }
+
+  // Recursively traverse down and try merging 2 plans. If merge is possible then return the merged
+  // plan with the attribute mapping from the new to the merged version.
+  // Please note that merging arbitrary plans can be complicated, the current version supports only
+  // some of the most important nodes.
+  private def tryMergePlans(
+      newPlan: LogicalPlan,
+      cachedPlan: LogicalPlan): Option[(LogicalPlan, AttributeMap[Attribute])] = {
+    checkIdenticalPlans(newPlan, cachedPlan).map(cachedPlan -> _).orElse(
+      (newPlan, cachedPlan) match {
+        case (np: Project, cp: Project) =>
+          tryMergePlans(np.child, cp.child).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.projectList, outputMap, cp.projectList)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np, cp: Project) =>
+          tryMergePlans(np, cp.child).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.output, outputMap, cp.projectList)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np: Project, cp) =>
+          tryMergePlans(np.child, cp).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.projectList, outputMap, cp.output)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np: Aggregate, cp: Aggregate) if supportedAggregateMerge(np, cp) =>
+          tryMergePlans(np.child, cp.child).flatMap { case (mergedChild, outputMap) =>
+            val mappedNewGroupingExpression =
+              np.groupingExpressions.map(mapAttributes(_, outputMap))
+            // Order of grouping expression doesn't matter so we can compare sets
+            if (ExpressionSet(mappedNewGroupingExpression) ==
+              ExpressionSet(cp.groupingExpressions)) {
+              val (mergedAggregateExpressions, newOutputMap) =
+                mergeNamedExpressions(np.aggregateExpressions, outputMap, cp.aggregateExpressions)
+              val mergedPlan =
+                Aggregate(cp.groupingExpressions, mergedAggregateExpressions, mergedChild)
+              Some(mergedPlan -> newOutputMap)
+            } else {
+              None
+            }
+          }
+
+        // Merging general nodes is complicated and this implementation supports only those nodes in
+        // which the order and the number of output attributes are not relevant (see
+        // `supportedMerge()` whitelist).
+        // Also, this implementation supports only those nodes in which children can be merged in
+        // the same order.
+        case (np, cp) if supportedMerge(np) && np.getClass == cp.getClass &&
+            np.children.size == cp.children.size &&
+            np.expressions.size == cp.expressions.size &&
+            // Fields that don't contain any children or expressions should match
+            np.productIterator.filterNot(np.children.contains)
+              .filter(QueryPlan.extractExpressions(_).isEmpty).toSeq ==
+              cp.productIterator.filterNot(cp.children.contains)
+                .filter(QueryPlan.extractExpressions(_).isEmpty).toSeq =>
+          val merged = np.children.zip(cp.children).map {
+            case (npChild, cpChild) => tryMergePlans(npChild, cpChild)
+          }
+          if (merged.forall(_.isDefined)) {
+            val (mergedChildren, outputMaps) = merged.map(_.get).unzip
+            val outputMap = AttributeMap(outputMaps.map(_.iterator).reduce(_ ++ _).toSeq)
+            // We know that fields that don't contain any children or expressions do match and
+            // children can be merged so we need to test expressions only
+            if (np.expressions.map(mapAttributes(_, outputMap).canonicalized) ==
+              cp.expressions.map(_.canonicalized)) {
+              val mergedPlan = cp.withNewChildren(mergedChildren)
+              Some(mergedPlan -> outputMap)
+            } else {
+              None
+            }
+          } else {
+            None
+          }
+
+        // As a follow-up, it would be possible to merge `CommonScalarSubqueries` nodes, which would
+        // allow merging subqueries with mergee subqueries.
+        // E.g. this query:
+        //
+        // SELECT
+        //   (
+        //     SELECT
+        //       (SELECT avg(a) FROM t GROUP BY b) +
+        //       (SELECT sum(b) FROM t GROUP BY b)
+        //   ),
+        //   (
+        //     SELECT
+        //       (SELECT max(a) FROM t GROUP BY b) +
+        //       (SELECT min(b) FROM t GROUP BY b)
+        //   )
+        //
+        // is currently optimized to:
+        //
+        // == Optimized Logical Plan ==
+        // Project [scalar-subquery#233 [] AS scalarsubquery()#255,
+        //          scalar-subquery#236 [] AS scalarsubquery()#256]
+        // :  :- CommonScalarSubqueries [scalar-subquery#264 []]
+        // :  :  :  +- Aggregate [b#238], [named_struct(avg(a), avg(a#237), sum(b), sum(b#238))
+        //                                 AS mergedValue#263]
+        // :  :  :     +- Relation default.t[a#237,b#238] parquet
+        // :  :  +- Project [(scalarsubqueryreference(0, 0, DoubleType, 231) +
+        //                   cast(scalarsubqueryreference(0, 1, LongType, 232) as double))
+        //                   AS (scalarsubquery() + scalarsubquery())#245]
+        // :  :     +- OneRowRelation
+        // :  +- CommonScalarSubqueries [scalar-subquery#269 []]
+        // :     :  +- Aggregate [b#254], [named_struct(min(a), min(a#253), max(b), max(b#254))
+        //                                 AS mergedValue#268]
+        // :     :     +- Relation default.t[a#253,b#254] parquet
+        // :     +- Project [(scalarsubqueryreference(0, 0, IntegerType, 234) +
+        //                   scalarsubqueryreference(0, 1, IntegerType, 235))
+        //                   AS (scalarsubquery() + scalarsubquery())#252]
+        // :        +- OneRowRelation
+        // +- OneRowRelation
+        //
+        // but if we implemented merging `CommonScalarSubqueries` nodes then the plan could be
+        // transformed further and all leaf subqueries could be merged.
+
+        // Otherwise merging is not possible.
+        case _ => None
+      })
+  }
+
+  private def createProject(elements: Seq[(String, Attribute)], plan: LogicalPlan): Project = {
+    Project(
+      Seq(Alias(
+        CreateNamedStruct(elements.flatMap {
+          case (name, attribute) => Seq(Literal(name), attribute)
+        }),
+        "mergedValue")()),
+      plan)
+  }
+
+  private def mapAttributes[T <: Expression](expr: T, outputMap: AttributeMap[Attribute]) = {
+    expr.transform {
+      case a: Attribute => outputMap.getOrElse(a, a)
+    }.asInstanceOf[T]
+  }
+
+  // Applies `outputMap` attribute mapping on elements of `newExpressions` and merges them into
+  // `cachedExpressions`. Returns the merged expressions and the attribute mapping from the new to
+  // the merged version that can be propagated up during merging nodes.
+  private def mergeNamedExpressions(
+      newExpressions: Seq[NamedExpression],
+      outputMap: AttributeMap[Attribute],
+      cachedExpressions: Seq[NamedExpression]) = {
+    val mergedExpressions = ListBuffer[NamedExpression](cachedExpressions: _*)
+    val newOutputMap = AttributeMap(newExpressions.map { ne =>
+      val mapped = mapAttributes(ne, outputMap)
+      val withoutAlias = mapped match {
+        case Alias(child, _) => child
+        case e => e
+      }
+      ne.toAttribute -> mergedExpressions.find {
+        case Alias(child, _) => child semanticEquals withoutAlias
+        case e => e semanticEquals withoutAlias
+      }.getOrElse {
+        mergedExpressions += mapped
+        mapped
+      }.toAttribute
+    })
+    (mergedExpressions.toSeq, newOutputMap)
+  }
+
+  // Merging different aggregate implementations could cause performance regression

Review comment:
       nit - perhaps enhance by just saying "Only allow aggregates of the same implementation because merging different...

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,421 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CommonScalarSubqueries, Filter, Join, LogicalPlan, Project, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s to compute multiple scalar
+ * values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ *   During this first traversal each [[ScalarSubquery]] expression is replaced to a
+ *   [[ScalarSubqueryReference]] pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a results of merging 2 or more
+ *   plans, or it is a plan that was seen only once.
+ *   Merged plans in the cache get a "header", that is is basically
+ *   `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in new root
+ *   [[Project]] node. This expression ensures that the merged plan is a valid scalar subquery that
+ *   returns only one value.
+ * - A second traversal checks if a [[ScalarSubqueryReference]] is pointing to a merged subquery
+ *   plan or not and either keeps the reference or restores the original [[ScalarSubquery]].
+ *   If there are [[ScalarSubqueryReference]] nodes remained a [[CommonScalarSubqueries]] root node
+ *   is added to the plan with the referenced scalar subqueries.
+ * - [[PlanSubqueries]] or [[PlanAdaptiveSubqueries]] rule does the physical planning of scalar
+ *   subqueries including the ones under [[CommonScalarSubqueriesExec]] node and replaces
+ *   each [[ScalarSubqueryReference]] to their referenced physical plan in
+ *   `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() header))` form.
+ *   It is important that references pointing to the same merged subquery are replaced to the same
+ *   planned instance to make sure that each merged subquery runs only once (even without a wrapping
+ *   [[ReuseSubquery]] node).
+ *   Finally, the [[CommonScalarSubqueriesExec]] node is removed from the physical plan.
+ * - The [[ReuseExchangeAndSubquery]] rule wraps the second, third, ... instances of the same
+ *   subquery into a [[ReuseSubquery]] node, but this just a cosmetic change in the plan.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *          scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * CommonScalarSubqueries [scalar-subquery#250 []]
+ * :  +- Project [named_struct(avg(a), avg(a)#236, sum(b), sum(b)#238L) AS mergedValue#249]
+ * :     +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :        +- Project [a#233, b#234]
+ * :           +- Relation default.t[a#233,b#234] parquet
+ * +- Project [scalarsubqueryreference(0, 0, DoubleType, 231) AS scalarsubquery()#241,
+ *             scalarsubqueryreference(0, 1, LongType, 232) AS scalarsubquery()#242L]
+ *    +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    plan match {
+      case Subquery(_: CommonScalarSubqueries, _) => plan
+      case s: Subquery => s.copy(child = extractCommonScalarSubqueries(s.child))
+      case _: CommonScalarSubqueries => plan
+      case _ => extractCommonScalarSubqueries(plan)
+    }
+  }
+
+  /**
+   * An item in the cache of merged scalar subqueries.
+   *
+   * @param elements  List of attributes that form the scalar return value of a merged subquery
+   * @param plan      The plan of a merged scalar subquery
+   * @param merged    A flag to identify if this item is the result of merging subqueries.
+   *                  Please note that `elements.size == 1` doesn't always mean that the plan is not
+   *                  merged as there can be subqueries that are different ([[checkIdenticalPlans]]
+   *                  is false) due to an extra [[Project]] node in one of them. In that case
+   *                  `elements.size` remains 1 after merging, but the merged flag becomes true.
+   */
+  case class Header(elements: Seq[(String, Attribute)], plan: LogicalPlan, merged: Boolean)
+
+  private def extractCommonScalarSubqueries(plan: LogicalPlan) = {
+    val cache = ListBuffer.empty[Header]
+    val newPlan = removeReferences(insertReferences(plan, cache), cache)
+    if (cache.nonEmpty) {
+      val scalarSubqueries = cache.map {
+        case Header(elements, child, _) => ScalarSubquery(createProject(elements, child))
+      }.toSeq
+      CommonScalarSubqueries(scalarSubqueries, newPlan)
+    } else {
+      newPlan
+    }
+  }
+
+  // First traversal builds up the cache and inserts `ScalarSubqueryReference`s to the plan.
+  private def insertReferences(plan: LogicalPlan, cache: ListBuffer[Header]): LogicalPlan = {
+    plan.transformAllExpressionsWithPruning(_.containsAnyPattern(SCALAR_SUBQUERY)) {
+      case s: ScalarSubquery if !s.isCorrelated && s.deterministic =>
+        val (subqueryIndex, headerIndex) = cacheSubquery(s.plan, cache)
+        ScalarSubqueryReference(subqueryIndex, headerIndex, s.dataType, s.exprId)
+    }
+  }
+
+  // Caching returns the index of the subquery in the cache and the index of scalar member in the
+  // "header", that `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in a
+  // [[Project]] node.
+  private def cacheSubquery(plan: LogicalPlan, cache: ListBuffer[Header]): (Int, Int) = {
+    val output = plan.output.head
+    cache.zipWithIndex.collectFirst(Function.unlift { case (header, subqueryIndex) =>
+      checkIdenticalPlans(plan, header.plan).map { outputMap =>
+        val mappedOutput = mapAttributes(output, outputMap)
+        val headerIndex = header.elements.indexWhere {
+          case (_, attribute) => attribute.exprId == mappedOutput.exprId
+        }
+        subqueryIndex -> headerIndex
+      }.orElse(tryMergePlans(plan, header.plan).map {
+        case (mergedPlan, outputMap) =>
+          val mappedOutput = mapAttributes(output, outputMap)
+          var headerIndex = header.elements.indexWhere {
+            case (_, attribute) => attribute.exprId == mappedOutput.exprId
+          }
+          val newHeaderElements = if (headerIndex == -1) {
+            headerIndex = header.elements.size
+            header.elements :+ (output.name -> mappedOutput)
+          } else {
+            header.elements
+          }
+          cache(subqueryIndex) = Header(newHeaderElements, mergedPlan, true)
+          subqueryIndex -> headerIndex
+      })
+    }).getOrElse {
+      cache += Header(Seq(output.name -> output), plan, false)
+      cache.length - 1 -> 0
+    }
+  }
+
+  // If 2 plans are identical return the attribute mapping from the new to the cached version.
+  private def checkIdenticalPlans(
+      newPlan: LogicalPlan,
+      cachedPlan: LogicalPlan): Option[AttributeMap[Attribute]] = {
+    if (newPlan.canonicalized == cachedPlan.canonicalized) {
+      Some(AttributeMap(newPlan.output.zip(cachedPlan.output)))
+    } else {
+      None
+    }
+  }
+
+  // Recursively traverse down and try merging 2 plans. If merge is possible then return the merged
+  // plan with the attribute mapping from the new to the merged version.
+  // Please note that merging arbitrary plans can be complicated, the current version supports only
+  // some of the most important nodes.
+  private def tryMergePlans(
+      newPlan: LogicalPlan,
+      cachedPlan: LogicalPlan): Option[(LogicalPlan, AttributeMap[Attribute])] = {
+    checkIdenticalPlans(newPlan, cachedPlan).map(cachedPlan -> _).orElse(
+      (newPlan, cachedPlan) match {
+        case (np: Project, cp: Project) =>
+          tryMergePlans(np.child, cp.child).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.projectList, outputMap, cp.projectList)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np, cp: Project) =>
+          tryMergePlans(np, cp.child).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.output, outputMap, cp.projectList)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np: Project, cp) =>
+          tryMergePlans(np.child, cp).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.projectList, outputMap, cp.output)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np: Aggregate, cp: Aggregate) if supportedAggregateMerge(np, cp) =>
+          tryMergePlans(np.child, cp.child).flatMap { case (mergedChild, outputMap) =>
+            val mappedNewGroupingExpression =
+              np.groupingExpressions.map(mapAttributes(_, outputMap))
+            // Order of grouping expression doesn't matter so we can compare sets
+            if (ExpressionSet(mappedNewGroupingExpression) ==
+              ExpressionSet(cp.groupingExpressions)) {
+              val (mergedAggregateExpressions, newOutputMap) =
+                mergeNamedExpressions(np.aggregateExpressions, outputMap, cp.aggregateExpressions)
+              val mergedPlan =
+                Aggregate(cp.groupingExpressions, mergedAggregateExpressions, mergedChild)
+              Some(mergedPlan -> newOutputMap)
+            } else {
+              None
+            }
+          }
+
+        // Merging general nodes is complicated and this implementation supports only those nodes in
+        // which the order and the number of output attributes are not relevant (see
+        // `supportedMerge()` whitelist).
+        // Also, this implementation supports only those nodes in which children can be merged in
+        // the same order.
+        case (np, cp) if supportedMerge(np) && np.getClass == cp.getClass &&
+            np.children.size == cp.children.size &&
+            np.expressions.size == cp.expressions.size &&
+            // Fields that don't contain any children or expressions should match
+            np.productIterator.filterNot(np.children.contains)
+              .filter(QueryPlan.extractExpressions(_).isEmpty).toSeq ==
+              cp.productIterator.filterNot(cp.children.contains)
+                .filter(QueryPlan.extractExpressions(_).isEmpty).toSeq =>
+          val merged = np.children.zip(cp.children).map {
+            case (npChild, cpChild) => tryMergePlans(npChild, cpChild)
+          }
+          if (merged.forall(_.isDefined)) {
+            val (mergedChildren, outputMaps) = merged.map(_.get).unzip
+            val outputMap = AttributeMap(outputMaps.map(_.iterator).reduce(_ ++ _).toSeq)
+            // We know that fields that don't contain any children or expressions do match and
+            // children can be merged so we need to test expressions only
+            if (np.expressions.map(mapAttributes(_, outputMap).canonicalized) ==
+              cp.expressions.map(_.canonicalized)) {
+              val mergedPlan = cp.withNewChildren(mergedChildren)
+              Some(mergedPlan -> outputMap)
+            } else {
+              None
+            }
+          } else {
+            None
+          }
+
+        // As a follow-up, it would be possible to merge `CommonScalarSubqueries` nodes, which would
+        // allow merging subqueries with mergee subqueries.

Review comment:
       nit "mergee" mispelled




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


   > * (2) however, `CTERef` is also a primitive for de-duplicate common plan subtrees. Those plan trees to be shared do not have to be identical, e.g., one can merge filter predicates with `OR` and union needed columns into a single, shared CTE definition. Other query engines do that, even though Spark doesn't do that for now. E.g., this paper describes such optimizations: http://www.vldb.org/pvldb/vol8/p1704-elhelw.pdf.
   > * (3) I think what this PR does is a special case of (2). E.g., if you have two plan subtrees (within the same query plan, but not subqueries) run different aggregations over the same table with the same grouping exprs, we can use `CTERef` but not `CommonSubqueries` to share the scan and computation.
   
   In think there are 2 different things here:
   - The merging logic (`tryMergePlans()`) in this PR is general enough to handle common plan subtrees. It can be improved of course and actually I have a follow-up PR to support merging different filter predicates with `OR`, I just didn't want to make this PR more complex. I think the logic can be extracted from `MergeScalarSubqueries` to a common place and used in follow-up PRs for different purposes. Like, it can be used for the general (2) and in that case `CTERef` is the only viable way to reference common parts, indeed.
   - Merging subqueries seems to be much simpler to start with (than merging arbitrary common parts of a plan). It is well defined which plan should be tried to merged with another plan. This PR wants to deal with that scope only. But for this limited scope `CTERef`'s physical implementation seems to be a bit overkill, but a simple expression a right fit.
   
   > * (4) subqueries might present more optimization opportunities, but I think the additional optimizations would better come up in physical plans rather than logical plans.
   > 
   > > I'm not sure I get this. Why ColumnPruning should consider these new nodes?
   > 
   > There's a pattern matching for CTE:
   > 
   > https://github.com/apache/spark/blob/efe43306fcab18f076f755c81c0406ebc1a5fee9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala#L874-L880
   > Similarly, if a scalar subquery reference was pruned by some other optimizations, we may want to remove the subquery too.
   
   I think this code only prunes the plan child of `WithCTE` not the CTE definitions, but indeed, we need a `case p @ Project(_, w: <definition hosting node>) =>` pattern there.
   I'm not sure as in that case `PlanSubqueries` / `PlanAdaptiveSubqueries` simply don't add the physical `ScalarSubquery` to the plan. Likewise, we don't seem to remove any CTE definition that are no longer needed.
   
   > > how about combining them into one node (CommonDefinitions?) that can host CTEs and scalar subqueries as well?
   > 
   > The difference seems minor at logical level - but the latter avoid things like SubqueryReference:
   > 
   > CommonDef +- Seq(Subquery)
   > 
   > v.s. CommonDef +- Seq(Plan) but wrap CTE into a scalar subquery of (Select .. FROM cte) at the place of original subqueries.
   
   I don't feel that `SubqueryReference` is bad and we need to avoid it, but if others also suggest removing it, I can change the implementation to use only `WithCTE` + `CTERef`s.
   
   Please also consider my alternative proposal to rename `WithCTE` to `CommonDef` and keep `CTERef` for common query parts and `SubqueryReference` for common subquery expressions:
   
   ```
   CommonDef
   : +- Seq(Subquery)
   +- Seq(Plan)
   ```
   


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BloomFilterMightContain.scala
##########
@@ -56,6 +56,9 @@ case class BloomFilterMightContain(
           case e : Expression if e.foldable => TypeCheckResult.TypeCheckSuccess
           case subquery : PlanExpression[_] if !subquery.containsPattern(OUTER_REFERENCE) =>
             TypeCheckResult.TypeCheckSuccess
+          case GetStructField(subquery: PlanExpression[_], _, _)

Review comment:
       This is because with this PR some bloom filter aggregate subqueries can be merged. E.g.
   ```
   Join Inner, ((c1#45920 = c2#45926) AND (b1#45919 = b2#45925))
   :- Filter might_contain(scalar-subquery#45986 [], xxhash64(b1#45919, 42))
   :  :  +- Aggregate [bloom_filter_agg(xxhash64(b2#45925, 42), 1000000, 8388608, 0, 0) AS bloomFilter#45985]
   :  :     +- Filter ((isnotnull(a2#45924) AND (a2#45924 = 62)) AND (isnotnull(c2#45926) AND isnotnull(b2#45925)))
   :  :        +- Relation default.bf2[a2#45924,b2#45925,c2#45926,d2#45927,e2#45928,f2#45929] parquet
   :  +- Filter might_contain(scalar-subquery#45983 [], xxhash64(c1#45920, 42))
   :     :  +- Aggregate [bloom_filter_agg(xxhash64(c2#45926, 42), 1000000, 8388608, 0, 0) AS bloomFilter#45982]
   :     :     +- Filter ((isnotnull(a2#45924) AND (a2#45924 = 62)) AND (isnotnull(c2#45926) AND isnotnull(b2#45925)))
   :     :        +- Relation default.bf2[a2#45924,b2#45925,c2#45926,d2#45927,e2#45928,f2#45929] parquet
   :     +- Filter (isnotnull(c1#45920) AND isnotnull(b1#45919))
   :        +- Relation default.bf1[a1#45918,b1#45919,c1#45920,d1#45921,e1#45922,f1#45923] parquet
   +- Filter ((isnotnull(a2#45924) AND (a2#45924 = 62)) AND (isnotnull(c2#45926) AND isnotnull(b2#45925)))
      +- Relation default.bf2[a2#45924,b2#45925,c2#45926,d2#45927,e2#45928,f2#45929] parquet
   ```
   => 
   ```
   WithCTE
   :- CTERelationDef 0
   :  +- Project [named_struct(bloomFilter, bloomFilter#45985, bloomFilter, bloomFilter#45982) AS mergedValue#45989]
   :     +- Aggregate [bloom_filter_agg(xxhash64(b2#45925, 42), 1000000, 8388608, 0, 0) AS bloomFilter#45985, bloom_filter_agg(xxhash64(c2#45926, 42), 1000000, 8388608, 0, 0) AS bloomFilter#45982]
   :        +- Filter ((isnotnull(a2#45924) AND (a2#45924 = 62)) AND (isnotnull(c2#45926) AND isnotnull(b2#45925)))
   :           +- Relation default.bf2[a2#45924,b2#45925,c2#45926,d2#45927,e2#45928,f2#45929] parquet
   +- Join Inner, ((c1#45920 = c2#45926) AND (b1#45919 = b2#45925))
      :- Filter might_contain(scalar-subquery#45986 [].bloomFilter, xxhash64(b1#45919, 42))
      :  :  +- CTERelationRef 0, true, [mergedValue#45989], true
      :  +- Filter might_contain(scalar-subquery#45983 [].bloomFilter, xxhash64(c1#45920, 42))
      :     :  +- CTERelationRef 0, true, [mergedValue#45989], true
      :     +- Filter (isnotnull(c1#45920) AND isnotnull(b1#45919))
      :        +- Relation default.bf1[a1#45918,b1#45919,c1#45920,d1#45921,e1#45922,f1#45923] parquet
      +- Filter ((isnotnull(a2#45924) AND (a2#45924 = 62)) AND (isnotnull(c2#45926) AND isnotnull(b2#45925)))
         +- Relation default.bf2[a2#45924,b2#45925,c2#45926,d2#45927,e2#45928,f2#45929] parquet
   ```




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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] tgravescs commented on pull request #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


   sent email to dev for 3.3 to see if any concerns with this.  Hopefully I can finish reviewing today.  
   We ran through a bunch of tests and it looks good from that testing.}
   
   Agree anything else should be in followup PRs, you could file an issue for them if you haven't 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,421 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CommonScalarSubqueries, Filter, Join, LogicalPlan, Project, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s to compute multiple scalar
+ * values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ *   During this first traversal each [[ScalarSubquery]] expression is replaced to a
+ *   [[ScalarSubqueryReference]] pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a results of merging 2 or more
+ *   plans, or it is a plan that was seen only once.
+ *   Merged plans in the cache get a "header", that is is basically
+ *   `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in new root
+ *   [[Project]] node. This expression ensures that the merged plan is a valid scalar subquery that
+ *   returns only one value.
+ * - A second traversal checks if a [[ScalarSubqueryReference]] is pointing to a merged subquery
+ *   plan or not and either keeps the reference or restores the original [[ScalarSubquery]].
+ *   If there are [[ScalarSubqueryReference]] nodes remained a [[CommonScalarSubqueries]] root node
+ *   is added to the plan with the referenced scalar subqueries.
+ * - [[PlanSubqueries]] or [[PlanAdaptiveSubqueries]] rule does the physical planning of scalar
+ *   subqueries including the ones under [[CommonScalarSubqueriesExec]] node and replaces
+ *   each [[ScalarSubqueryReference]] to their referenced physical plan in
+ *   `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() header))` form.
+ *   It is important that references pointing to the same merged subquery are replaced to the same
+ *   planned instance to make sure that each merged subquery runs only once (even without a wrapping
+ *   [[ReuseSubquery]] node).
+ *   Finally, the [[CommonScalarSubqueriesExec]] node is removed from the physical plan.
+ * - The [[ReuseExchangeAndSubquery]] rule wraps the second, third, ... instances of the same
+ *   subquery into a [[ReuseSubquery]] node, but this just a cosmetic change in the plan.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *          scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * CommonScalarSubqueries [scalar-subquery#250 []]
+ * :  +- Project [named_struct(avg(a), avg(a)#236, sum(b), sum(b)#238L) AS mergedValue#249]
+ * :     +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :        +- Project [a#233, b#234]
+ * :           +- Relation default.t[a#233,b#234] parquet
+ * +- Project [scalarsubqueryreference(0, 0, DoubleType, 231) AS scalarsubquery()#241,
+ *             scalarsubqueryreference(0, 1, LongType, 232) AS scalarsubquery()#242L]
+ *    +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    plan match {
+      case Subquery(_: CommonScalarSubqueries, _) => plan
+      case s: Subquery => s.copy(child = extractCommonScalarSubqueries(s.child))
+      case _: CommonScalarSubqueries => plan
+      case _ => extractCommonScalarSubqueries(plan)
+    }
+  }
+
+  /**
+   * An item in the cache of merged scalar subqueries.
+   *
+   * @param elements  List of attributes that form the scalar return value of a merged subquery
+   * @param plan      The plan of a merged scalar subquery
+   * @param merged    A flag to identify if this item is the result of merging subqueries.
+   *                  Please note that `elements.size == 1` doesn't always mean that the plan is not
+   *                  merged as there can be subqueries that are different ([[checkIdenticalPlans]]
+   *                  is false) due to an extra [[Project]] node in one of them. In that case
+   *                  `elements.size` remains 1 after merging, but the merged flag becomes true.
+   */
+  case class Header(elements: Seq[(String, Attribute)], plan: LogicalPlan, merged: Boolean)
+
+  private def extractCommonScalarSubqueries(plan: LogicalPlan) = {
+    val cache = ListBuffer.empty[Header]
+    val newPlan = removeReferences(insertReferences(plan, cache), cache)
+    if (cache.nonEmpty) {
+      val scalarSubqueries = cache.map {
+        case Header(elements, child, _) => ScalarSubquery(createProject(elements, child))
+      }.toSeq
+      CommonScalarSubqueries(scalarSubqueries, newPlan)
+    } else {
+      newPlan
+    }
+  }
+
+  // First traversal builds up the cache and inserts `ScalarSubqueryReference`s to the plan.
+  private def insertReferences(plan: LogicalPlan, cache: ListBuffer[Header]): LogicalPlan = {
+    plan.transformAllExpressionsWithPruning(_.containsAnyPattern(SCALAR_SUBQUERY)) {
+      case s: ScalarSubquery if !s.isCorrelated && s.deterministic =>
+        val (subqueryIndex, headerIndex) = cacheSubquery(s.plan, cache)
+        ScalarSubqueryReference(subqueryIndex, headerIndex, s.dataType, s.exprId)
+    }
+  }
+
+  // Caching returns the index of the subquery in the cache and the index of scalar member in the
+  // "header", that `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in a
+  // [[Project]] node.
+  private def cacheSubquery(plan: LogicalPlan, cache: ListBuffer[Header]): (Int, Int) = {
+    val output = plan.output.head
+    cache.zipWithIndex.collectFirst(Function.unlift { case (header, subqueryIndex) =>
+      checkIdenticalPlans(plan, header.plan).map { outputMap =>
+        val mappedOutput = mapAttributes(output, outputMap)
+        val headerIndex = header.elements.indexWhere {
+          case (_, attribute) => attribute.exprId == mappedOutput.exprId
+        }
+        subqueryIndex -> headerIndex
+      }.orElse(tryMergePlans(plan, header.plan).map {
+        case (mergedPlan, outputMap) =>
+          val mappedOutput = mapAttributes(output, outputMap)
+          var headerIndex = header.elements.indexWhere {
+            case (_, attribute) => attribute.exprId == mappedOutput.exprId
+          }
+          val newHeaderElements = if (headerIndex == -1) {
+            headerIndex = header.elements.size
+            header.elements :+ (output.name -> mappedOutput)
+          } else {
+            header.elements
+          }
+          cache(subqueryIndex) = Header(newHeaderElements, mergedPlan, true)
+          subqueryIndex -> headerIndex
+      })
+    }).getOrElse {
+      cache += Header(Seq(output.name -> output), plan, false)
+      cache.length - 1 -> 0
+    }
+  }
+
+  // If 2 plans are identical return the attribute mapping from the new to the cached version.
+  private def checkIdenticalPlans(
+      newPlan: LogicalPlan,
+      cachedPlan: LogicalPlan): Option[AttributeMap[Attribute]] = {
+    if (newPlan.canonicalized == cachedPlan.canonicalized) {
+      Some(AttributeMap(newPlan.output.zip(cachedPlan.output)))
+    } else {
+      None
+    }
+  }
+
+  // Recursively traverse down and try merging 2 plans. If merge is possible then return the merged
+  // plan with the attribute mapping from the new to the merged version.
+  // Please note that merging arbitrary plans can be complicated, the current version supports only
+  // some of the most important nodes.
+  private def tryMergePlans(
+      newPlan: LogicalPlan,
+      cachedPlan: LogicalPlan): Option[(LogicalPlan, AttributeMap[Attribute])] = {
+    checkIdenticalPlans(newPlan, cachedPlan).map(cachedPlan -> _).orElse(
+      (newPlan, cachedPlan) match {
+        case (np: Project, cp: Project) =>
+          tryMergePlans(np.child, cp.child).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.projectList, outputMap, cp.projectList)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np, cp: Project) =>
+          tryMergePlans(np, cp.child).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.output, outputMap, cp.projectList)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np: Project, cp) =>
+          tryMergePlans(np.child, cp).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.projectList, outputMap, cp.output)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np: Aggregate, cp: Aggregate) if supportedAggregateMerge(np, cp) =>
+          tryMergePlans(np.child, cp.child).flatMap { case (mergedChild, outputMap) =>
+            val mappedNewGroupingExpression =
+              np.groupingExpressions.map(mapAttributes(_, outputMap))
+            // Order of grouping expression doesn't matter so we can compare sets
+            if (ExpressionSet(mappedNewGroupingExpression) ==
+              ExpressionSet(cp.groupingExpressions)) {
+              val (mergedAggregateExpressions, newOutputMap) =
+                mergeNamedExpressions(np.aggregateExpressions, outputMap, cp.aggregateExpressions)
+              val mergedPlan =
+                Aggregate(cp.groupingExpressions, mergedAggregateExpressions, mergedChild)
+              Some(mergedPlan -> newOutputMap)
+            } else {
+              None
+            }
+          }
+
+        // Merging general nodes is complicated and this implementation supports only those nodes in
+        // which the order and the number of output attributes are not relevant (see
+        // `supportedMerge()` whitelist).
+        // Also, this implementation supports only those nodes in which children can be merged in
+        // the same order.
+        case (np, cp) if supportedMerge(np) && np.getClass == cp.getClass &&
+            np.children.size == cp.children.size &&
+            np.expressions.size == cp.expressions.size &&
+            // Fields that don't contain any children or expressions should match
+            np.productIterator.filterNot(np.children.contains)
+              .filter(QueryPlan.extractExpressions(_).isEmpty).toSeq ==
+              cp.productIterator.filterNot(cp.children.contains)
+                .filter(QueryPlan.extractExpressions(_).isEmpty).toSeq =>
+          val merged = np.children.zip(cp.children).map {
+            case (npChild, cpChild) => tryMergePlans(npChild, cpChild)
+          }
+          if (merged.forall(_.isDefined)) {
+            val (mergedChildren, outputMaps) = merged.map(_.get).unzip
+            val outputMap = AttributeMap(outputMaps.map(_.iterator).reduce(_ ++ _).toSeq)
+            // We know that fields that don't contain any children or expressions do match and
+            // children can be merged so we need to test expressions only
+            if (np.expressions.map(mapAttributes(_, outputMap).canonicalized) ==
+              cp.expressions.map(_.canonicalized)) {
+              val mergedPlan = cp.withNewChildren(mergedChildren)
+              Some(mergedPlan -> outputMap)
+            } else {
+              None
+            }
+          } else {
+            None
+          }
+
+        // As a follow-up, it would be possible to merge `CommonScalarSubqueries` nodes, which would
+        // allow merging subqueries with mergee subqueries.

Review comment:
       Thanks, fixed in https://github.com/apache/spark/pull/32298/commits/87ba289589c41a9c83e39c493e4dabf90b95c14a

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,421 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CommonScalarSubqueries, Filter, Join, LogicalPlan, Project, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s to compute multiple scalar
+ * values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ *   During this first traversal each [[ScalarSubquery]] expression is replaced to a
+ *   [[ScalarSubqueryReference]] pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a results of merging 2 or more
+ *   plans, or it is a plan that was seen only once.
+ *   Merged plans in the cache get a "header", that is is basically
+ *   `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in new root
+ *   [[Project]] node. This expression ensures that the merged plan is a valid scalar subquery that
+ *   returns only one value.
+ * - A second traversal checks if a [[ScalarSubqueryReference]] is pointing to a merged subquery
+ *   plan or not and either keeps the reference or restores the original [[ScalarSubquery]].
+ *   If there are [[ScalarSubqueryReference]] nodes remained a [[CommonScalarSubqueries]] root node
+ *   is added to the plan with the referenced scalar subqueries.
+ * - [[PlanSubqueries]] or [[PlanAdaptiveSubqueries]] rule does the physical planning of scalar
+ *   subqueries including the ones under [[CommonScalarSubqueriesExec]] node and replaces
+ *   each [[ScalarSubqueryReference]] to their referenced physical plan in
+ *   `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() header))` form.
+ *   It is important that references pointing to the same merged subquery are replaced to the same
+ *   planned instance to make sure that each merged subquery runs only once (even without a wrapping
+ *   [[ReuseSubquery]] node).
+ *   Finally, the [[CommonScalarSubqueriesExec]] node is removed from the physical plan.
+ * - The [[ReuseExchangeAndSubquery]] rule wraps the second, third, ... instances of the same
+ *   subquery into a [[ReuseSubquery]] node, but this just a cosmetic change in the plan.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *          scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * CommonScalarSubqueries [scalar-subquery#250 []]
+ * :  +- Project [named_struct(avg(a), avg(a)#236, sum(b), sum(b)#238L) AS mergedValue#249]
+ * :     +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :        +- Project [a#233, b#234]
+ * :           +- Relation default.t[a#233,b#234] parquet
+ * +- Project [scalarsubqueryreference(0, 0, DoubleType, 231) AS scalarsubquery()#241,
+ *             scalarsubqueryreference(0, 1, LongType, 232) AS scalarsubquery()#242L]
+ *    +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    plan match {
+      case Subquery(_: CommonScalarSubqueries, _) => plan
+      case s: Subquery => s.copy(child = extractCommonScalarSubqueries(s.child))
+      case _: CommonScalarSubqueries => plan
+      case _ => extractCommonScalarSubqueries(plan)
+    }
+  }
+
+  /**
+   * An item in the cache of merged scalar subqueries.
+   *
+   * @param elements  List of attributes that form the scalar return value of a merged subquery
+   * @param plan      The plan of a merged scalar subquery
+   * @param merged    A flag to identify if this item is the result of merging subqueries.
+   *                  Please note that `elements.size == 1` doesn't always mean that the plan is not
+   *                  merged as there can be subqueries that are different ([[checkIdenticalPlans]]
+   *                  is false) due to an extra [[Project]] node in one of them. In that case
+   *                  `elements.size` remains 1 after merging, but the merged flag becomes true.
+   */
+  case class Header(elements: Seq[(String, Attribute)], plan: LogicalPlan, merged: Boolean)
+
+  private def extractCommonScalarSubqueries(plan: LogicalPlan) = {
+    val cache = ListBuffer.empty[Header]
+    val newPlan = removeReferences(insertReferences(plan, cache), cache)
+    if (cache.nonEmpty) {
+      val scalarSubqueries = cache.map {
+        case Header(elements, child, _) => ScalarSubquery(createProject(elements, child))
+      }.toSeq
+      CommonScalarSubqueries(scalarSubqueries, newPlan)
+    } else {
+      newPlan
+    }
+  }
+
+  // First traversal builds up the cache and inserts `ScalarSubqueryReference`s to the plan.
+  private def insertReferences(plan: LogicalPlan, cache: ListBuffer[Header]): LogicalPlan = {
+    plan.transformAllExpressionsWithPruning(_.containsAnyPattern(SCALAR_SUBQUERY)) {
+      case s: ScalarSubquery if !s.isCorrelated && s.deterministic =>
+        val (subqueryIndex, headerIndex) = cacheSubquery(s.plan, cache)
+        ScalarSubqueryReference(subqueryIndex, headerIndex, s.dataType, s.exprId)
+    }
+  }
+
+  // Caching returns the index of the subquery in the cache and the index of scalar member in the
+  // "header", that `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in a
+  // [[Project]] node.
+  private def cacheSubquery(plan: LogicalPlan, cache: ListBuffer[Header]): (Int, Int) = {
+    val output = plan.output.head
+    cache.zipWithIndex.collectFirst(Function.unlift { case (header, subqueryIndex) =>
+      checkIdenticalPlans(plan, header.plan).map { outputMap =>
+        val mappedOutput = mapAttributes(output, outputMap)
+        val headerIndex = header.elements.indexWhere {
+          case (_, attribute) => attribute.exprId == mappedOutput.exprId
+        }
+        subqueryIndex -> headerIndex
+      }.orElse(tryMergePlans(plan, header.plan).map {
+        case (mergedPlan, outputMap) =>
+          val mappedOutput = mapAttributes(output, outputMap)
+          var headerIndex = header.elements.indexWhere {
+            case (_, attribute) => attribute.exprId == mappedOutput.exprId
+          }
+          val newHeaderElements = if (headerIndex == -1) {
+            headerIndex = header.elements.size
+            header.elements :+ (output.name -> mappedOutput)
+          } else {
+            header.elements
+          }
+          cache(subqueryIndex) = Header(newHeaderElements, mergedPlan, true)
+          subqueryIndex -> headerIndex
+      })
+    }).getOrElse {
+      cache += Header(Seq(output.name -> output), plan, false)
+      cache.length - 1 -> 0
+    }
+  }
+
+  // If 2 plans are identical return the attribute mapping from the new to the cached version.
+  private def checkIdenticalPlans(
+      newPlan: LogicalPlan,
+      cachedPlan: LogicalPlan): Option[AttributeMap[Attribute]] = {
+    if (newPlan.canonicalized == cachedPlan.canonicalized) {
+      Some(AttributeMap(newPlan.output.zip(cachedPlan.output)))
+    } else {
+      None
+    }
+  }
+
+  // Recursively traverse down and try merging 2 plans. If merge is possible then return the merged
+  // plan with the attribute mapping from the new to the merged version.
+  // Please note that merging arbitrary plans can be complicated, the current version supports only
+  // some of the most important nodes.
+  private def tryMergePlans(
+      newPlan: LogicalPlan,
+      cachedPlan: LogicalPlan): Option[(LogicalPlan, AttributeMap[Attribute])] = {
+    checkIdenticalPlans(newPlan, cachedPlan).map(cachedPlan -> _).orElse(
+      (newPlan, cachedPlan) match {
+        case (np: Project, cp: Project) =>
+          tryMergePlans(np.child, cp.child).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.projectList, outputMap, cp.projectList)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np, cp: Project) =>
+          tryMergePlans(np, cp.child).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.output, outputMap, cp.projectList)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np: Project, cp) =>
+          tryMergePlans(np.child, cp).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.projectList, outputMap, cp.output)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np: Aggregate, cp: Aggregate) if supportedAggregateMerge(np, cp) =>
+          tryMergePlans(np.child, cp.child).flatMap { case (mergedChild, outputMap) =>
+            val mappedNewGroupingExpression =
+              np.groupingExpressions.map(mapAttributes(_, outputMap))
+            // Order of grouping expression doesn't matter so we can compare sets
+            if (ExpressionSet(mappedNewGroupingExpression) ==
+              ExpressionSet(cp.groupingExpressions)) {
+              val (mergedAggregateExpressions, newOutputMap) =
+                mergeNamedExpressions(np.aggregateExpressions, outputMap, cp.aggregateExpressions)
+              val mergedPlan =
+                Aggregate(cp.groupingExpressions, mergedAggregateExpressions, mergedChild)
+              Some(mergedPlan -> newOutputMap)
+            } else {
+              None
+            }
+          }
+
+        // Merging general nodes is complicated and this implementation supports only those nodes in
+        // which the order and the number of output attributes are not relevant (see
+        // `supportedMerge()` whitelist).
+        // Also, this implementation supports only those nodes in which children can be merged in
+        // the same order.
+        case (np, cp) if supportedMerge(np) && np.getClass == cp.getClass &&
+            np.children.size == cp.children.size &&
+            np.expressions.size == cp.expressions.size &&
+            // Fields that don't contain any children or expressions should match
+            np.productIterator.filterNot(np.children.contains)
+              .filter(QueryPlan.extractExpressions(_).isEmpty).toSeq ==
+              cp.productIterator.filterNot(cp.children.contains)
+                .filter(QueryPlan.extractExpressions(_).isEmpty).toSeq =>
+          val merged = np.children.zip(cp.children).map {
+            case (npChild, cpChild) => tryMergePlans(npChild, cpChild)
+          }
+          if (merged.forall(_.isDefined)) {
+            val (mergedChildren, outputMaps) = merged.map(_.get).unzip
+            val outputMap = AttributeMap(outputMaps.map(_.iterator).reduce(_ ++ _).toSeq)
+            // We know that fields that don't contain any children or expressions do match and
+            // children can be merged so we need to test expressions only
+            if (np.expressions.map(mapAttributes(_, outputMap).canonicalized) ==
+              cp.expressions.map(_.canonicalized)) {
+              val mergedPlan = cp.withNewChildren(mergedChildren)
+              Some(mergedPlan -> outputMap)
+            } else {
+              None
+            }
+          } else {
+            None
+          }
+
+        // As a follow-up, it would be possible to merge `CommonScalarSubqueries` nodes, which would
+        // allow merging subqueries with mergee subqueries.
+        // E.g. this query:
+        //
+        // SELECT
+        //   (
+        //     SELECT
+        //       (SELECT avg(a) FROM t GROUP BY b) +
+        //       (SELECT sum(b) FROM t GROUP BY b)
+        //   ),
+        //   (
+        //     SELECT
+        //       (SELECT max(a) FROM t GROUP BY b) +
+        //       (SELECT min(b) FROM t GROUP BY b)
+        //   )
+        //
+        // is currently optimized to:
+        //
+        // == Optimized Logical Plan ==
+        // Project [scalar-subquery#233 [] AS scalarsubquery()#255,
+        //          scalar-subquery#236 [] AS scalarsubquery()#256]
+        // :  :- CommonScalarSubqueries [scalar-subquery#264 []]
+        // :  :  :  +- Aggregate [b#238], [named_struct(avg(a), avg(a#237), sum(b), sum(b#238))
+        //                                 AS mergedValue#263]
+        // :  :  :     +- Relation default.t[a#237,b#238] parquet
+        // :  :  +- Project [(scalarsubqueryreference(0, 0, DoubleType, 231) +
+        //                   cast(scalarsubqueryreference(0, 1, LongType, 232) as double))
+        //                   AS (scalarsubquery() + scalarsubquery())#245]
+        // :  :     +- OneRowRelation
+        // :  +- CommonScalarSubqueries [scalar-subquery#269 []]
+        // :     :  +- Aggregate [b#254], [named_struct(min(a), min(a#253), max(b), max(b#254))
+        //                                 AS mergedValue#268]
+        // :     :     +- Relation default.t[a#253,b#254] parquet
+        // :     +- Project [(scalarsubqueryreference(0, 0, IntegerType, 234) +
+        //                   scalarsubqueryreference(0, 1, IntegerType, 235))
+        //                   AS (scalarsubquery() + scalarsubquery())#252]
+        // :        +- OneRowRelation
+        // +- OneRowRelation
+        //
+        // but if we implemented merging `CommonScalarSubqueries` nodes then the plan could be
+        // transformed further and all leaf subqueries could be merged.
+
+        // Otherwise merging is not possible.
+        case _ => None
+      })
+  }
+
+  private def createProject(elements: Seq[(String, Attribute)], plan: LogicalPlan): Project = {
+    Project(
+      Seq(Alias(
+        CreateNamedStruct(elements.flatMap {
+          case (name, attribute) => Seq(Literal(name), attribute)
+        }),
+        "mergedValue")()),
+      plan)
+  }
+
+  private def mapAttributes[T <: Expression](expr: T, outputMap: AttributeMap[Attribute]) = {
+    expr.transform {
+      case a: Attribute => outputMap.getOrElse(a, a)
+    }.asInstanceOf[T]
+  }
+
+  // Applies `outputMap` attribute mapping on elements of `newExpressions` and merges them into
+  // `cachedExpressions`. Returns the merged expressions and the attribute mapping from the new to
+  // the merged version that can be propagated up during merging nodes.
+  private def mergeNamedExpressions(
+      newExpressions: Seq[NamedExpression],
+      outputMap: AttributeMap[Attribute],
+      cachedExpressions: Seq[NamedExpression]) = {
+    val mergedExpressions = ListBuffer[NamedExpression](cachedExpressions: _*)
+    val newOutputMap = AttributeMap(newExpressions.map { ne =>
+      val mapped = mapAttributes(ne, outputMap)
+      val withoutAlias = mapped match {
+        case Alias(child, _) => child
+        case e => e
+      }
+      ne.toAttribute -> mergedExpressions.find {
+        case Alias(child, _) => child semanticEquals withoutAlias
+        case e => e semanticEquals withoutAlias
+      }.getOrElse {
+        mergedExpressions += mapped
+        mapped
+      }.toAttribute
+    })
+    (mergedExpressions.toSeq, newOutputMap)
+  }
+
+  // Merging different aggregate implementations could cause performance regression

Review comment:
       Done in https://github.com/apache/spark/pull/32298/commits/87ba289589c41a9c83e39c493e4dabf90b95c14a




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] sigmod edited a comment on pull request #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

Posted by GitBox <gi...@apache.org>.
sigmod edited a comment on pull request #32298:
URL: https://github.com/apache/spark/pull/32298#issuecomment-1075524713


   > But I think that an extra shuffle could mean performance degradation
   > in case of scalar subqueries (CTEs returning only one row).
   
   Is it still way better than running the scalar subqueries over the same table multiple times?
   I'm more worried about the complexities (i.e., pattern matching cognitive overhead) with new plan nodes like CommonSubqueries and CommonSubqueriesExec. Many rules have been implemented as pattern matching, e.g., a rule that matches a Project is supposed to also match CommonSubqueries (in theory)?


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


   > I'm more worried about the complexities (i.e., pattern matching cognitive overhead) with new plan nodes like CommonSubqueries and CommonSubqueriesExec. Many rules have been implemented as pattern matching, e.g., a rule that matches a Project is supposed to also match CommonSubqueries (in theory)?
   
   Hmm, `CommonScalarSubqueries` (and `CommonScalarSubqueriesExec`) can appear only as a root node, it is't exactly the same as `Project`. It never replaces Project. I think it is more similar to `WithCTE` in that regard. Can you give an example where we need to handle these new nodes as `Project`?


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


   **[Test build #139068 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/139068/testReport)** for PR 32298 at commit [`0fe66dc`](https://github.com/apache/spark/commit/0fe66dc65327a23ea98b754abb2740b636c1cf8b).
    * 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] sigmod commented on a change in pull request #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LeafNode, LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{MULTI_SCALAR_SUBQUERY, SCALAR_SUBQUERY}
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s into a
+ * [[MultiScalarSubquery]] to compute multiple scalar values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ * - The original [[ScalarSubquery]] expression is replaced to a reference pointing to its cached
+ *   version in this form: `GetStructField(MultiScalarSubquery(SubqueryReference(...)))`.
+ * - A second traversal checks if a [[SubqueryReference]] is pointing to a subquery plan that
+ *   returns multiple values and either replaces only [[SubqueryReference]] to the cached plan or
+ *   restores the whole expression to its original [[ScalarSubquery]] form.
+ * - [[ReuseSubquery]] rule makes sure that merged subqueries are computed once.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *   scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet

Review comment:
       >> In this PR the new MergeScalarSubqueries rule runs in a separate batch 
   >> after column pruning, close to the end of optimization. 
   >> This is by design to make sure no subsequent rule changes the structure 
   
   I don't see how we can guarantee that. I think the hidden, inter-rule dependency can add complexities for future development and maintenance. 
   
   For instance, someone could implement a new Strategy that internally calls ColumnPruning after exploring one logical plan alternative. By the time such a Strategy is implemented, the authors wouldn't be aware of the fact that ColumnPruning should *not* be called after MergeScalarSubqueries. 
   - First of all, such issues can hardly be detected by the author's new unit/query tests, as an effective test has to have both effective patterns to trigger the Strategy and MergeScalarSubqueries. However, such a case can happen in prod traffic;
   - Second, if they do find that's an issue, they would then have to either (a) add some hacks in the Aggregate to mark that MergeScalarSubqueries has been applied and hence ColumnPruning should not go through it, or (b) re-implement MergeScalarSubqueries per my proposal (1).
     
   I'm wondering whether we can pursue (2) for now, if it meets your need. It's less ambitious but may address most of your issue?  If you indeed have to extract subqueries over the entire tree, I don't have a clean approach in mind other than (1).
   
   >> Add a performance improvement to 1. so as to physical plan a merged 
   >> subquery only once. This is your (1) basically.
   
   The performance was not my initial concern, but rather, I think we'd better make MergeScalarSubqueries self-contained and does not depend on *an assumption that could later be changed*.
   
   >>  But I don't think we need lateral views
   
   Sub-querying over arrays are important use cases, for which we don't want to de-correlate. In this case, a subquery is more like an ordinary expression and should be evaluated within the operator 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 a change in pull request #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LeafNode, LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{MULTI_SCALAR_SUBQUERY, SCALAR_SUBQUERY}
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s into a
+ * [[MultiScalarSubquery]] to compute multiple scalar values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ * - The original [[ScalarSubquery]] expression is replaced to a reference pointing to its cached
+ *   version in this form: `GetStructField(MultiScalarSubquery(SubqueryReference(...)))`.
+ * - A second traversal checks if a [[SubqueryReference]] is pointing to a subquery plan that
+ *   returns multiple values and either replaces only [[SubqueryReference]] to the cached plan or
+ *   restores the whole expression to its original [[ScalarSubquery]] form.
+ * - [[ReuseSubquery]] rule makes sure that merged subqueries are computed once.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *   scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet

Review comment:
       Thanks @sigmod. I will try to update the PR by end of this week.




-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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] sigmod commented on a change in pull request #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LeafNode, LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{MULTI_SCALAR_SUBQUERY, SCALAR_SUBQUERY}
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s into a
+ * [[MultiScalarSubquery]] to compute multiple scalar values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ * - The original [[ScalarSubquery]] expression is replaced to a reference pointing to its cached
+ *   version in this form: `GetStructField(MultiScalarSubquery(SubqueryReference(...)))`.
+ * - A second traversal checks if a [[SubqueryReference]] is pointing to a subquery plan that
+ *   returns multiple values and either replaces only [[SubqueryReference]] to the cached plan or
+ *   restores the whole expression to its original [[ScalarSubquery]] form.
+ * - [[ReuseSubquery]] rule makes sure that merged subqueries are computed once.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *   scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet

Review comment:
       >  I would pursue (1) in this PR first and maybe (2) in a separate one. Does this sound acceptable?
   
   Yeah, that sounds great. Thanks a lot, @peter-toth!
   
   > There are 2 aggregates in both subqueries so without dedup both (2) and this PR could cause regressions.
   
   IIUC, I think it sounds like an existing bug for struct subfield pruning, which could be blocking (2) but is orthogonal to (2). For instance, if I write your example join query manually, I'd expect the struct subfield pruning to happen to the struct constructor, regardless of the existence of subqueries.
   
   > I've never seen such transformations in SparkStrategys.
   
   It's not uncommon in exploration Strategies such as index selection, common subplan dedup, when we substitute the subtree of a tree node T with another subtree (from somewhere else in the plan or a different access path) that may contain unneeded columns for 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] sigmod commented on a change in pull request #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LeafNode, LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{MULTI_SCALAR_SUBQUERY, SCALAR_SUBQUERY}
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s into a
+ * [[MultiScalarSubquery]] to compute multiple scalar values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ * - The original [[ScalarSubquery]] expression is replaced to a reference pointing to its cached
+ *   version in this form: `GetStructField(MultiScalarSubquery(SubqueryReference(...)))`.
+ * - A second traversal checks if a [[SubqueryReference]] is pointing to a subquery plan that
+ *   returns multiple values and either replaces only [[SubqueryReference]] to the cached plan or
+ *   restores the whole expression to its original [[ScalarSubquery]] form.
+ * - [[ReuseSubquery]] rule makes sure that merged subqueries are computed once.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *   scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet

Review comment:
       >> In this PR the new MergeScalarSubqueries rule runs in a separate batch 
   >> after column pruning, close to the end of optimization. 
   >> This is by design to make sure no subsequent rule changes the structure 
   
   I don't see how we can guarantee that. I think the hidden, inter-rule dependency can add complexities for future development and maintenance. 
   
   For instance, someone could implement a new Strategy that internally calls ColumnPruning after exploring one logical plan alternative. By the time such a Strategy is implemented, the authors wouldn't be aware of the fact that ColumnPruning should *not* be called after MergeScalarSubqueries. 
   - First of all, such issues can hardly be detected by the author's new unit/query tests, as an effective test has to have both effective patterns to trigger the Strategy and for MergeScalarSubqueries. However, such a case can happen in prod traffic;
   - Second, if they do find that's an issue, they would then have to either (a) add some hacks in the Aggregate to mark that MergeScalarSubqueries has been applied and hence ColumnPruning should not go through it, or (b) re-implement MergeScalarSubqueries per my proposal (1).
     
   I'm wondering whether we can pursue (2) for now, if it meets your need. It's less ambitious but may address most of your issue?  If you indeed have to extract subqueries over the entire tree, I don't have a clean approach in mind other than (1).
   
   >> Add a performance improvement to 1. so as to physical plan a merged 
   >> subquery only once. This is your (1) basically.
   
   The performance was not my initial concern, but rather, I think we'd better make MergeScalarSubqueries self-contained and does not depend on *an assumption that could later be changed*.
   
   >>  But I don't think we need lateral views
   
   Sub-querying over arrays are important use cases, for which we don't want to de-correlate. In this case, a subquery is more like an ordinary expression and should be evaluated within the operator 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 commented on pull request #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


   **[Test build #139058 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/139058/testReport)** for PR 32298 at commit [`3cb1f38`](https://github.com/apache/spark/commit/3cb1f38c123b76db539d13c0b94164d51318c8af).
    * This patch **fails Java style 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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


   **[Test build #140073 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/140073/testReport)** for PR 32298 at commit [`41c0f0a`](https://github.com/apache/spark/commit/41c0f0aa9a1ab0b19a538e20b586807e39059eec).


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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






-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


   **[Test build #138112 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138112/testReport)** for PR 32298 at commit [`e63111d`](https://github.com/apache/spark/commit/e63111d2e26b92a983c5561b90608d80e0b4e7ee).
    * 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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


   **[Test build #145035 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/145035/testReport)** for PR 32298 at commit [`61f2b34`](https://github.com/apache/spark/commit/61f2b34c49c2f7c76f3ebd320b9d2f927f7ea100).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `public class JavaModuleOptions `
     * `class SparkConf(object):`
     * `class MultivariateGaussian(NamedTuple):`
     * `class HistogramPlotBase(NumericPlotBase):`
     * `class KdePlotBase(NumericPlotBase):`
     * `        new_class = type(NameTypeHolder.short_name, (NameTypeHolder,), `
     * `class ResultIterable(Iterable[T]):`
     * `class Database(NamedTuple):`
     * `class Table(NamedTuple):`
     * `class Column(NamedTuple):`
     * `class Function(NamedTuple):`
     * `class SparkUpgradeException(CapturedException):`
     * `class SparkJobInfo(NamedTuple):`
     * `class SparkStageInfo(NamedTuple):`
     * `  protected class YarnSchedulerEndpoint(override val rpcEnv: RpcEnv)`
     * `public class ExpressionImplUtils `
     * `public class NumericHistogram `
     * `   * The Coord class defines a histogram bin, which is just an (x,y) pair.`
     * `  public static class Coord implements Comparable `
     * `class IndexAlreadyExistsException(message: String, cause: Option[Throwable] = None)`
     * `trait ExpressionBuilder `
     * `class NoSuchIndexException(message: String, cause: Option[Throwable] = None)`
     * `trait FunctionExpressionBuilder `
     * `case class HistogramNumeric(`
     * `trait ExtractValue extends Expression with NullIntolerant `
     * `case class AesEncrypt(input: Expression, key: Expression, child: Expression)`
     * `case class AesDecrypt(input: Expression, key: Expression, child: Expression)`
     * `trait PadExpressionBuilderBase extends ExpressionBuilder `
     * `case class StringLPad(str: Expression, len: Expression, pad: Expression)`
     * `case class BinaryLPad(str: Expression, len: Expression, pad: Expression, child: Expression)`
     * `case class BinaryRPad(str: Expression, len: Expression, pad: Expression, child: Expression)`
     * `case class SetCatalogAndNamespace(child: LogicalPlan) extends UnaryCommand `
     * `case class CreateFunction(`
     * `case class CreateView(`
     * `case class CreateIndex(`
     * `case class DropIndex(`
     * `public class ColumnIOUtil `
     * `public class OrcColumnStatistics `
     * `public class OrcFooterReader `
     * `case class SetNamespaceCommand(namespace: Seq[String]) extends LeafRunnableCommand `
     * `case class ShowCatalogsCommand(pattern: Option[String]) extends LeafRunnableCommand `
     * `case class ParquetColumn(`
     * `case class CreateIndexExec(`
     * `case class DropIndexExec(`
     * `case class PushedDownOperators(`
     * `case class TableSampleInfo(`
     * `case class HashedRelationBroadcastMode(key: Seq[Expression], isNullAware: Boolean = false)`
     * `class RatePerMicroBatchProvider extends SimpleTableProvider with DataSourceRegister `
     * `class RatePerMicroBatchTable(`
     * `class RatePerMicroBatchStream(`
     * `case class RatePerMicroBatchStreamOffset(offset: Long, timestamp: Long) extends Offset `
     * `case class RatePerMicroBatchStreamInputPartition(`
     * `class RatePerMicroBatchStreamPartitionReader(`
     * `class SparkUDFExpressionBuilder extends FunctionExpressionBuilder `


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,416 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CommonScalarSubqueries, Filter, Join, LogicalPlan, Project, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s to compute multiple scalar
+ * values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ *   During this first traversal each [[ScalarSubquery]] expression is replaced to a
+ *   [[ScalarSubqueryReference]] pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a results of merging 2 or more
+ *   plans, or it is a plan that was seen only once.
+ *   Merged plans in the cache get a "header", that is is basically
+ *   `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in new root
+ *   [[Project]] node. This expression ensures that the merged plan is a valid scalar subquery that
+ *   returns only one value.
+ * - A second traversal checks if a [[ScalarSubqueryReference]] is pointing to a merged subquery
+ *   plan or not and either keeps the reference or restores the original [[ScalarSubquery]].
+ *   If there are [[ScalarSubqueryReference]] nodes remained a [[CommonScalarSubqueries]] root node
+ *   is added to the plan with the referenced scalar subqueries.
+ * - [[PlanSubqueries]] or [[PlanAdaptiveSubqueries]] rule does the physical planning of scalar
+ *   subqueries including the ones under [[CommonScalarSubqueriesExec]] node and replaces
+ *   each [[ScalarSubqueryReference]] to their referenced physical plan in
+ *   `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() header))` form.
+ *   It is important that references pointing to the same merged subquery are replaced to the same
+ *   planned instance to make sure that each merged subquery runs only once (even without a wrapping
+ *   [[ReuseSubquery]] node).
+ *   Finally, the [[CommonScalarSubqueriesExec]] node is removed from the physical plan.
+ * - The [[ReuseExchangeAndSubquery]] rule wraps the second, third, ... instances of the same
+ *   subquery into a [[ReuseSubquery]] node, but this just a cosmetic change in the plan.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *          scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * CommonScalarSubqueries [scalar-subquery#250 []]
+ * :  +- Project [named_struct(avg(a), avg(a)#236, sum(b), sum(b)#238L) AS mergedValue#249]
+ * :     +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :        +- Project [a#233, b#234]
+ * :           +- Relation default.t[a#233,b#234] parquet
+ * +- Project [scalarsubqueryreference(0, 0, DoubleType, 231) AS scalarsubquery()#241,
+ *             scalarsubqueryreference(0, 1, LongType, 232) AS scalarsubquery()#242L]
+ *    +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    if (conf.subqueryReuseEnabled) {
+      plan match {
+        case Subquery(_: CommonScalarSubqueries, _) => plan
+        case s: Subquery => s.copy(child = extractCommonScalarSubqueries(s.child))
+        case _: CommonScalarSubqueries => plan
+        case _ => extractCommonScalarSubqueries(plan)
+      }
+    } else {
+      plan
+    }
+  }
+
+  type Header = Project

Review comment:
       Thanks for the suggestion! Accepted your PR and indeed the code is much cleaner now. But, I think I need to keep the `merged` flag. I've added it back here: https://github.com/apache/spark/pull/32298/commits/e0a7610ea12f7c6712f5b7d521b1ccafa6107e76 with some explanation why it is needed.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,422 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CommonScalarSubqueries, Filter, Join, LogicalPlan, Project, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s to compute multiple scalar
+ * values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ *   During this first traversal each [[ScalarSubquery]] expression is replaced to a
+ *   [[ScalarSubqueryReference]] pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a results of merging 2 or more
+ *   plans, or it is a plan that was seen only once.
+ *   Merged plans in the cache get a "header", that is is basically
+ *   `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in new root
+ *   [[Project]] node. This expression ensures that the merged plan is a valid scalar subquery that
+ *   returns only one value.
+ * - A second traversal checks if a [[ScalarSubqueryReference]] is pointing to a merged subquery
+ *   plan or not and either keeps the reference or restores the original [[ScalarSubquery]].
+ *   If there are [[ScalarSubqueryReference]] nodes remained a [[CommonScalarSubqueries]] root node
+ *   is added to the plan with the referenced scalar subqueries.
+ * - [[PlanSubqueries]] or [[PlanAdaptiveSubqueries]] rule does the physical planning of scalar
+ *   subqueries including the ones under [[CommonScalarSubqueriesExec]] node and replaces
+ *   each [[ScalarSubqueryReference]] to their referenced physical plan in
+ *   `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() header))` form.
+ *   It is important that references pointing to the same merged subquery are replaced to the same
+ *   planned instance to make sure that each merged subquery runs only once (even without a wrapping
+ *   [[ReuseSubquery]] node).
+ *   Finally, the [[CommonScalarSubqueriesExec]] node is removed from the physical plan.
+ * - The [[ReuseExchangeAndSubquery]] rule wraps the second, third, ... instances of the same
+ *   subquery into a [[ReuseSubquery]] node, but this just a cosmetic change in the plan.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *          scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * CommonScalarSubqueries [scalar-subquery#250 []]
+ * :  +- Project [named_struct(avg(a), avg(a)#236, sum(b), sum(b)#238L) AS mergedValue#249]
+ * :     +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :        +- Project [a#233, b#234]
+ * :           +- Relation default.t[a#233,b#234] parquet
+ * +- Project [scalarsubqueryreference(0, 0, DoubleType, 231) AS scalarsubquery()#241,
+ *             scalarsubqueryreference(0, 1, LongType, 232) AS scalarsubquery()#242L]
+ *    +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    plan match {
+      case Subquery(_: CommonScalarSubqueries, _) => plan
+      case s: Subquery => s.copy(child = extractCommonScalarSubqueries(s.child))
+      case _: CommonScalarSubqueries => plan
+      case _ => extractCommonScalarSubqueries(plan)
+    }
+  }
+
+  /**
+   * An item in the cache of merged scalar subqueries.
+   *
+   * @param elements  List of attributes that form the scalar return value of a merged subquery
+   * @param plan      The plan of a merged scalar subquery
+   * @param merged    A flag to identify if this item is the result of merging subqueries.
+   *                  Please note that `elements.size == 1` doesn't always mean that the plan is not
+   *                  merged as there can be subqueries that are different ([[checkIdenticalPlans]]
+   *                  is false) due to an extra [[Project]] node in one of them. In that case
+   *                  `elements.size` remains 1 after merging, but the merged flag becomes true.
+   */
+  case class Header(elements: Seq[(String, Attribute)], plan: LogicalPlan, merged: Boolean)
+
+  private def extractCommonScalarSubqueries(plan: LogicalPlan) = {
+    val cache = ListBuffer.empty[Header]
+    val newPlan = removeReferences(insertReferences(plan, cache), cache)
+    if (cache.nonEmpty) {
+      val scalarSubqueries = cache.map {
+        case Header(elements, child, _) => ScalarSubquery(createProject(elements, child))
+      }.toSeq
+      CommonScalarSubqueries(scalarSubqueries, newPlan)
+    } else {
+      newPlan
+    }
+  }
+
+  // First traversal builds up the cache and inserts `ScalarSubqueryReference`s to the plan.
+  private def insertReferences(plan: LogicalPlan, cache: ListBuffer[Header]): LogicalPlan = {
+    plan.transformAllExpressionsWithPruning(_.containsAnyPattern(SCALAR_SUBQUERY)) {
+      case s: ScalarSubquery if s.children.isEmpty =>
+        val (subqueryIndex, headerIndex) = cacheSubquery(s.plan, cache)
+        ScalarSubqueryReference(subqueryIndex, headerIndex, s.dataType, s.exprId)
+    }
+  }
+
+  // Caching returns the index of the subquery in the cache and the index of scalar member in the
+  // "header", that `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in a
+  // [[Project]] node.
+  private def cacheSubquery(plan: LogicalPlan, cache: ListBuffer[Header]): (Int, Int) = {
+    val output = plan.output.head
+    cache.zipWithIndex.collectFirst(Function.unlift { case (header, subqueryIndex) =>

Review comment:
       If `cache.zipWithIndex.collectFirst(Function.unlift {` is hard to read then I can covert it to a loop.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,416 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CommonScalarSubqueries, Filter, Join, LogicalPlan, Project, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s to compute multiple scalar
+ * values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ *   During this first traversal each [[ScalarSubquery]] expression is replaced to a
+ *   [[ScalarSubqueryReference]] pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a results of merging 2 or more
+ *   plans, or it is a plan that was seen only once.
+ *   Merged plans in the cache get a "header", that is is basically
+ *   `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in new root
+ *   [[Project]] node. This expression ensures that the merged plan is a valid scalar subquery that
+ *   returns only one value.
+ * - A second traversal checks if a [[ScalarSubqueryReference]] is pointing to a merged subquery
+ *   plan or not and either keeps the reference or restores the original [[ScalarSubquery]].
+ *   If there are [[ScalarSubqueryReference]] nodes remained a [[CommonScalarSubqueries]] root node
+ *   is added to the plan with the referenced scalar subqueries.
+ * - [[PlanSubqueries]] or [[PlanAdaptiveSubqueries]] rule does the physical planning of scalar
+ *   subqueries including the ones under [[CommonScalarSubqueriesExec]] node and replaces
+ *   each [[ScalarSubqueryReference]] to their referenced physical plan in
+ *   `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() header))` form.
+ *   It is important that references pointing to the same merged subquery are replaced to the same
+ *   planned instance to make sure that each merged subquery runs only once (even without a wrapping
+ *   [[ReuseSubquery]] node).
+ *   Finally, the [[CommonScalarSubqueriesExec]] node is removed from the physical plan.
+ * - The [[ReuseExchangeAndSubquery]] rule wraps the second, third, ... instances of the same
+ *   subquery into a [[ReuseSubquery]] node, but this just a cosmetic change in the plan.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *          scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * CommonScalarSubqueries [scalar-subquery#250 []]
+ * :  +- Project [named_struct(avg(a), avg(a)#236, sum(b), sum(b)#238L) AS mergedValue#249]
+ * :     +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :        +- Project [a#233, b#234]
+ * :           +- Relation default.t[a#233,b#234] parquet
+ * +- Project [scalarsubqueryreference(0, 0, DoubleType, 231) AS scalarsubquery()#241,
+ *             scalarsubqueryreference(0, 1, LongType, 232) AS scalarsubquery()#242L]
+ *    +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    if (conf.subqueryReuseEnabled) {
+      plan match {
+        case Subquery(_: CommonScalarSubqueries, _) => plan
+        case s: Subquery => s.copy(child = extractCommonScalarSubqueries(s.child))
+        case _: CommonScalarSubqueries => plan
+        case _ => extractCommonScalarSubqueries(plan)
+      }
+    } else {
+      plan
+    }
+  }
+
+  type Header = Project

Review comment:
       I see, thanks for the explanation.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


   **[Test build #139635 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/139635/testReport)** for PR 32298 at commit [`f83f22b`](https://github.com/apache/spark/commit/f83f22bedf4247baea0c48906c4a7399996cd6d8).
    * 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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


   **[Test build #140202 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/140202/testReport)** for PR 32298 at commit [`d10a8be`](https://github.com/apache/spark/commit/d10a8bef691e052b85611f75164f7d21ec4ddc55).
    * This patch **fails to build**.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `public class MergedBlockMetaRequest extends AbstractMessage implements RequestMessage `
     * `public class MergedBlockMetaSuccess extends AbstractResponseMessage `
     * `public abstract class AbstractFetchShuffleBlocks extends BlockTransferMessage `
     * `public class FetchShuffleBlockChunks extends AbstractFetchShuffleBlocks `
     * `public class FetchShuffleBlocks extends AbstractFetchShuffleBlocks `
     * `class IntegralExtensionOps(IntegralOps):`
     * `class FractionalExtensionOps(FractionalOps):`
     * `class StringExtensionOps(StringOps):`
     * `class GroupBy(Generic[T_Frame], metaclass=ABCMeta):`
     * `class DataFrameGroupBy(GroupBy[DataFrame]):`
     * `class SeriesGroupBy(GroupBy[Series]):`
     * `class SparkIndexOpsMethods(Generic[T_IndexOps], metaclass=ABCMeta):`
     * `class SparkSeriesMethods(SparkIndexOpsMethods[\"ps.Series\"]):`
     * `class SparkIndexMethods(SparkIndexOpsMethods[\"ps.Index\"]):`
     * `case class GetTimestampWithoutTZ(`
     * `case class ParseToTimestampWithoutTZ(`
     * `trait TestGroupState[S] extends GroupState[S] `


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


   **[Test build #140206 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/140206/testReport)** for PR 32298 at commit [`d10a8be`](https://github.com/apache/spark/commit/d10a8bef691e052b85611f75164f7d21ec4ddc55).


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


   **[Test build #145957 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/145957/testReport)** for PR 32298 at commit [`fa5e786`](https://github.com/apache/spark/commit/fa5e7860d05e224f42d8005f0ecb2cf89e8d27b0).


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,413 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CommonScalarSubqueries, Filter, Join, LogicalPlan, Project, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s to compute multiple scalar
+ * values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ *   During this first traversal each [[ScalarSubquery]] expression is replaced to a
+ *   [[ScalarSubqueryReference]] pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a results of merging 2 or more
+ *   plans, or it is a plan that was seen only once.
+ *   Merged plans in the cache get a "header" that is is basically
+ *   `CreateNamedStructure(name1, attribute1, name2, attribute2, ...)`
+ *   expression in new root [[Project]] node. This expression ensures that the merged plan is a
+ *   valid scalar subquery that returns only one value.
+ * - A second traversal checks if a [[ScalarSubqueryReference]] is pointing to a merged subquery
+ *   plan or not and either keeps the reference or restores the original [[ScalarSubquery]].
+ *   If there are [[ScalarSubqueryReference]] nodes remained a [[CommonScalarSubqueries]] root node
+ *   is added to the plan with the referenced scalar subqueries.
+ * - [[PlanSubqueries]] or [[PlanAdaptiveSubqueries]] rule does the physical planning of scalar
+ *   subqueries including the ones under [[CommonScalarSubqueriesExec]] node and replaces
+ *   each [[ScalarSubqueryReference]] to their referenced physical plan in
+ *   `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() header))` form.
+ *   It is important that references pointing to the same merged subquery are replaced to the same
+ *   planned instance to make sure that each merged subquery runs only once (even without a wrapping
+ *   [[ReuseSubquery]] node).
+ *   Finally, the [[CommonScalarSubqueriesExec]] node is removed from the physical plan.
+ * - The [[ReuseExchangeAndSubquery]] rule wraps the second, third, ... instances of the same
+ *   subquery into a [[ReuseSubquery]] node, but this just a cosmetic change in the plan.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *          scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * CommonScalarSubqueries [scalar-subquery#250 []]
+ * :  +- Project [named_struct(avg(a), avg(a)#236, sum(b), sum(b)#238L) AS mergedValue#249]
+ * :     +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :        +- Project [a#233, b#234]
+ * :           +- Relation default.t[a#233,b#234] parquet
+ * +- Project [scalarsubqueryreference(0, 0, DoubleType, 231) AS scalarsubquery()#241,
+ *             scalarsubqueryreference(0, 1, LongType, 232) AS scalarsubquery()#242L]
+ *    +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    if (conf.subqueryReuseEnabled) {
+      plan match {
+        case Subquery(_: CommonScalarSubqueries, _) => plan
+        case s: Subquery => s.copy(child = extractCommonScalarSubqueries(s.child))
+        case _: CommonScalarSubqueries => plan
+        case _ => extractCommonScalarSubqueries(plan)
+      }
+    } else {
+      plan
+    }
+  }
+
+  private def extractCommonScalarSubqueries(plan: LogicalPlan) = {
+    // Plan of subqueries and a flag is the plan is merged
+    val cache = ListBuffer.empty[(Project, Boolean)]

Review comment:
       > > is it worthwhile to alias Project to some different type name that hint this is the merged header of scalar query ?
   > 
   > Thanks, I will change it soon.
   
   Fixed in https://github.com/apache/spark/pull/32298/commits/d86d2c48a3e6244fc091ae09cb9377ade98f66b0.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


   **[Test build #139061 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/139061/testReport)** for PR 32298 at commit [`dca324e`](https://github.com/apache/spark/commit/dca324e173a639952172ff87efab762c665cdb8e).


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/InjectRuntimeFilterSuite.scala
##########
@@ -253,20 +253,14 @@ class InjectRuntimeFilterSuite extends QueryTest with SQLTestUtils with SharedSp
   }
 
   def getNumBloomFilters(plan: LogicalPlan): Integer = {
-    val numBloomFilterAggs = plan.collect {
-      case Filter(condition, _) => condition.collect {
-        case subquery: org.apache.spark.sql.catalyst.expressions.ScalarSubquery
-        => subquery.plan.collect {
-          case Aggregate(_, aggregateExpressions, _) =>
-            aggregateExpressions.map {
-              case Alias(AggregateExpression(bfAgg : BloomFilterAggregate, _, _, _, _),
-              _) =>
-                assert(bfAgg.estimatedNumItemsExpression.isInstanceOf[Literal])
-                assert(bfAgg.numBitsExpression.isInstanceOf[Literal])
-                1
-            }.sum
+    val numBloomFilterAggs = plan.collectWithSubqueries {

Review comment:
       Ditto. Let me know it this modified check became too loose.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


   Tests are fine now.


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


   @sigmod,
   - https://github.com/apache/spark/pull/32298/commits/96d0cabef001960ace0b193b01863a8377f78b6f changes the implementation to use CTE nodes.
   - https://github.com/apache/spark/pull/32298/commits/a57ed32588b8df6e3d5bafbba950c96fa48181a9 adds the flag to `CTERelationRef` and changes `WithCTEStrategy` to avoid extra shuffles if the CTE is used as a subquery.


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


   **[Test build #140202 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/140202/testReport)** for PR 32298 at commit [`d10a8be`](https://github.com/apache/spark/commit/d10a8bef691e052b85611f75164f7d21ec4ddc55).


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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






-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


   **[Test build #139232 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/139232/testReport)** for PR 32298 at commit [`f83f22b`](https://github.com/apache/spark/commit/f83f22bedf4247baea0c48906c4a7399996cd6d8).
    * 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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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






-- 
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] sigmod commented on a change in pull request #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LeafNode, LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{MULTI_SCALAR_SUBQUERY, SCALAR_SUBQUERY}
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s into a
+ * [[MultiScalarSubquery]] to compute multiple scalar values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ * - The original [[ScalarSubquery]] expression is replaced to a reference pointing to its cached
+ *   version in this form: `GetStructField(MultiScalarSubquery(SubqueryReference(...)))`.
+ * - A second traversal checks if a [[SubqueryReference]] is pointing to a subquery plan that
+ *   returns multiple values and either replaces only [[SubqueryReference]] to the cached plan or
+ *   restores the whole expression to its original [[ScalarSubquery]] form.
+ * - [[ReuseSubquery]] rule makes sure that merged subqueries are computed once.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *   scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet

Review comment:
       >> In this PR the new MergeScalarSubqueries rule runs in a separate batch 
   >> after column pruning, close to the end of optimization. 
   >> This is by design to make sure no subsequent rule changes the structure 
   
   I don't see how we can guarantee that. I think the hidden, inter-rule dependency can add complexities for future development and maintenance. 
   
   For instance, someone could implement a new Strategy that internally calls ColumnPruning after exploring one logical plan alternatives. By the time such a Strategy is implemented, the authors wouldn't be aware of the fact that ColumnPruning should *not* be called after MergeScalarSubqueries. When they find that's an issue, they would then have to either (a) add some hacks in the Aggregate to mark that MergeScalarSubqueries has been applied and hence ColumnPruning should not go through it, or (b) re-implement the rule per my proposal (1).
     
   I'm wondering whether we can pursue (2) for now, if it meets your need. It's less ambitious but may address most of your issue? 
   
   >> Add a performance improvement to 1. so as to physical plan a merged 
   >> subquery only once. This is your (1) basically.
   
   The performance was not my initial concern, but rather, we'd better make MergeScalarSubqueries self-contained and does not depend on *an assumption that could be changed*.
   
   >>  But I don't think we need lateral views
   
   Sub-querying over arrays are important use cases, for which we don't want to de-correlate. In this case, a subquery is more like an ordinary expression and should be evaluated within the operator 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] SparkQA commented on pull request #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


   Kubernetes integration test unable to build dist.
   
   exiting with code: 1
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/43642/
   


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala
##########
@@ -267,6 +268,33 @@ object ScalarSubquery {
   }
 }
 
+/**
+ * A subquery that is capable to return multiple scalar values.
+ */
+case class MultiScalarSubquery(

Review comment:
       I think you are right.




-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


   **[Test build #143353 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/143353/testReport)** for PR 32298 at commit [`db17431`](https://github.com/apache/spark/commit/db174316bece02ae5e1ca2db2344bfc965022ab2).


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


   **[Test build #143852 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/143852/testReport)** for PR 32298 at commit [`0a97c8b`](https://github.com/apache/spark/commit/0a97c8b8052e80524e98821078df4c55a28b14e6).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `class IndexNameTypeHolder(object):`
     * `        new_class = type(\"NameType\", (NameTypeHolder,), `
     * `        new_class = param.type if isinstance(param, np.dtype) else param`
     * `    (<class 'int'>,)`
     * `    (<class 'int'>, <class 'int'>, <class 'int'>)`
     * `public final class TableIndex `
     * `public abstract class Filter implements Expression, Serializable `
     * `public class ColumnarBatch implements AutoCloseable `
     * `public final class ColumnarBatchRow extends InternalRow `
     * `class IndexAlreadyExistsException(indexName: String, table: Identifier)`
     * `class NoSuchIndexException(indexName: String)`
     * `case class Sec(child: Expression)`
     * `case class Csc(child: Expression)`
     * `trait OperationHelper extends AliasHelper with PredicateHelper `
     * `case class AsOfJoin(`
     * `case class SkewJoinChildWrapper(plan: SparkPlan) extends LeafExecNode `
     * `case class SetCatalogCommand(catalogName: String) extends LeafRunnableCommand `
     * `case class ShowCurrentNamespaceCommand() extends LeafRunnableCommand `
     * `case class WriterBucketSpec(`
     * `case class HashedRelationBroadcastMode(`


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


   **[Test build #140202 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/140202/testReport)** for PR 32298 at commit [`d10a8be`](https://github.com/apache/spark/commit/d10a8bef691e052b85611f75164f7d21ec4ddc55).


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


   **[Test build #143364 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/143364/testReport)** for PR 32298 at commit [`2eb14f1`](https://github.com/apache/spark/commit/2eb14f129498dd7dcd23b43f46f8fc24f95369bd).
    * 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


   **[Test build #143456 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/143456/testReport)** for PR 32298 at commit [`d86d2c4`](https://github.com/apache/spark/commit/d86d2c48a3e6244fc091ae09cb9377ade98f66b0).


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,421 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CommonScalarSubqueries, Filter, Join, LogicalPlan, Project, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s to compute multiple scalar
+ * values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ *   During this first traversal each [[ScalarSubquery]] expression is replaced to a
+ *   [[ScalarSubqueryReference]] pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a results of merging 2 or more
+ *   plans, or it is a plan that was seen only once.
+ *   Merged plans in the cache get a "header", that is is basically
+ *   `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in new root
+ *   [[Project]] node. This expression ensures that the merged plan is a valid scalar subquery that
+ *   returns only one value.
+ * - A second traversal checks if a [[ScalarSubqueryReference]] is pointing to a merged subquery
+ *   plan or not and either keeps the reference or restores the original [[ScalarSubquery]].
+ *   If there are [[ScalarSubqueryReference]] nodes remained a [[CommonScalarSubqueries]] root node
+ *   is added to the plan with the referenced scalar subqueries.
+ * - [[PlanSubqueries]] or [[PlanAdaptiveSubqueries]] rule does the physical planning of scalar
+ *   subqueries including the ones under [[CommonScalarSubqueriesExec]] node and replaces
+ *   each [[ScalarSubqueryReference]] to their referenced physical plan in
+ *   `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() header))` form.
+ *   It is important that references pointing to the same merged subquery are replaced to the same
+ *   planned instance to make sure that each merged subquery runs only once (even without a wrapping
+ *   [[ReuseSubquery]] node).
+ *   Finally, the [[CommonScalarSubqueriesExec]] node is removed from the physical plan.
+ * - The [[ReuseExchangeAndSubquery]] rule wraps the second, third, ... instances of the same
+ *   subquery into a [[ReuseSubquery]] node, but this just a cosmetic change in the plan.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *          scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * CommonScalarSubqueries [scalar-subquery#250 []]
+ * :  +- Project [named_struct(avg(a), avg(a)#236, sum(b), sum(b)#238L) AS mergedValue#249]
+ * :     +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :        +- Project [a#233, b#234]
+ * :           +- Relation default.t[a#233,b#234] parquet
+ * +- Project [scalarsubqueryreference(0, 0, DoubleType, 231) AS scalarsubquery()#241,
+ *             scalarsubqueryreference(0, 1, LongType, 232) AS scalarsubquery()#242L]
+ *    +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    plan match {
+      case Subquery(_: CommonScalarSubqueries, _) => plan
+      case s: Subquery => s.copy(child = extractCommonScalarSubqueries(s.child))
+      case _: CommonScalarSubqueries => plan
+      case _ => extractCommonScalarSubqueries(plan)
+    }
+  }
+
+  /**
+   * An item in the cache of merged scalar subqueries.
+   *
+   * @param elements  List of attributes that form the scalar return value of a merged subquery
+   * @param plan      The plan of a merged scalar subquery
+   * @param merged    A flag to identify if this item is the result of merging subqueries.
+   *                  Please note that `elements.size == 1` doesn't always mean that the plan is not
+   *                  merged as there can be subqueries that are different ([[checkIdenticalPlans]]
+   *                  is false) due to an extra [[Project]] node in one of them. In that case
+   *                  `elements.size` remains 1 after merging, but the merged flag becomes true.
+   */
+  case class Header(elements: Seq[(String, Attribute)], plan: LogicalPlan, merged: Boolean)
+
+  private def extractCommonScalarSubqueries(plan: LogicalPlan) = {
+    val cache = ListBuffer.empty[Header]
+    val newPlan = removeReferences(insertReferences(plan, cache), cache)
+    if (cache.nonEmpty) {
+      val scalarSubqueries = cache.map {
+        case Header(elements, child, _) => ScalarSubquery(createProject(elements, child))
+      }.toSeq
+      CommonScalarSubqueries(scalarSubqueries, newPlan)
+    } else {
+      newPlan
+    }
+  }
+
+  // First traversal builds up the cache and inserts `ScalarSubqueryReference`s to the plan.
+  private def insertReferences(plan: LogicalPlan, cache: ListBuffer[Header]): LogicalPlan = {
+    plan.transformAllExpressionsWithPruning(_.containsAnyPattern(SCALAR_SUBQUERY)) {
+      case s: ScalarSubquery if !s.isCorrelated && s.deterministic =>
+        val (subqueryIndex, headerIndex) = cacheSubquery(s.plan, cache)
+        ScalarSubqueryReference(subqueryIndex, headerIndex, s.dataType, s.exprId)
+    }
+  }
+
+  // Caching returns the index of the subquery in the cache and the index of scalar member in the
+  // "header", that `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in a
+  // [[Project]] node.
+  private def cacheSubquery(plan: LogicalPlan, cache: ListBuffer[Header]): (Int, Int) = {
+    val output = plan.output.head
+    cache.zipWithIndex.collectFirst(Function.unlift { case (header, subqueryIndex) =>
+      checkIdenticalPlans(plan, header.plan).map { outputMap =>
+        val mappedOutput = mapAttributes(output, outputMap)
+        val headerIndex = header.elements.indexWhere {
+          case (_, attribute) => attribute.exprId == mappedOutput.exprId
+        }
+        subqueryIndex -> headerIndex
+      }.orElse(tryMergePlans(plan, header.plan).map {
+        case (mergedPlan, outputMap) =>
+          val mappedOutput = mapAttributes(output, outputMap)
+          var headerIndex = header.elements.indexWhere {
+            case (_, attribute) => attribute.exprId == mappedOutput.exprId
+          }
+          val newHeaderElements = if (headerIndex == -1) {
+            headerIndex = header.elements.size
+            header.elements :+ (output.name -> mappedOutput)
+          } else {
+            header.elements
+          }
+          cache(subqueryIndex) = Header(newHeaderElements, mergedPlan, true)
+          subqueryIndex -> headerIndex
+      })
+    }).getOrElse {
+      cache += Header(Seq(output.name -> output), plan, false)
+      cache.length - 1 -> 0
+    }
+  }
+
+  // If 2 plans are identical return the attribute mapping from the new to the cached version.
+  private def checkIdenticalPlans(
+      newPlan: LogicalPlan,
+      cachedPlan: LogicalPlan): Option[AttributeMap[Attribute]] = {
+    if (newPlan.canonicalized == cachedPlan.canonicalized) {
+      Some(AttributeMap(newPlan.output.zip(cachedPlan.output)))
+    } else {
+      None
+    }
+  }
+
+  // Recursively traverse down and try merging 2 plans. If merge is possible then return the merged
+  // plan with the attribute mapping from the new to the merged version.
+  // Please note that merging arbitrary plans can be complicated, the current version supports only
+  // some of the most important nodes.
+  private def tryMergePlans(
+      newPlan: LogicalPlan,
+      cachedPlan: LogicalPlan): Option[(LogicalPlan, AttributeMap[Attribute])] = {
+    checkIdenticalPlans(newPlan, cachedPlan).map(cachedPlan -> _).orElse(
+      (newPlan, cachedPlan) match {
+        case (np: Project, cp: Project) =>
+          tryMergePlans(np.child, cp.child).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.projectList, outputMap, cp.projectList)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np, cp: Project) =>
+          tryMergePlans(np, cp.child).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.output, outputMap, cp.projectList)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np: Project, cp) =>
+          tryMergePlans(np.child, cp).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.projectList, outputMap, cp.output)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np: Aggregate, cp: Aggregate) if supportedAggregateMerge(np, cp) =>
+          tryMergePlans(np.child, cp.child).flatMap { case (mergedChild, outputMap) =>
+            val mappedNewGroupingExpression =
+              np.groupingExpressions.map(mapAttributes(_, outputMap))
+            // Order of grouping expression doesn't matter so we can compare sets
+            if (ExpressionSet(mappedNewGroupingExpression) ==
+              ExpressionSet(cp.groupingExpressions)) {
+              val (mergedAggregateExpressions, newOutputMap) =
+                mergeNamedExpressions(np.aggregateExpressions, outputMap, cp.aggregateExpressions)
+              val mergedPlan =
+                Aggregate(cp.groupingExpressions, mergedAggregateExpressions, mergedChild)
+              Some(mergedPlan -> newOutputMap)
+            } else {
+              None
+            }
+          }
+
+        // Merging general nodes is complicated and this implementation supports only those nodes in
+        // which the order and the number of output attributes are not relevant (see
+        // `supportedMerge()` whitelist).
+        // Also, this implementation supports only those nodes in which children can be merged in
+        // the same order.
+        case (np, cp) if supportedMerge(np) && np.getClass == cp.getClass &&
+            np.children.size == cp.children.size &&
+            np.expressions.size == cp.expressions.size &&
+            // Fields that don't contain any children or expressions should match
+            np.productIterator.filterNot(np.children.contains)
+              .filter(QueryPlan.extractExpressions(_).isEmpty).toSeq ==
+              cp.productIterator.filterNot(cp.children.contains)
+                .filter(QueryPlan.extractExpressions(_).isEmpty).toSeq =>
+          val merged = np.children.zip(cp.children).map {
+            case (npChild, cpChild) => tryMergePlans(npChild, cpChild)
+          }
+          if (merged.forall(_.isDefined)) {
+            val (mergedChildren, outputMaps) = merged.map(_.get).unzip
+            val outputMap = AttributeMap(outputMaps.map(_.iterator).reduce(_ ++ _).toSeq)
+            // We know that fields that don't contain any children or expressions do match and
+            // children can be merged so we need to test expressions only
+            if (np.expressions.map(mapAttributes(_, outputMap).canonicalized) ==
+              cp.expressions.map(_.canonicalized)) {
+              val mergedPlan = cp.withNewChildren(mergedChildren)
+              Some(mergedPlan -> outputMap)
+            } else {
+              None
+            }
+          } else {
+            None
+          }
+
+        // As a follow-up, it would be possible to merge `CommonScalarSubqueries` nodes, which would
+        // allow merging subqueries with mergee subqueries.

Review comment:
       Thanks, fixed in https://github.com/apache/spark/pull/32298/commits/87ba289589c41a9c83e39c493e4dabf90b95c14a




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


   @tgravescs, thanks for your feedback and interest in in this PR. I reran the tests, they seem to be fine now.
   This change adds basic query merging logic and affects only `q9` from TPCDS, but it brings considerable performance improvement to it.
   This PR has been ready for review for a while, but haven't got much attention. Your and others review would be much appreciated.
   
   I also have some follow-up PRs/plans:
   - to add more advanced logic to support merging aggregate subqueries with different `Filter` nodes
   - support DSv2
   - try merging correlated subqueries
   
   but this PR is complex enough so I would add these in subsequent PRs only.
   
   We can target 3.3, but AFAIK the branch has been cut so probably we need some kind of approval on the dev list?
   


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] sigmod edited a comment on pull request #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

Posted by GitBox <gi...@apache.org>.
sigmod edited a comment on pull request #32298:
URL: https://github.com/apache/spark/pull/32298#issuecomment-1076603246


   Thanks, @peter-toth!
   
   > I have a follow-up PR to support merging different filter predicates with OR, 
   > I just didn't want to make this PR more complex
   > This PR wants to deal with that scope only. 
   
   Definitely. I was just saying `CTERelationRef` can be used in more general cases beyond non-deterministic CTE definitions. Let's not expand the scope of this PR.
   
   > @sigmod, how about doing this kind of transformation?
   
   It looks good to me. IIUC, you want to wrap columns with a struct so that you can execute it as a scalar subquery?
   
   > and adding a flag to cte CTERelationDef that it hosts a scalar query
   
   Sounds good to me.  Will you add a "optimization" rule to add such an "annotation" by looking at the plan holistically, e.g., all consumers of a CTE are simply to pull out a field value?
   
   I'm thinking of the following scenario for future improvements:
   - a non-subquery plan subtree can share the plan structure with scalar subqueries too
   - in this case, the CTE is reused by both subqueries and ordinary plan subtrees
   
   We might also want to make sure MergeSubqueries do not prevent such reuse opportunities down the road.
   
   > + changing WithCTEStrategy a bit to avoid extra shuffles in those cases as 
   > ReuseExchangeAndSubquery can insert ReusedSubqueryExec nodes (no need to insert ReusedExchangeExec).
   
   Will you rewrite the physical plan to change the consumer subqueries to GetStructField?
   


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] sigmod edited a comment on pull request #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

Posted by GitBox <gi...@apache.org>.
sigmod edited a comment on pull request #32298:
URL: https://github.com/apache/spark/pull/32298#issuecomment-1075406139


   One high level comment. 
   
   Does it make sense to implement the rewrite in the following way?
   
   ```
   SELECT
     (SELECT avg(a) FROM t GROUP BY b),
     (SELECT sum(b) FROM t GROUP BY b)
   FROM T
   ```
   
   =>
   ```
   WITH cte AS (SELECT avg(a) a, sum(b) b FROM t GROUP BY b)
   SELECT (SELECT a FROM cte), (SELECT b FROM cte)
   FROM T
   ```
   
   Since we already have [WithCTE](https://github.com/apache/spark/blob/efe43306fcab18f076f755c81c0406ebc1a5fee9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala#L703-L710) and [CTERelationRef](https://github.com/apache/spark/blob/efe43306fcab18f076f755c81c0406ebc1a5fee9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala#L678-L686), the rewrite looks similar to what you want to achieve, while do not need to add yet-another Logical/Exec 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] sigmod commented on pull request #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


   One high level comment. 
   
   Does it make sense to implement the rewrite in the following way?
   
   ```
   SELECT
     (SELECT avg(a) FROM t GROUP BY b),
     (SELECT sum(b) FROM t GROUP BY b)
   FROM T
   ```
   
   =>
   ```
   WITH cte AS (SELECT avg(a) a, sum(b) b FROM t GROUP BY b)
   SELECT (SELECT a FROM cte), (SELECT b FROM cte)
   FROM T
   ```
   
   Since we already have [WithCTE](https://github.com/apache/spark/blob/efe43306fcab18f076f755c81c0406ebc1a5fee9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala#L703-L710) and [CTERelationRef](https://github.com/apache/spark/blob/efe43306fcab18f076f755c81c0406ebc1a5fee9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala#L678-L686), the rewrite looks similar to what you want to achieve, while do not need to add yet-another Logical/Exec 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] sigmod edited a comment on pull request #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

Posted by GitBox <gi...@apache.org>.
sigmod edited a comment on pull request #32298:
URL: https://github.com/apache/spark/pull/32298#issuecomment-1075524713


   > But I think that an extra shuffle could mean performance degradation
   > in case of scalar subqueries (CTEs returning only one row).
   
   Is it still way better than running the scalar subqueries over the same table multiple times?
   I'm more worried about the complexities (i.e., pattern matching cognitive overhead) with new plan nodes like CommonSubqueries and CommonSubqueriesExec. So are other general purpose rules.


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] tgravescs commented on pull request #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


   @peter-toth you said you updated performance numbers, was there a difference after changing 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


   **[Test build #145035 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/145035/testReport)** for PR 32298 at commit [`61f2b34`](https://github.com/apache/spark/commit/61f2b34c49c2f7c76f3ebd320b9d2f927f7ea100).


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,416 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CommonScalarSubqueries, Filter, Join, LogicalPlan, Project, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s to compute multiple scalar
+ * values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ *   During this first traversal each [[ScalarSubquery]] expression is replaced to a
+ *   [[ScalarSubqueryReference]] pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a results of merging 2 or more
+ *   plans, or it is a plan that was seen only once.
+ *   Merged plans in the cache get a "header", that is is basically
+ *   `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in new root
+ *   [[Project]] node. This expression ensures that the merged plan is a valid scalar subquery that
+ *   returns only one value.
+ * - A second traversal checks if a [[ScalarSubqueryReference]] is pointing to a merged subquery
+ *   plan or not and either keeps the reference or restores the original [[ScalarSubquery]].
+ *   If there are [[ScalarSubqueryReference]] nodes remained a [[CommonScalarSubqueries]] root node
+ *   is added to the plan with the referenced scalar subqueries.
+ * - [[PlanSubqueries]] or [[PlanAdaptiveSubqueries]] rule does the physical planning of scalar
+ *   subqueries including the ones under [[CommonScalarSubqueriesExec]] node and replaces
+ *   each [[ScalarSubqueryReference]] to their referenced physical plan in
+ *   `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() header))` form.
+ *   It is important that references pointing to the same merged subquery are replaced to the same
+ *   planned instance to make sure that each merged subquery runs only once (even without a wrapping
+ *   [[ReuseSubquery]] node).
+ *   Finally, the [[CommonScalarSubqueriesExec]] node is removed from the physical plan.
+ * - The [[ReuseExchangeAndSubquery]] rule wraps the second, third, ... instances of the same
+ *   subquery into a [[ReuseSubquery]] node, but this just a cosmetic change in the plan.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *          scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * CommonScalarSubqueries [scalar-subquery#250 []]
+ * :  +- Project [named_struct(avg(a), avg(a)#236, sum(b), sum(b)#238L) AS mergedValue#249]
+ * :     +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :        +- Project [a#233, b#234]
+ * :           +- Relation default.t[a#233,b#234] parquet
+ * +- Project [scalarsubqueryreference(0, 0, DoubleType, 231) AS scalarsubquery()#241,
+ *             scalarsubqueryreference(0, 1, LongType, 232) AS scalarsubquery()#242L]
+ *    +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    if (conf.subqueryReuseEnabled) {
+      plan match {
+        case Subquery(_: CommonScalarSubqueries, _) => plan
+        case s: Subquery => s.copy(child = extractCommonScalarSubqueries(s.child))
+        case _: CommonScalarSubqueries => plan
+        case _ => extractCommonScalarSubqueries(plan)
+      }
+    } else {
+      plan
+    }
+  }
+
+  type Header = Project
+
+  private def extractCommonScalarSubqueries(plan: LogicalPlan) = {
+    // Plan of subqueries and a flag is the plan is merged
+    val cache = ListBuffer.empty[(Header, Boolean)]
+    val newPlan = removeReferences(insertReferences(plan, cache), cache)
+    if (cache.nonEmpty) {
+      val scalarSubqueries = cache.map { case (header, _) => ScalarSubquery(header) }.toSeq
+      CommonScalarSubqueries(scalarSubqueries, newPlan)
+    } else {
+      newPlan
+    }
+  }
+
+  // First traversal builds up the cache and inserts `ScalarSubqueryReference`s to the plan.
+  private def insertReferences(
+      plan: LogicalPlan,
+      cache: ListBuffer[(Header, Boolean)]): LogicalPlan = {
+    plan.transformAllExpressionsWithPruning(_.containsAnyPattern(SCALAR_SUBQUERY)) {
+      case s: ScalarSubquery if s.children.isEmpty =>
+        val (subqueryIndex, headerIndex) = cacheSubquery(s.plan, cache)
+        ScalarSubqueryReference(subqueryIndex, headerIndex, s.dataType, s.exprId)
+    }
+  }
+
+  // Caching returns the index of the subquery in the cache and the index of scalar member in the
+  // "header", that `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in a
+  // [[Project]] node.
+  private def cacheSubquery(
+      plan: LogicalPlan,
+      cache: ListBuffer[(Header, Boolean)]): (Int, Int) = {
+    val firstOutput = plan.output.head
+    cache.zipWithIndex.collectFirst(Function.unlift { case ((header, merged), subqueryIndex) =>
+      checkIdenticalPlans(plan, header.child)
+        .map((subqueryIndex, header, header.child, _, merged))
+        .orElse(tryMergePlans(plan, header.child).map {
+          case (mergedPlan, outputMap) => (subqueryIndex, header, mergedPlan, outputMap, true)
+        })
+    }).map { case (subqueryIndex, header, mergedPlan, outputMap, merged) =>
+      val mappedFirstOutput = mapAttributes(firstOutput, outputMap)
+      val headerElements = getHeaderElements(header)
+      var headerIndex = headerElements.indexWhere {
+        case (_, attribute) => attribute.exprId == mappedFirstOutput.exprId
+      }
+      if (headerIndex == -1) {
+        val newHeaderElements = headerElements :+ (Literal(firstOutput.name) -> mappedFirstOutput)
+        cache(subqueryIndex) = createHeader(newHeaderElements, mergedPlan) -> merged
+        headerIndex = headerElements.size
+      }
+      subqueryIndex -> headerIndex
+    }.getOrElse {
+      cache += createHeader(Seq(Literal(firstOutput.name) -> firstOutput), plan) -> false
+      cache.length - 1 -> 0
+    }
+  }
+
+  // If 2 plans are identical return the attribute mapping from the new to the cached version.
+  private def checkIdenticalPlans(newPlan: LogicalPlan, cachedPlan: LogicalPlan) = {
+    if (newPlan.canonicalized == cachedPlan.canonicalized) {
+      Some(AttributeMap(newPlan.output.zip(cachedPlan.output)))
+    } else {
+      None
+    }
+  }
+
+  // Recursively traverse down and try merging 2 plans. If merge is possible then return the merged
+  // plan with the attribute mapping from the new to the merged version.
+  // Please note that merging arbitrary plans can be complicated, the current version supports only
+  // some of the most important nodes.
+  private def tryMergePlans(
+      newPlan: LogicalPlan,
+      cachedPlan: LogicalPlan): Option[(LogicalPlan, AttributeMap[Attribute])] = {
+    checkIdenticalPlans(newPlan, cachedPlan).map(cachedPlan -> _).orElse(
+      (newPlan, cachedPlan) match {
+        case (np: Project, cp: Project) =>
+          tryMergePlans(np.child, cp.child).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.projectList, outputMap, cp.projectList)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np, cp: Project) =>
+          tryMergePlans(np, cp.child).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.output, outputMap, cp.projectList)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np: Project, cp) =>
+          tryMergePlans(np.child, cp).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.projectList, outputMap, cp.output)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np: Aggregate, cp: Aggregate) if supportedAggregateMerge(np, cp) =>
+          tryMergePlans(np.child, cp.child).flatMap { case (mergedChild, outputMap) =>
+            val mappedNewGroupingExpression =
+              np.groupingExpressions.map(mapAttributes(_, outputMap))
+            if (ExpressionSet(mappedNewGroupingExpression) ==
+              ExpressionSet(cp.groupingExpressions)) {
+              val (mergedAggregateExpressions, newOutputMap) =
+                mergeNamedExpressions(np.aggregateExpressions, outputMap, cp.aggregateExpressions)
+              val mergedPlan =
+                Aggregate(cp.groupingExpressions, mergedAggregateExpressions, mergedChild)
+              Some(mergedPlan -> newOutputMap)
+            } else {
+              None
+            }
+          }
+
+        // Merging general nodes is complicated and this implementation supports only those nodes in
+        // which the order and the number of output attributes are not relevant (see
+        // `supportedMerge()` whitelist).
+        // Also, this implementation supports only those nodes in which children can be merged in
+        // the same order.
+        case (np, cp) if supportedMerge(np) && np.getClass == cp.getClass &&
+          np.children.size == cp.children.size =>
+          val merged = np.children.zip(cp.children).map {
+            case (npChild, cpChild) => tryMergePlans(npChild, cpChild)
+          }
+          if (merged.forall(_.isDefined)) {
+            val (mergedChildren, outputMaps) = merged.map(_.get).unzip
+            val outputMap = AttributeMap(outputMaps.map(_.iterator).reduce(_ ++ _).toSeq)
+            val mappedNewPlan = mapAttributes(np.withNewChildren(mergedChildren), outputMap)
+            val mergedPlan = cp.withNewChildren(mergedChildren)
+            if (mappedNewPlan.canonicalized == mergedPlan.canonicalized) {
+              Some(mergedPlan -> outputMap)
+            } else {
+              None
+            }
+          } else {
+            None
+          }
+
+        // As a follow-up, it would be possible to merge `CommonScalarSubqueries` nodes, which would
+        // allow merging subqueries with mergee subqueries.
+        // E.g. this query:
+        //
+        // SELECT
+        //   (
+        //     SELECT
+        //       (SELECT avg(a) FROM t GROUP BY b) +
+        //       (SELECT sum(b) FROM t GROUP BY b)
+        //   ),
+        //   (
+        //     SELECT
+        //       (SELECT max(a) FROM t GROUP BY b) +
+        //       (SELECT min(b) FROM t GROUP BY b)
+        //   )
+        //
+        // is currently optimized to:
+        //
+        // == Optimized Logical Plan ==
+        // Project [scalar-subquery#233 [] AS scalarsubquery()#255,
+        //          scalar-subquery#236 [] AS scalarsubquery()#256]
+        // :  :- CommonScalarSubqueries [scalar-subquery#264 []]
+        // :  :  :  +- Aggregate [b#238], [named_struct(avg(a), avg(a#237), sum(b), sum(b#238))
+        //                                 AS mergedValue#263]
+        // :  :  :     +- Relation default.t[a#237,b#238] parquet
+        // :  :  +- Project [(scalarsubqueryreference(0, 0, DoubleType, 231) +
+        //                   cast(scalarsubqueryreference(0, 1, LongType, 232) as double))
+        //                   AS (scalarsubquery() + scalarsubquery())#245]
+        // :  :     +- OneRowRelation
+        // :  +- CommonScalarSubqueries [scalar-subquery#269 []]
+        // :     :  +- Aggregate [b#254], [named_struct(min(a), min(a#253), max(b), max(b#254))
+        //                                 AS mergedValue#268]
+        // :     :     +- Relation default.t[a#253,b#254] parquet
+        // :     +- Project [(scalarsubqueryreference(0, 0, IntegerType, 234) +
+        //                   scalarsubqueryreference(0, 1, IntegerType, 235))
+        //                   AS (scalarsubquery() + scalarsubquery())#252]
+        // :        +- OneRowRelation
+        // +- OneRowRelation
+        //
+        // but if we implemented merging `CommonScalarSubqueries` nodes then the plan could be
+        // transformed further and all leaf subqueries could be merged.
+
+        // Otherwise merging is not possible.
+        case _ => None
+      })
+  }
+
+  private def createHeader(headerElements: Seq[(Literal, Attribute)], plan: LogicalPlan): Header = {
+    Project(
+      Seq(Alias(
+        CreateNamedStruct(headerElements.flatMap {
+          case (name, attribute) => Seq(name, attribute)
+        }),
+        "mergedValue")()),
+      plan)
+  }
+
+  private def getHeaderElements(header: Header) = {
+    val mergedValue =
+      header.projectList.head.asInstanceOf[Alias].child.asInstanceOf[CreateNamedStruct]
+    mergedValue.children.grouped(2).map {
+      case Seq(name: Literal, attribute: Attribute) => name -> attribute
+    }.toSeq
+  }
+
+  private def mapAttributes[T <: Expression](expr: T, outputMap: AttributeMap[Attribute]) = {
+    expr.transform {
+      case a: Attribute => outputMap.getOrElse(a, a)
+    }.asInstanceOf[T]
+  }
+
+  private def mapAttributes(plan: LogicalPlan, outputMap: AttributeMap[Attribute]) = {
+    plan.transformExpressions {
+      case a: Attribute => outputMap.getOrElse(a, a)
+    }
+  }
+
+  // Applies `outputMap` attribute mapping on elements of `newExpressions` and merges them into
+  // `cachedExpressions`. Returns the merged expressions and the attribute mapping from the new to
+  // the merged version that can be propagated up during merging nodes.
+  private def mergeNamedExpressions(
+      newExpressions: Seq[NamedExpression],
+      outputMap: AttributeMap[Attribute],
+      cachedExpressions: Seq[NamedExpression]) = {
+    val mergedExpressions = ListBuffer[NamedExpression](cachedExpressions: _*)
+    val newOutputMap = AttributeMap(newExpressions.map { ne =>
+      val mapped = mapAttributes(ne, outputMap)
+      val withoutAlias = mapped match {
+        case Alias(child, _) => child
+        case e => e
+      }
+      ne.toAttribute -> mergedExpressions.find {
+        case Alias(child, _) => child semanticEquals withoutAlias
+        case e => e semanticEquals withoutAlias
+      }.getOrElse {
+        mergedExpressions += mapped
+        mapped
+      }.toAttribute
+    })
+    (mergedExpressions.toSeq, newOutputMap)
+  }
+
+  // Merging different aggregate implementations could cause performance regression
+  private def supportedAggregateMerge(newPlan: Aggregate, cachedPlan: Aggregate) = {
+    val newPlanAggregateExpressions = newPlan.aggregateExpressions.flatMap(_.collect {
+      case a: AggregateExpression => a
+    })
+    val cachedPlanAggregateExpressions = cachedPlan.aggregateExpressions.flatMap(_.collect {
+      case a: AggregateExpression => a
+    })
+    val newPlanSupportsHashAggregate = Aggregate.supportsHashAggregate(
+      newPlanAggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes))
+    val cachedPlanSupportsHashAggregate = Aggregate.supportsHashAggregate(
+      cachedPlanAggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes))
+    newPlanSupportsHashAggregate && cachedPlanSupportsHashAggregate ||
+      !newPlanSupportsHashAggregate && !cachedPlanSupportsHashAggregate && {
+        val newPlanSupportsObjectHashAggregate =
+          Aggregate.supportsObjectHashAggregate(newPlanAggregateExpressions)
+        val cachedPlanSupportsObjectHashAggregate =
+          Aggregate.supportsObjectHashAggregate(cachedPlanAggregateExpressions)
+        newPlanSupportsObjectHashAggregate && cachedPlanSupportsObjectHashAggregate ||
+          !newPlanSupportsObjectHashAggregate && !cachedPlanSupportsObjectHashAggregate
+      }
+  }
+
+  // Whitelist of mergeable general nodes
+  private def supportedMerge(plan: LogicalPlan) = {
+    plan match {
+      case _: Filter => true
+      case _: Join => true
+      case _ => false
+    }
+  }
+
+  // Second traversal replaces `ScalarSubqueryReference`s to either
+  // `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() header))` if the plan is
+  // merged from multiple subqueries or `ScalarSubquery(original plan)` it it isn't.

Review comment:
       Thanks, fixed in: https://github.com/apache/spark/pull/32298/commits/63c370966577985ebfde03b4bd309fda85ca909b




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


   **[Test build #145822 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/145822/testReport)** for PR 32298 at commit [`3e8f7fa`](https://github.com/apache/spark/commit/3e8f7faa79272d2149cb29ce62700b942c83d864).


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


   **[Test build #146071 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/146071/testReport)** for PR 32298 at commit [`96a502d`](https://github.com/apache/spark/commit/96a502d6f4ed44ed56a92f636f793d365dfa69d3).
    * 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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






-- 
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] zinking commented on a change in pull request #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,413 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CommonScalarSubqueries, Filter, Join, LogicalPlan, Project, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s to compute multiple scalar
+ * values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ *   During this first traversal each [[ScalarSubquery]] expression is replaced to a
+ *   [[ScalarSubqueryReference]] pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a results of merging 2 or more
+ *   plans, or it is a plan that was seen only once.
+ *   Merged plans in the cache get a "header" that is is basically
+ *   `CreateNamedStructure(name1, attribute1, name2, attribute2, ...)`
+ *   expression in new root [[Project]] node. This expression ensures that the merged plan is a
+ *   valid scalar subquery that returns only one value.
+ * - A second traversal checks if a [[ScalarSubqueryReference]] is pointing to a merged subquery
+ *   plan or not and either keeps the reference or restores the original [[ScalarSubquery]].

Review comment:
       ok understood




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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






-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LeafNode, LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{MULTI_SCALAR_SUBQUERY, SCALAR_SUBQUERY}
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s into a
+ * [[MultiScalarSubquery]] to compute multiple scalar values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ * - The original [[ScalarSubquery]] expression is replaced to a reference pointing to its cached
+ *   version in this form: `GetStructField(MultiScalarSubquery(SubqueryReference(...)))`.
+ * - A second traversal checks if a [[SubqueryReference]] is pointing to a subquery plan that
+ *   returns multiple values and either replaces only [[SubqueryReference]] to the cached plan or
+ *   restores the whole expression to its original [[ScalarSubquery]] form.
+ * - [[ReuseSubquery]] rule makes sure that merged subqueries are computed once.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *   scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet

Review comment:
       > The proposed rule augments two subqueries, makes them look identical, and hopes (a) column-pruning doesn't prune too aggressively and (b) physical de-dup could dedup them. In case (a) changes later and the two aggregate trees are not deduped in the physical plan, there could potentially be regressions -- each aggregation then becomes more expensive.
   
   In this PR the new `MergeScalarSubqueries` rule runs in a separate batch after column pruning, close to the end of optimization. This is by design to make sure no subsequent rule changes the structure of different instances of a merged subquery plan at different places in the logical plan differently. So the physical planing creates the same physical plan for these instances and there shouldn't be any dedup issues.
   
   I think probably the downside of my current PR is that the physical planning of merged subqueries happen multiple times (as many times as they they appear in the logical plan) and physical dedup comes only after that. This could be improved if we had subquery references in logical plan as well (something like `ReuseSubqueryExec`). But I think that's what your (1) is about. Move the merged subqueries to a special top logical plan node and add subquery references at places where they are actually used.
   
   > SELECT y
   FROM LATERAL VIEW explode(ARRAY(ARRAY(1), ARRAY(1, 2), ARRAY(1, 2, 3))) AS y
   WHERE
   ( SELECT COUNT(*) FROM LATERAL VIEW explode(y) AS element ) > 1
   AND
   ( SELECT SUM(element) FROM LATERAL VIEW explode(y) AS element ) > 3
   I noticed that such subqueries do not work for now. But they align with the language spec and has well defined semantics. Once we support them, we want your proposed rule to be able to speedup them as well.
   
   Ah ok, but what should be the optimized plan of that query? This looks like we have 2 correlated subqueries and (2) makes perfect sense to merge them. But I don't think we need lateral views, just take the following query:
   ```
   SELECT
     (SELECT avg(a) FROM t WHERE t.a = outer.a),
     (SELECT sum(b) FROM t WHERE t.a = outer.a)
   FROM t AS outer
   ```
   which is
   ```
   Project [scalar-subquery#231 [a#233] AS scalarsubquery(a)#243, scalar-subquery#232 [a#233] AS scalarsubquery(a)#244L]
   :  :- Aggregate [avg(a#239) AS avg(a)#236]
   :  :  +- Filter (a#239 = outer(a#233))
   :  :     +- SubqueryAlias spark_catalog.default.t
   :  :        +- Relation default.t[a#239,b#240] parquet
   :  +- Aggregate [sum(b#242) AS sum(b)#238L]
   :     +- Filter (a#241 = outer(a#233))
   :        +- SubqueryAlias spark_catalog.default.t
   :           +- Relation default.t[a#241,b#242] parquet
   +- SubqueryAlias outer
      +- SubqueryAlias spark_catalog.default.t
         +- Relation default.t[a#233,b#234] parquet
   ```
   / 
   ```
   Project [avg(a)#236 AS scalarsubquery(a)#243, sum(b)#238L AS scalarsubquery(a)#244L]
   +- Join LeftOuter, (a#241 = a#233)
      :- Project [a#233, avg(a)#236]
      :  +- Join LeftOuter, (a#239 = a#233)
      :     :- Project [a#233]
      :     :  +- Relation default.t[a#233,b#234] parquet
      :     +- Aggregate [a#239], [avg(a#239) AS avg(a)#236, a#239]
      :        +- Project [a#239]
      :           +- Filter isnotnull(a#239)
      :              +- Relation default.t[a#239,b#240] parquet
      +- Aggregate [a#241], [sum(b#242) AS sum(b)#238L, a#241]
         +- Filter isnotnull(a#241)
            +- Relation default.t[a#241,b#242] parquet
   ```
   now, and this PR doesn't help at all, but it could be optimized using your (2).
   
   I wonder the following steps (tickets/PRs) would make sense:
   
   1. Finish this PR and support only on non-correlated mergeable subqueries. Mainly focus on merging plans and keep the physical reuse dependency for simplicity. This supports subquery merging within a plan regardless they are in the same logical node.
   2. Add a performance improvement to 1. so as to physical plan a merged subquery only once. This is your (1) basically. Move the merged subqueries to a top node and introduce subquery references in logical plan.
   3. Add support for correlated subqueries using your (2). As you mentioned this will only support subqueries within the same logical node.
   
   Probably we should implement separate rules for 1. + 2. and 3. but the plan merging logic can be common.




-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,413 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CommonScalarSubqueries, Filter, Join, LogicalPlan, Project, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s to compute multiple scalar
+ * values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ *   During this first traversal each [[ScalarSubquery]] expression is replaced to a
+ *   [[ScalarSubqueryReference]] pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a results of merging 2 or more
+ *   plans, or it is a plan that was seen only once.
+ *   Merged plans in the cache get a "header" that is is basically
+ *   `CreateNamedStructure(name1, attribute1, name2, attribute2, ...)`
+ *   expression in new root [[Project]] node. This expression ensures that the merged plan is a
+ *   valid scalar subquery that returns only one value.
+ * - A second traversal checks if a [[ScalarSubqueryReference]] is pointing to a merged subquery
+ *   plan or not and either keeps the reference or restores the original [[ScalarSubquery]].
+ *   If there are [[ScalarSubqueryReference]] nodes remained a [[CommonScalarSubqueries]] root node
+ *   is added to the plan with the referenced scalar subqueries.
+ * - [[PlanSubqueries]] or [[PlanAdaptiveSubqueries]] rule does the physical planning of scalar
+ *   subqueries including the ones under [[CommonScalarSubqueriesExec]] node and replaces
+ *   each [[ScalarSubqueryReference]] to their referenced physical plan in
+ *   `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() header))` form.
+ *   It is important that references pointing to the same merged subquery are replaced to the same
+ *   planned instance to make sure that each merged subquery runs only once (even without a wrapping
+ *   [[ReuseSubquery]] node).
+ *   Finally, the [[CommonScalarSubqueriesExec]] node is removed from the physical plan.
+ * - The [[ReuseExchangeAndSubquery]] rule wraps the second, third, ... instances of the same
+ *   subquery into a [[ReuseSubquery]] node, but this just a cosmetic change in the plan.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *          scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * CommonScalarSubqueries [scalar-subquery#250 []]
+ * :  +- Project [named_struct(avg(a), avg(a)#236, sum(b), sum(b)#238L) AS mergedValue#249]
+ * :     +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :        +- Project [a#233, b#234]
+ * :           +- Relation default.t[a#233,b#234] parquet
+ * +- Project [scalarsubqueryreference(0, 0, DoubleType, 231) AS scalarsubquery()#241,
+ *             scalarsubqueryreference(0, 1, LongType, 232) AS scalarsubquery()#242L]
+ *    +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    if (conf.subqueryReuseEnabled) {
+      plan match {
+        case Subquery(_: CommonScalarSubqueries, _) => plan
+        case s: Subquery => s.copy(child = extractCommonScalarSubqueries(s.child))
+        case _: CommonScalarSubqueries => plan
+        case _ => extractCommonScalarSubqueries(plan)
+      }
+    } else {
+      plan
+    }
+  }
+
+  private def extractCommonScalarSubqueries(plan: LogicalPlan) = {
+    // Plan of subqueries and a flag is the plan is merged
+    val cache = ListBuffer.empty[(Project, Boolean)]
+    val newPlan = removeReferences(insertReferences(plan, cache), cache)
+    if (cache.nonEmpty) {
+      val scalarSubqueries = cache.map { case (header, _) => ScalarSubquery(header) }.toSeq
+      CommonScalarSubqueries(scalarSubqueries, newPlan)
+    } else {
+      newPlan
+    }
+  }
+
+  // First traversal builds up the cache and inserts `ScalarSubqueryReference`s to the plan.
+  private def insertReferences(
+      plan: LogicalPlan,
+      cache: ListBuffer[(Project, Boolean)]): LogicalPlan = {
+    plan.transformAllExpressionsWithPruning(_.containsAnyPattern(SCALAR_SUBQUERY)) {
+      case s: ScalarSubquery if s.children.isEmpty =>
+        val (subqueryIndex, headerIndex) = cacheSubquery(s.plan, cache)
+        ScalarSubqueryReference(subqueryIndex, headerIndex, s.dataType, s.exprId)
+    }
+  }
+
+  // Caching returns the index of the subquery in the cache and the index of scalar member in the
+  // `CreateNamedStruct` header.
+  private def cacheSubquery(
+      plan: LogicalPlan,
+      cache: ListBuffer[(Project, Boolean)]): (Int, Int) = {
+    val firstOutput = plan.output.head
+    cache.zipWithIndex.collectFirst(Function.unlift { case ((header, merged), subqueryIndex) =>
+      checkIdenticalPlans(plan, header.child)
+        .map((subqueryIndex, header, header.child, _, merged))
+        .orElse(tryMergePlans(plan, header.child).map {
+          case (mergedPlan, outputMap) => (subqueryIndex, header, mergedPlan, outputMap, true)
+        })
+    }).map { case (subqueryIndex, header, mergedPlan, outputMap, merged) =>
+      val mappedFirstOutput = mapAttributes(firstOutput, outputMap)
+      val headerElements = getHeaderElements(header)
+      var headerIndex = headerElements.indexWhere {
+        case (_, attribute) => attribute.exprId == mappedFirstOutput.exprId
+      }
+      if (headerIndex == -1) {
+        val newHeaderElements = headerElements :+ (Literal(firstOutput.name) -> mappedFirstOutput)
+        cache(subqueryIndex) = createHeader(newHeaderElements, mergedPlan) -> merged
+        headerIndex = headerElements.size
+      }
+      subqueryIndex -> headerIndex
+    }.getOrElse {
+      cache += createHeader(Seq(Literal(firstOutput.name) -> firstOutput), plan) -> false
+      cache.length - 1 -> 0
+    }
+  }
+
+  // If 2 plans are identical return the attribute mapping from the new to the cached version.
+  private def checkIdenticalPlans(newPlan: LogicalPlan, cachedPlan: LogicalPlan) = {
+    if (newPlan.canonicalized == cachedPlan.canonicalized) {
+      Some(AttributeMap(newPlan.output.zip(cachedPlan.output)))
+    } else {
+      None
+    }
+  }
+
+  // Recursively traverse down and try merging 2 plans. If merge is possible then return the merged
+  // plan with the attribute mapping from the new to the merged version.
+  // Please note that merging arbitrary plans can be complicated, the current version supports only
+  // some of the most important nodes.
+  private def tryMergePlans(
+      newPlan: LogicalPlan,
+      cachedPlan: LogicalPlan): Option[(LogicalPlan, AttributeMap[Attribute])] = {
+    checkIdenticalPlans(newPlan, cachedPlan).map(cachedPlan -> _).orElse(
+      (newPlan, cachedPlan) match {
+        case (np: Project, cp: Project) =>
+          tryMergePlans(np.child, cp.child).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.projectList, outputMap, cp.projectList)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np, cp: Project) =>
+          tryMergePlans(np, cp.child).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.output, outputMap, cp.projectList)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np: Project, cp) =>
+          tryMergePlans(np.child, cp).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.projectList, outputMap, cp.output)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np: Aggregate, cp: Aggregate) if supportedAggregateMerge(np, cp) =>
+          tryMergePlans(np.child, cp.child).flatMap { case (mergedChild, outputMap) =>
+            val mappedNewGroupingExpression =
+              np.groupingExpressions.map(mapAttributes(_, outputMap))
+            if (ExpressionSet(mappedNewGroupingExpression) ==
+              ExpressionSet(cp.groupingExpressions)) {
+              val (mergedAggregateExpressions, newOutputMap) =
+                mergeNamedExpressions(np.aggregateExpressions, outputMap, cp.aggregateExpressions)
+              val mergedPlan =
+                Aggregate(cp.groupingExpressions, mergedAggregateExpressions, mergedChild)
+              Some(mergedPlan -> newOutputMap)
+            } else {
+              None
+            }
+          }
+
+        // Merging general nodes is complicated and this implementation supports only those nodes in
+        // which the order and the number of output attributes are not relevant (see
+        // `supportedMerge()` whitelist).
+        // Also, this implementation supports only those nodes in which children can be merged in
+        // the same order.
+        case (np, cp) if supportedMerge(np) && np.getClass == cp.getClass &&
+          np.children.size == cp.children.size =>
+          val merged = np.children.zip(cp.children).map {
+            case (npChild, cpChild) => tryMergePlans(npChild, cpChild)
+          }
+          if (merged.forall(_.isDefined)) {
+            val (mergedChildren, outputMaps) = merged.map(_.get).unzip
+            val outputMap = AttributeMap(outputMaps.map(_.iterator).reduce(_ ++ _).toSeq)
+            val mappedNewPlan = mapAttributes(np.withNewChildren(mergedChildren), outputMap)
+            val mergedPlan = cp.withNewChildren(mergedChildren)
+            if (mappedNewPlan.canonicalized == mergedPlan.canonicalized) {
+              Some(mergedPlan -> outputMap)
+            } else {
+              None
+            }
+          } else {
+            None
+          }
+
+        // As a follow-up, it would be possible to merge `CommonScalarSubqueries` nodes, which would

Review comment:
       I don't think it is straightforward to refer to a subquery in a common root node of the whole plan, from another subquery, but it is probably doable. Unfortunately, even this simpler change hasn't got much reviews...
   
   Anyway, I will update this PR soon to resolve the conflicts.
    




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LeafNode, LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{MULTI_SCALAR_SUBQUERY, SCALAR_SUBQUERY}
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s into a
+ * [[MultiScalarSubquery]] to compute multiple scalar values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ * - The original [[ScalarSubquery]] expression is replaced to a reference pointing to its cached
+ *   version in this form: `GetStructField(MultiScalarSubquery(SubqueryReference(...)))`.
+ * - A second traversal checks if a [[SubqueryReference]] is pointing to a subquery plan that
+ *   returns multiple values and either replaces only [[SubqueryReference]] to the cached plan or
+ *   restores the whole expression to its original [[ScalarSubquery]] form.
+ * - [[ReuseSubquery]] rule makes sure that merged subqueries are computed once.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *   scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet

Review comment:
       > If for some reason ReuseSubquery does not trigger, applying (2) does not make the plan worse than not applying (2).
   
   In the above example the final optimized plan of (2) is the very same as with this PR. There are 2 aggregates in both subqueries so without dedup both (2) and this PR could cause regressions.
   I agree that (2) is self-contained and this PR is not, but IMO it looks like there are inter-rule dependencies currently in Spark (like `PushDownPredicates` relies on `ReuseSubquery`) that overall doesn't make (2) safer than this PR.
   
   I think this means that your (1) suggestion is probably the right approach and we need to move common non-correlated subqueries to a top node and reference to them in logical plan.
   
   I also think that (2) is a good improvement for correlated subqueries, but I would pursue (1) in this PR first and maybe (2) in a separate one. Does this sound acceptable?
   
   @cloud-fan, @maropu do you have any thoughts on this topic?




-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


   @cloud-fan, @maropu, @sigmod, I've updated this PR with the (1) suggestion: https://github.com/apache/spark/pull/32298#discussion_r627800694 and also added tests and comments.
   


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


   I move this back to WIP for now:
   - to discuss if (1) in https://github.com/apache/spark/pull/32298#discussion_r627800694 is acceptable approach and
   - to fix a few issues with `tryMergePlans()`


-- 
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] sigmod commented on a change in pull request #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LeafNode, LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{MULTI_SCALAR_SUBQUERY, SCALAR_SUBQUERY}
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s into a
+ * [[MultiScalarSubquery]] to compute multiple scalar values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ * - The original [[ScalarSubquery]] expression is replaced to a reference pointing to its cached
+ *   version in this form: `GetStructField(MultiScalarSubquery(SubqueryReference(...)))`.
+ * - A second traversal checks if a [[SubqueryReference]] is pointing to a subquery plan that
+ *   returns multiple values and either replaces only [[SubqueryReference]] to the cached plan or
+ *   restores the whole expression to its original [[ScalarSubquery]] form.
+ * - [[ReuseSubquery]] rule makes sure that merged subqueries are computed once.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *   scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet

Review comment:
       >  I would pursue (1) in this PR first and maybe (2) in a separate one. Does this sound acceptable?
   
   Yeah, that sounds great. Thanks a lot, @peter-toth!
   
   > There are 2 aggregates in both subqueries so without dedup both (2) and this PR could cause regressions.
   
   IIUC, I think it sounds like an existing bug (or missing feature) for struct subfield pruning, which could be blocking (2) but is orthogonal to (2). For instance, if I write your example join query manually, I'd expect the struct subfield pruning to happen to the struct constructor, regardless of the existence of subqueries.
   
   > I've never seen such transformations in SparkStrategys.
   
   It's not uncommon in exploration Strategies such as index selection, common subplan dedup etc., when we substitute the subtree of a tree node T with another subtree (from somewhere else in the plan or a different access path) that may contain unneeded columns for T.  Spark doesn't have those strategies for now, but I'll not be surprised if we add them down the road.
   
   




-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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] sigmod commented on a change in pull request #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LeafNode, LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{MULTI_SCALAR_SUBQUERY, SCALAR_SUBQUERY}
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s into a
+ * [[MultiScalarSubquery]] to compute multiple scalar values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ * - The original [[ScalarSubquery]] expression is replaced to a reference pointing to its cached
+ *   version in this form: `GetStructField(MultiScalarSubquery(SubqueryReference(...)))`.
+ * - A second traversal checks if a [[SubqueryReference]] is pointing to a subquery plan that
+ *   returns multiple values and either replaces only [[SubqueryReference]] to the cached plan or
+ *   restores the whole expression to its original [[ScalarSubquery]] form.
+ * - [[ReuseSubquery]] rule makes sure that merged subqueries are computed once.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *   scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet

Review comment:
       >> In this PR the new MergeScalarSubqueries rule runs in a separate batch 
   >> after column pruning, close to the end of optimization. 
   >> This is by design to make sure no subsequent rule changes the structure 
   
   I don't see how we can guarantee that. I think the hidden, inter-rule dependency can add complexities for future development and maintenance. 
   
   For instance, someone could implement a new Strategy that internally calls ColumnPruning after exploring one logical plan alternative. By the time such a Strategy is implemented, the authors wouldn't be aware of the fact that ColumnPruning should *not* be called after MergeScalarSubqueries. When they find that's an issue, they would then have to either (a) add some hacks in the Aggregate to mark that MergeScalarSubqueries has been applied and hence ColumnPruning should not go through it, or (b) re-implement the rule per my proposal (1).
     
   I'm wondering whether we can pursue (2) for now, if it meets your need. It's less ambitious but may address most of your issue? 
   
   >> Add a performance improvement to 1. so as to physical plan a merged 
   >> subquery only once. This is your (1) basically.
   
   The performance was not my initial concern, but rather, we'd better make MergeScalarSubqueries self-contained and does not depend on *an assumption that could be changed*.
   
   >>  But I don't think we need lateral views
   
   Sub-querying over arrays are important use cases, for which we don't want to de-correlate. In this case, a subquery is more like an ordinary expression and should be evaluated within the operator 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 commented on pull request #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


   **[Test build #138260 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138260/testReport)** for PR 32298 at commit [`1f2f75c`](https://github.com/apache/spark/commit/1f2f75c9c52f7494db4e3bb15368c87d9b8e17c7).


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


   **[Test build #145342 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/145342/testReport)** for PR 32298 at commit [`cc8690e`](https://github.com/apache/spark/commit/cc8690edf34297b1f4948f0221963d21eccffa5a).
    * 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,422 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CommonScalarSubqueries, Filter, Join, LogicalPlan, Project, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s to compute multiple scalar
+ * values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ *   During this first traversal each [[ScalarSubquery]] expression is replaced to a
+ *   [[ScalarSubqueryReference]] pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a results of merging 2 or more
+ *   plans, or it is a plan that was seen only once.
+ *   Merged plans in the cache get a "header", that is is basically
+ *   `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in new root
+ *   [[Project]] node. This expression ensures that the merged plan is a valid scalar subquery that
+ *   returns only one value.
+ * - A second traversal checks if a [[ScalarSubqueryReference]] is pointing to a merged subquery
+ *   plan or not and either keeps the reference or restores the original [[ScalarSubquery]].
+ *   If there are [[ScalarSubqueryReference]] nodes remained a [[CommonScalarSubqueries]] root node
+ *   is added to the plan with the referenced scalar subqueries.
+ * - [[PlanSubqueries]] or [[PlanAdaptiveSubqueries]] rule does the physical planning of scalar
+ *   subqueries including the ones under [[CommonScalarSubqueriesExec]] node and replaces
+ *   each [[ScalarSubqueryReference]] to their referenced physical plan in
+ *   `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() header))` form.
+ *   It is important that references pointing to the same merged subquery are replaced to the same
+ *   planned instance to make sure that each merged subquery runs only once (even without a wrapping
+ *   [[ReuseSubquery]] node).
+ *   Finally, the [[CommonScalarSubqueriesExec]] node is removed from the physical plan.
+ * - The [[ReuseExchangeAndSubquery]] rule wraps the second, third, ... instances of the same
+ *   subquery into a [[ReuseSubquery]] node, but this just a cosmetic change in the plan.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *          scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * CommonScalarSubqueries [scalar-subquery#250 []]
+ * :  +- Project [named_struct(avg(a), avg(a)#236, sum(b), sum(b)#238L) AS mergedValue#249]
+ * :     +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :        +- Project [a#233, b#234]
+ * :           +- Relation default.t[a#233,b#234] parquet
+ * +- Project [scalarsubqueryreference(0, 0, DoubleType, 231) AS scalarsubquery()#241,
+ *             scalarsubqueryreference(0, 1, LongType, 232) AS scalarsubquery()#242L]
+ *    +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    plan match {
+      case Subquery(_: CommonScalarSubqueries, _) => plan
+      case s: Subquery => s.copy(child = extractCommonScalarSubqueries(s.child))
+      case _: CommonScalarSubqueries => plan
+      case _ => extractCommonScalarSubqueries(plan)
+    }
+  }
+
+  /**
+   * An item in the cache of merged scalar subqueries.
+   *
+   * @param elements  List of attributes that form the scalar return value of a merged subquery
+   * @param plan      The plan of a merged scalar subquery
+   * @param merged    A flag to identify if this item is the result of merging subqueries.
+   *                  Please note that `elements.size == 1` doesn't always mean that the plan is not
+   *                  merged as there can be subqueries that are different ([[checkIdenticalPlans]]
+   *                  is false) due to an extra [[Project]] node in one of them. In that case
+   *                  `elements.size` remains 1 after merging, but the merged flag becomes true.
+   */
+  case class Header(elements: Seq[(String, Attribute)], plan: LogicalPlan, merged: Boolean)
+
+  private def extractCommonScalarSubqueries(plan: LogicalPlan) = {
+    val cache = ListBuffer.empty[Header]
+    val newPlan = removeReferences(insertReferences(plan, cache), cache)
+    if (cache.nonEmpty) {
+      val scalarSubqueries = cache.map {
+        case Header(elements, child, _) => ScalarSubquery(createProject(elements, child))
+      }.toSeq
+      CommonScalarSubqueries(scalarSubqueries, newPlan)
+    } else {
+      newPlan
+    }
+  }
+
+  // First traversal builds up the cache and inserts `ScalarSubqueryReference`s to the plan.
+  private def insertReferences(plan: LogicalPlan, cache: ListBuffer[Header]): LogicalPlan = {
+    plan.transformAllExpressionsWithPruning(_.containsAnyPattern(SCALAR_SUBQUERY)) {
+      case s: ScalarSubquery if s.children.isEmpty =>

Review comment:
       Thanks! Fixed in: https://github.com/apache/spark/pull/32298/commits/96a502d6f4ed44ed56a92f636f793d365dfa69d3.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


   **[Test build #145035 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/145035/testReport)** for PR 32298 at commit [`61f2b34`](https://github.com/apache/spark/commit/61f2b34c49c2f7c76f3ebd320b9d2f927f7ea100).


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,416 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CommonScalarSubqueries, Filter, Join, LogicalPlan, Project, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s to compute multiple scalar
+ * values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ *   During this first traversal each [[ScalarSubquery]] expression is replaced to a
+ *   [[ScalarSubqueryReference]] pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a results of merging 2 or more
+ *   plans, or it is a plan that was seen only once.
+ *   Merged plans in the cache get a "header", that is is basically
+ *   `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in new root
+ *   [[Project]] node. This expression ensures that the merged plan is a valid scalar subquery that
+ *   returns only one value.
+ * - A second traversal checks if a [[ScalarSubqueryReference]] is pointing to a merged subquery
+ *   plan or not and either keeps the reference or restores the original [[ScalarSubquery]].
+ *   If there are [[ScalarSubqueryReference]] nodes remained a [[CommonScalarSubqueries]] root node
+ *   is added to the plan with the referenced scalar subqueries.
+ * - [[PlanSubqueries]] or [[PlanAdaptiveSubqueries]] rule does the physical planning of scalar
+ *   subqueries including the ones under [[CommonScalarSubqueriesExec]] node and replaces
+ *   each [[ScalarSubqueryReference]] to their referenced physical plan in
+ *   `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() header))` form.
+ *   It is important that references pointing to the same merged subquery are replaced to the same
+ *   planned instance to make sure that each merged subquery runs only once (even without a wrapping
+ *   [[ReuseSubquery]] node).
+ *   Finally, the [[CommonScalarSubqueriesExec]] node is removed from the physical plan.
+ * - The [[ReuseExchangeAndSubquery]] rule wraps the second, third, ... instances of the same
+ *   subquery into a [[ReuseSubquery]] node, but this just a cosmetic change in the plan.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *          scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * CommonScalarSubqueries [scalar-subquery#250 []]
+ * :  +- Project [named_struct(avg(a), avg(a)#236, sum(b), sum(b)#238L) AS mergedValue#249]
+ * :     +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :        +- Project [a#233, b#234]
+ * :           +- Relation default.t[a#233,b#234] parquet
+ * +- Project [scalarsubqueryreference(0, 0, DoubleType, 231) AS scalarsubquery()#241,
+ *             scalarsubqueryreference(0, 1, LongType, 232) AS scalarsubquery()#242L]
+ *    +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    if (conf.subqueryReuseEnabled) {
+      plan match {
+        case Subquery(_: CommonScalarSubqueries, _) => plan
+        case s: Subquery => s.copy(child = extractCommonScalarSubqueries(s.child))
+        case _: CommonScalarSubqueries => plan
+        case _ => extractCommonScalarSubqueries(plan)
+      }
+    } else {
+      plan
+    }
+  }
+
+  type Header = Project
+
+  private def extractCommonScalarSubqueries(plan: LogicalPlan) = {
+    // Plan of subqueries and a flag is the plan is merged
+    val cache = ListBuffer.empty[(Header, Boolean)]
+    val newPlan = removeReferences(insertReferences(plan, cache), cache)
+    if (cache.nonEmpty) {
+      val scalarSubqueries = cache.map { case (header, _) => ScalarSubquery(header) }.toSeq
+      CommonScalarSubqueries(scalarSubqueries, newPlan)
+    } else {
+      newPlan
+    }
+  }
+
+  // First traversal builds up the cache and inserts `ScalarSubqueryReference`s to the plan.
+  private def insertReferences(
+      plan: LogicalPlan,
+      cache: ListBuffer[(Header, Boolean)]): LogicalPlan = {
+    plan.transformAllExpressionsWithPruning(_.containsAnyPattern(SCALAR_SUBQUERY)) {
+      case s: ScalarSubquery if s.children.isEmpty =>
+        val (subqueryIndex, headerIndex) = cacheSubquery(s.plan, cache)
+        ScalarSubqueryReference(subqueryIndex, headerIndex, s.dataType, s.exprId)
+    }
+  }
+
+  // Caching returns the index of the subquery in the cache and the index of scalar member in the
+  // "header", that `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in a
+  // [[Project]] node.
+  private def cacheSubquery(
+      plan: LogicalPlan,
+      cache: ListBuffer[(Header, Boolean)]): (Int, Int) = {
+    val firstOutput = plan.output.head
+    cache.zipWithIndex.collectFirst(Function.unlift { case ((header, merged), subqueryIndex) =>
+      checkIdenticalPlans(plan, header.child)
+        .map((subqueryIndex, header, header.child, _, merged))
+        .orElse(tryMergePlans(plan, header.child).map {
+          case (mergedPlan, outputMap) => (subqueryIndex, header, mergedPlan, outputMap, true)
+        })
+    }).map { case (subqueryIndex, header, mergedPlan, outputMap, merged) =>
+      val mappedFirstOutput = mapAttributes(firstOutput, outputMap)
+      val headerElements = getHeaderElements(header)
+      var headerIndex = headerElements.indexWhere {
+        case (_, attribute) => attribute.exprId == mappedFirstOutput.exprId
+      }
+      if (headerIndex == -1) {
+        val newHeaderElements = headerElements :+ (Literal(firstOutput.name) -> mappedFirstOutput)
+        cache(subqueryIndex) = createHeader(newHeaderElements, mergedPlan) -> merged
+        headerIndex = headerElements.size
+      }
+      subqueryIndex -> headerIndex
+    }.getOrElse {
+      cache += createHeader(Seq(Literal(firstOutput.name) -> firstOutput), plan) -> false
+      cache.length - 1 -> 0
+    }
+  }
+
+  // If 2 plans are identical return the attribute mapping from the new to the cached version.
+  private def checkIdenticalPlans(newPlan: LogicalPlan, cachedPlan: LogicalPlan) = {
+    if (newPlan.canonicalized == cachedPlan.canonicalized) {
+      Some(AttributeMap(newPlan.output.zip(cachedPlan.output)))
+    } else {
+      None
+    }
+  }
+
+  // Recursively traverse down and try merging 2 plans. If merge is possible then return the merged
+  // plan with the attribute mapping from the new to the merged version.
+  // Please note that merging arbitrary plans can be complicated, the current version supports only
+  // some of the most important nodes.
+  private def tryMergePlans(
+      newPlan: LogicalPlan,
+      cachedPlan: LogicalPlan): Option[(LogicalPlan, AttributeMap[Attribute])] = {
+    checkIdenticalPlans(newPlan, cachedPlan).map(cachedPlan -> _).orElse(
+      (newPlan, cachedPlan) match {
+        case (np: Project, cp: Project) =>
+          tryMergePlans(np.child, cp.child).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.projectList, outputMap, cp.projectList)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np, cp: Project) =>
+          tryMergePlans(np, cp.child).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.output, outputMap, cp.projectList)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np: Project, cp) =>
+          tryMergePlans(np.child, cp).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.projectList, outputMap, cp.output)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np: Aggregate, cp: Aggregate) if supportedAggregateMerge(np, cp) =>
+          tryMergePlans(np.child, cp.child).flatMap { case (mergedChild, outputMap) =>
+            val mappedNewGroupingExpression =
+              np.groupingExpressions.map(mapAttributes(_, outputMap))
+            if (ExpressionSet(mappedNewGroupingExpression) ==

Review comment:
       Nit: Please add comment here: although `groupingExpressions ` is a `Seq` the order of expressions does not matter so `ExpressionSet` is safe to be used here to decide about whether these aggregates can be merged

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,416 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CommonScalarSubqueries, Filter, Join, LogicalPlan, Project, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s to compute multiple scalar
+ * values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ *   During this first traversal each [[ScalarSubquery]] expression is replaced to a
+ *   [[ScalarSubqueryReference]] pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a results of merging 2 or more
+ *   plans, or it is a plan that was seen only once.
+ *   Merged plans in the cache get a "header", that is is basically
+ *   `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in new root
+ *   [[Project]] node. This expression ensures that the merged plan is a valid scalar subquery that
+ *   returns only one value.
+ * - A second traversal checks if a [[ScalarSubqueryReference]] is pointing to a merged subquery
+ *   plan or not and either keeps the reference or restores the original [[ScalarSubquery]].
+ *   If there are [[ScalarSubqueryReference]] nodes remained a [[CommonScalarSubqueries]] root node
+ *   is added to the plan with the referenced scalar subqueries.
+ * - [[PlanSubqueries]] or [[PlanAdaptiveSubqueries]] rule does the physical planning of scalar
+ *   subqueries including the ones under [[CommonScalarSubqueriesExec]] node and replaces
+ *   each [[ScalarSubqueryReference]] to their referenced physical plan in
+ *   `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() header))` form.
+ *   It is important that references pointing to the same merged subquery are replaced to the same
+ *   planned instance to make sure that each merged subquery runs only once (even without a wrapping
+ *   [[ReuseSubquery]] node).
+ *   Finally, the [[CommonScalarSubqueriesExec]] node is removed from the physical plan.
+ * - The [[ReuseExchangeAndSubquery]] rule wraps the second, third, ... instances of the same
+ *   subquery into a [[ReuseSubquery]] node, but this just a cosmetic change in the plan.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *          scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * CommonScalarSubqueries [scalar-subquery#250 []]
+ * :  +- Project [named_struct(avg(a), avg(a)#236, sum(b), sum(b)#238L) AS mergedValue#249]
+ * :     +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :        +- Project [a#233, b#234]
+ * :           +- Relation default.t[a#233,b#234] parquet
+ * +- Project [scalarsubqueryreference(0, 0, DoubleType, 231) AS scalarsubquery()#241,
+ *             scalarsubqueryreference(0, 1, LongType, 232) AS scalarsubquery()#242L]
+ *    +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    if (conf.subqueryReuseEnabled) {
+      plan match {
+        case Subquery(_: CommonScalarSubqueries, _) => plan
+        case s: Subquery => s.copy(child = extractCommonScalarSubqueries(s.child))
+        case _: CommonScalarSubqueries => plan
+        case _ => extractCommonScalarSubqueries(plan)
+      }
+    } else {
+      plan
+    }
+  }
+
+  type Header = Project

Review comment:
       To avoid repeated calls of `createHeader` after and `getHeaderElements`  I suggest to change this to `case class Header(elements: Seq[(Literal, Attribute)], child: LogicalPlan)`. 
   
   This way you can rid of the `merged` flag at the cache too because from the size of the element you can decide whether is it a created for a single scalar subquery or a result of a merge. 
   
   I made the change locally. Let me open a PR against your one to avoid doing the work twice. 
   

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,416 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CommonScalarSubqueries, Filter, Join, LogicalPlan, Project, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s to compute multiple scalar
+ * values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ *   During this first traversal each [[ScalarSubquery]] expression is replaced to a
+ *   [[ScalarSubqueryReference]] pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a results of merging 2 or more
+ *   plans, or it is a plan that was seen only once.
+ *   Merged plans in the cache get a "header", that is is basically
+ *   `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in new root
+ *   [[Project]] node. This expression ensures that the merged plan is a valid scalar subquery that
+ *   returns only one value.
+ * - A second traversal checks if a [[ScalarSubqueryReference]] is pointing to a merged subquery
+ *   plan or not and either keeps the reference or restores the original [[ScalarSubquery]].
+ *   If there are [[ScalarSubqueryReference]] nodes remained a [[CommonScalarSubqueries]] root node
+ *   is added to the plan with the referenced scalar subqueries.
+ * - [[PlanSubqueries]] or [[PlanAdaptiveSubqueries]] rule does the physical planning of scalar
+ *   subqueries including the ones under [[CommonScalarSubqueriesExec]] node and replaces
+ *   each [[ScalarSubqueryReference]] to their referenced physical plan in
+ *   `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() header))` form.
+ *   It is important that references pointing to the same merged subquery are replaced to the same
+ *   planned instance to make sure that each merged subquery runs only once (even without a wrapping
+ *   [[ReuseSubquery]] node).
+ *   Finally, the [[CommonScalarSubqueriesExec]] node is removed from the physical plan.
+ * - The [[ReuseExchangeAndSubquery]] rule wraps the second, third, ... instances of the same
+ *   subquery into a [[ReuseSubquery]] node, but this just a cosmetic change in the plan.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *          scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * CommonScalarSubqueries [scalar-subquery#250 []]
+ * :  +- Project [named_struct(avg(a), avg(a)#236, sum(b), sum(b)#238L) AS mergedValue#249]
+ * :     +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :        +- Project [a#233, b#234]
+ * :           +- Relation default.t[a#233,b#234] parquet
+ * +- Project [scalarsubqueryreference(0, 0, DoubleType, 231) AS scalarsubquery()#241,
+ *             scalarsubqueryreference(0, 1, LongType, 232) AS scalarsubquery()#242L]
+ *    +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    if (conf.subqueryReuseEnabled) {
+      plan match {
+        case Subquery(_: CommonScalarSubqueries, _) => plan
+        case s: Subquery => s.copy(child = extractCommonScalarSubqueries(s.child))
+        case _: CommonScalarSubqueries => plan
+        case _ => extractCommonScalarSubqueries(plan)
+      }
+    } else {
+      plan
+    }
+  }
+
+  type Header = Project
+
+  private def extractCommonScalarSubqueries(plan: LogicalPlan) = {
+    // Plan of subqueries and a flag is the plan is merged
+    val cache = ListBuffer.empty[(Header, Boolean)]
+    val newPlan = removeReferences(insertReferences(plan, cache), cache)
+    if (cache.nonEmpty) {
+      val scalarSubqueries = cache.map { case (header, _) => ScalarSubquery(header) }.toSeq
+      CommonScalarSubqueries(scalarSubqueries, newPlan)
+    } else {
+      newPlan
+    }
+  }
+
+  // First traversal builds up the cache and inserts `ScalarSubqueryReference`s to the plan.
+  private def insertReferences(
+      plan: LogicalPlan,
+      cache: ListBuffer[(Header, Boolean)]): LogicalPlan = {
+    plan.transformAllExpressionsWithPruning(_.containsAnyPattern(SCALAR_SUBQUERY)) {
+      case s: ScalarSubquery if s.children.isEmpty =>
+        val (subqueryIndex, headerIndex) = cacheSubquery(s.plan, cache)
+        ScalarSubqueryReference(subqueryIndex, headerIndex, s.dataType, s.exprId)
+    }
+  }
+
+  // Caching returns the index of the subquery in the cache and the index of scalar member in the
+  // "header", that `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in a
+  // [[Project]] node.
+  private def cacheSubquery(
+      plan: LogicalPlan,
+      cache: ListBuffer[(Header, Boolean)]): (Int, Int) = {
+    val firstOutput = plan.output.head
+    cache.zipWithIndex.collectFirst(Function.unlift { case ((header, merged), subqueryIndex) =>
+      checkIdenticalPlans(plan, header.child)
+        .map((subqueryIndex, header, header.child, _, merged))
+        .orElse(tryMergePlans(plan, header.child).map {
+          case (mergedPlan, outputMap) => (subqueryIndex, header, mergedPlan, outputMap, true)
+        })
+    }).map { case (subqueryIndex, header, mergedPlan, outputMap, merged) =>
+      val mappedFirstOutput = mapAttributes(firstOutput, outputMap)
+      val headerElements = getHeaderElements(header)
+      var headerIndex = headerElements.indexWhere {
+        case (_, attribute) => attribute.exprId == mappedFirstOutput.exprId
+      }
+      if (headerIndex == -1) {
+        val newHeaderElements = headerElements :+ (Literal(firstOutput.name) -> mappedFirstOutput)
+        cache(subqueryIndex) = createHeader(newHeaderElements, mergedPlan) -> merged
+        headerIndex = headerElements.size
+      }
+      subqueryIndex -> headerIndex
+    }.getOrElse {
+      cache += createHeader(Seq(Literal(firstOutput.name) -> firstOutput), plan) -> false
+      cache.length - 1 -> 0
+    }
+  }
+
+  // If 2 plans are identical return the attribute mapping from the new to the cached version.
+  private def checkIdenticalPlans(newPlan: LogicalPlan, cachedPlan: LogicalPlan) = {
+    if (newPlan.canonicalized == cachedPlan.canonicalized) {
+      Some(AttributeMap(newPlan.output.zip(cachedPlan.output)))
+    } else {
+      None
+    }
+  }
+
+  // Recursively traverse down and try merging 2 plans. If merge is possible then return the merged
+  // plan with the attribute mapping from the new to the merged version.
+  // Please note that merging arbitrary plans can be complicated, the current version supports only
+  // some of the most important nodes.
+  private def tryMergePlans(
+      newPlan: LogicalPlan,
+      cachedPlan: LogicalPlan): Option[(LogicalPlan, AttributeMap[Attribute])] = {
+    checkIdenticalPlans(newPlan, cachedPlan).map(cachedPlan -> _).orElse(
+      (newPlan, cachedPlan) match {
+        case (np: Project, cp: Project) =>
+          tryMergePlans(np.child, cp.child).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.projectList, outputMap, cp.projectList)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np, cp: Project) =>
+          tryMergePlans(np, cp.child).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.output, outputMap, cp.projectList)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np: Project, cp) =>
+          tryMergePlans(np.child, cp).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.projectList, outputMap, cp.output)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np: Aggregate, cp: Aggregate) if supportedAggregateMerge(np, cp) =>
+          tryMergePlans(np.child, cp.child).flatMap { case (mergedChild, outputMap) =>
+            val mappedNewGroupingExpression =
+              np.groupingExpressions.map(mapAttributes(_, outputMap))
+            if (ExpressionSet(mappedNewGroupingExpression) ==
+              ExpressionSet(cp.groupingExpressions)) {
+              val (mergedAggregateExpressions, newOutputMap) =
+                mergeNamedExpressions(np.aggregateExpressions, outputMap, cp.aggregateExpressions)
+              val mergedPlan =
+                Aggregate(cp.groupingExpressions, mergedAggregateExpressions, mergedChild)
+              Some(mergedPlan -> newOutputMap)
+            } else {
+              None
+            }
+          }
+
+        // Merging general nodes is complicated and this implementation supports only those nodes in
+        // which the order and the number of output attributes are not relevant (see
+        // `supportedMerge()` whitelist).
+        // Also, this implementation supports only those nodes in which children can be merged in
+        // the same order.
+        case (np, cp) if supportedMerge(np) && np.getClass == cp.getClass &&
+          np.children.size == cp.children.size =>
+          val merged = np.children.zip(cp.children).map {
+            case (npChild, cpChild) => tryMergePlans(npChild, cpChild)
+          }
+          if (merged.forall(_.isDefined)) {
+            val (mergedChildren, outputMaps) = merged.map(_.get).unzip
+            val outputMap = AttributeMap(outputMaps.map(_.iterator).reduce(_ ++ _).toSeq)
+            val mappedNewPlan = mapAttributes(np.withNewChildren(mergedChildren), outputMap)
+            val mergedPlan = cp.withNewChildren(mergedChildren)
+            if (mappedNewPlan.canonicalized == mergedPlan.canonicalized) {
+              Some(mergedPlan -> outputMap)
+            } else {
+              None
+            }
+          } else {
+            None
+          }
+
+        // As a follow-up, it would be possible to merge `CommonScalarSubqueries` nodes, which would
+        // allow merging subqueries with mergee subqueries.
+        // E.g. this query:
+        //
+        // SELECT
+        //   (
+        //     SELECT
+        //       (SELECT avg(a) FROM t GROUP BY b) +
+        //       (SELECT sum(b) FROM t GROUP BY b)
+        //   ),
+        //   (
+        //     SELECT
+        //       (SELECT max(a) FROM t GROUP BY b) +
+        //       (SELECT min(b) FROM t GROUP BY b)
+        //   )
+        //
+        // is currently optimized to:
+        //
+        // == Optimized Logical Plan ==
+        // Project [scalar-subquery#233 [] AS scalarsubquery()#255,
+        //          scalar-subquery#236 [] AS scalarsubquery()#256]
+        // :  :- CommonScalarSubqueries [scalar-subquery#264 []]
+        // :  :  :  +- Aggregate [b#238], [named_struct(avg(a), avg(a#237), sum(b), sum(b#238))
+        //                                 AS mergedValue#263]
+        // :  :  :     +- Relation default.t[a#237,b#238] parquet
+        // :  :  +- Project [(scalarsubqueryreference(0, 0, DoubleType, 231) +
+        //                   cast(scalarsubqueryreference(0, 1, LongType, 232) as double))
+        //                   AS (scalarsubquery() + scalarsubquery())#245]
+        // :  :     +- OneRowRelation
+        // :  +- CommonScalarSubqueries [scalar-subquery#269 []]
+        // :     :  +- Aggregate [b#254], [named_struct(min(a), min(a#253), max(b), max(b#254))
+        //                                 AS mergedValue#268]
+        // :     :     +- Relation default.t[a#253,b#254] parquet
+        // :     +- Project [(scalarsubqueryreference(0, 0, IntegerType, 234) +
+        //                   scalarsubqueryreference(0, 1, IntegerType, 235))
+        //                   AS (scalarsubquery() + scalarsubquery())#252]
+        // :        +- OneRowRelation
+        // +- OneRowRelation
+        //
+        // but if we implemented merging `CommonScalarSubqueries` nodes then the plan could be
+        // transformed further and all leaf subqueries could be merged.
+
+        // Otherwise merging is not possible.
+        case _ => None
+      })
+  }
+
+  private def createHeader(headerElements: Seq[(Literal, Attribute)], plan: LogicalPlan): Header = {
+    Project(
+      Seq(Alias(
+        CreateNamedStruct(headerElements.flatMap {
+          case (name, attribute) => Seq(name, attribute)
+        }),
+        "mergedValue")()),
+      plan)
+  }
+
+  private def getHeaderElements(header: Header) = {
+    val mergedValue =
+      header.projectList.head.asInstanceOf[Alias].child.asInstanceOf[CreateNamedStruct]
+    mergedValue.children.grouped(2).map {
+      case Seq(name: Literal, attribute: Attribute) => name -> attribute
+    }.toSeq
+  }
+
+  private def mapAttributes[T <: Expression](expr: T, outputMap: AttributeMap[Attribute]) = {
+    expr.transform {
+      case a: Attribute => outputMap.getOrElse(a, a)
+    }.asInstanceOf[T]
+  }
+
+  private def mapAttributes(plan: LogicalPlan, outputMap: AttributeMap[Attribute]) = {
+    plan.transformExpressions {
+      case a: Attribute => outputMap.getOrElse(a, a)
+    }
+  }
+
+  // Applies `outputMap` attribute mapping on elements of `newExpressions` and merges them into
+  // `cachedExpressions`. Returns the merged expressions and the attribute mapping from the new to
+  // the merged version that can be propagated up during merging nodes.
+  private def mergeNamedExpressions(
+      newExpressions: Seq[NamedExpression],
+      outputMap: AttributeMap[Attribute],
+      cachedExpressions: Seq[NamedExpression]) = {
+    val mergedExpressions = ListBuffer[NamedExpression](cachedExpressions: _*)
+    val newOutputMap = AttributeMap(newExpressions.map { ne =>
+      val mapped = mapAttributes(ne, outputMap)
+      val withoutAlias = mapped match {
+        case Alias(child, _) => child
+        case e => e
+      }
+      ne.toAttribute -> mergedExpressions.find {
+        case Alias(child, _) => child semanticEquals withoutAlias
+        case e => e semanticEquals withoutAlias
+      }.getOrElse {
+        mergedExpressions += mapped
+        mapped
+      }.toAttribute
+    })
+    (mergedExpressions.toSeq, newOutputMap)
+  }
+
+  // Merging different aggregate implementations could cause performance regression
+  private def supportedAggregateMerge(newPlan: Aggregate, cachedPlan: Aggregate) = {
+    val newPlanAggregateExpressions = newPlan.aggregateExpressions.flatMap(_.collect {
+      case a: AggregateExpression => a
+    })
+    val cachedPlanAggregateExpressions = cachedPlan.aggregateExpressions.flatMap(_.collect {
+      case a: AggregateExpression => a
+    })
+    val newPlanSupportsHashAggregate = Aggregate.supportsHashAggregate(
+      newPlanAggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes))
+    val cachedPlanSupportsHashAggregate = Aggregate.supportsHashAggregate(
+      cachedPlanAggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes))
+    newPlanSupportsHashAggregate && cachedPlanSupportsHashAggregate ||
+      !newPlanSupportsHashAggregate && !cachedPlanSupportsHashAggregate && {
+        val newPlanSupportsObjectHashAggregate =
+          Aggregate.supportsObjectHashAggregate(newPlanAggregateExpressions)
+        val cachedPlanSupportsObjectHashAggregate =
+          Aggregate.supportsObjectHashAggregate(cachedPlanAggregateExpressions)
+        newPlanSupportsObjectHashAggregate && cachedPlanSupportsObjectHashAggregate ||
+          !newPlanSupportsObjectHashAggregate && !cachedPlanSupportsObjectHashAggregate
+      }
+  }
+
+  // Whitelist of mergeable general nodes
+  private def supportedMerge(plan: LogicalPlan) = {
+    plan match {
+      case _: Filter => true
+      case _: Join => true
+      case _ => false
+    }
+  }
+
+  // Second traversal replaces `ScalarSubqueryReference`s to either
+  // `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() header))` if the plan is
+  // merged from multiple subqueries or `ScalarSubquery(original plan)` it it isn't.

Review comment:
       Nit:
   ```suggestion
     // merged from multiple subqueries or `ScalarSubquery(original plan)` if it isn't.
   ```

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,416 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CommonScalarSubqueries, Filter, Join, LogicalPlan, Project, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s to compute multiple scalar
+ * values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ *   During this first traversal each [[ScalarSubquery]] expression is replaced to a
+ *   [[ScalarSubqueryReference]] pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a results of merging 2 or more
+ *   plans, or it is a plan that was seen only once.
+ *   Merged plans in the cache get a "header", that is is basically
+ *   `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in new root
+ *   [[Project]] node. This expression ensures that the merged plan is a valid scalar subquery that
+ *   returns only one value.
+ * - A second traversal checks if a [[ScalarSubqueryReference]] is pointing to a merged subquery
+ *   plan or not and either keeps the reference or restores the original [[ScalarSubquery]].
+ *   If there are [[ScalarSubqueryReference]] nodes remained a [[CommonScalarSubqueries]] root node
+ *   is added to the plan with the referenced scalar subqueries.
+ * - [[PlanSubqueries]] or [[PlanAdaptiveSubqueries]] rule does the physical planning of scalar
+ *   subqueries including the ones under [[CommonScalarSubqueriesExec]] node and replaces
+ *   each [[ScalarSubqueryReference]] to their referenced physical plan in
+ *   `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() header))` form.
+ *   It is important that references pointing to the same merged subquery are replaced to the same
+ *   planned instance to make sure that each merged subquery runs only once (even without a wrapping
+ *   [[ReuseSubquery]] node).
+ *   Finally, the [[CommonScalarSubqueriesExec]] node is removed from the physical plan.
+ * - The [[ReuseExchangeAndSubquery]] rule wraps the second, third, ... instances of the same
+ *   subquery into a [[ReuseSubquery]] node, but this just a cosmetic change in the plan.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *          scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * CommonScalarSubqueries [scalar-subquery#250 []]
+ * :  +- Project [named_struct(avg(a), avg(a)#236, sum(b), sum(b)#238L) AS mergedValue#249]
+ * :     +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :        +- Project [a#233, b#234]
+ * :           +- Relation default.t[a#233,b#234] parquet
+ * +- Project [scalarsubqueryreference(0, 0, DoubleType, 231) AS scalarsubquery()#241,
+ *             scalarsubqueryreference(0, 1, LongType, 232) AS scalarsubquery()#242L]
+ *    +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    if (conf.subqueryReuseEnabled) {
+      plan match {
+        case Subquery(_: CommonScalarSubqueries, _) => plan
+        case s: Subquery => s.copy(child = extractCommonScalarSubqueries(s.child))
+        case _: CommonScalarSubqueries => plan
+        case _ => extractCommonScalarSubqueries(plan)
+      }
+    } else {
+      plan
+    }
+  }
+
+  type Header = Project
+
+  private def extractCommonScalarSubqueries(plan: LogicalPlan) = {
+    // Plan of subqueries and a flag is the plan is merged
+    val cache = ListBuffer.empty[(Header, Boolean)]
+    val newPlan = removeReferences(insertReferences(plan, cache), cache)
+    if (cache.nonEmpty) {
+      val scalarSubqueries = cache.map { case (header, _) => ScalarSubquery(header) }.toSeq
+      CommonScalarSubqueries(scalarSubqueries, newPlan)
+    } else {
+      newPlan
+    }
+  }
+
+  // First traversal builds up the cache and inserts `ScalarSubqueryReference`s to the plan.
+  private def insertReferences(
+      plan: LogicalPlan,
+      cache: ListBuffer[(Header, Boolean)]): LogicalPlan = {
+    plan.transformAllExpressionsWithPruning(_.containsAnyPattern(SCALAR_SUBQUERY)) {
+      case s: ScalarSubquery if s.children.isEmpty =>
+        val (subqueryIndex, headerIndex) = cacheSubquery(s.plan, cache)
+        ScalarSubqueryReference(subqueryIndex, headerIndex, s.dataType, s.exprId)
+    }
+  }
+
+  // Caching returns the index of the subquery in the cache and the index of scalar member in the
+  // "header", that `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in a
+  // [[Project]] node.
+  private def cacheSubquery(
+      plan: LogicalPlan,
+      cache: ListBuffer[(Header, Boolean)]): (Int, Int) = {
+    val firstOutput = plan.output.head
+    cache.zipWithIndex.collectFirst(Function.unlift { case ((header, merged), subqueryIndex) =>
+      checkIdenticalPlans(plan, header.child)
+        .map((subqueryIndex, header, header.child, _, merged))
+        .orElse(tryMergePlans(plan, header.child).map {
+          case (mergedPlan, outputMap) => (subqueryIndex, header, mergedPlan, outputMap, true)
+        })
+    }).map { case (subqueryIndex, header, mergedPlan, outputMap, merged) =>
+      val mappedFirstOutput = mapAttributes(firstOutput, outputMap)
+      val headerElements = getHeaderElements(header)
+      var headerIndex = headerElements.indexWhere {
+        case (_, attribute) => attribute.exprId == mappedFirstOutput.exprId
+      }
+      if (headerIndex == -1) {
+        val newHeaderElements = headerElements :+ (Literal(firstOutput.name) -> mappedFirstOutput)
+        cache(subqueryIndex) = createHeader(newHeaderElements, mergedPlan) -> merged
+        headerIndex = headerElements.size
+      }
+      subqueryIndex -> headerIndex
+    }.getOrElse {
+      cache += createHeader(Seq(Literal(firstOutput.name) -> firstOutput), plan) -> false
+      cache.length - 1 -> 0
+    }
+  }
+
+  // If 2 plans are identical return the attribute mapping from the new to the cached version.
+  private def checkIdenticalPlans(newPlan: LogicalPlan, cachedPlan: LogicalPlan) = {
+    if (newPlan.canonicalized == cachedPlan.canonicalized) {
+      Some(AttributeMap(newPlan.output.zip(cachedPlan.output)))
+    } else {
+      None
+    }
+  }
+
+  // Recursively traverse down and try merging 2 plans. If merge is possible then return the merged
+  // plan with the attribute mapping from the new to the merged version.
+  // Please note that merging arbitrary plans can be complicated, the current version supports only
+  // some of the most important nodes.
+  private def tryMergePlans(
+      newPlan: LogicalPlan,
+      cachedPlan: LogicalPlan): Option[(LogicalPlan, AttributeMap[Attribute])] = {
+    checkIdenticalPlans(newPlan, cachedPlan).map(cachedPlan -> _).orElse(
+      (newPlan, cachedPlan) match {
+        case (np: Project, cp: Project) =>
+          tryMergePlans(np.child, cp.child).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.projectList, outputMap, cp.projectList)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np, cp: Project) =>

Review comment:
       I have added extra logs to each `case` and as I see this one is not tested currently. 
   Can you please extend the tests to cover this?

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,416 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CommonScalarSubqueries, Filter, Join, LogicalPlan, Project, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s to compute multiple scalar
+ * values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ *   During this first traversal each [[ScalarSubquery]] expression is replaced to a
+ *   [[ScalarSubqueryReference]] pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a results of merging 2 or more
+ *   plans, or it is a plan that was seen only once.
+ *   Merged plans in the cache get a "header", that is is basically
+ *   `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in new root
+ *   [[Project]] node. This expression ensures that the merged plan is a valid scalar subquery that
+ *   returns only one value.
+ * - A second traversal checks if a [[ScalarSubqueryReference]] is pointing to a merged subquery
+ *   plan or not and either keeps the reference or restores the original [[ScalarSubquery]].
+ *   If there are [[ScalarSubqueryReference]] nodes remained a [[CommonScalarSubqueries]] root node
+ *   is added to the plan with the referenced scalar subqueries.
+ * - [[PlanSubqueries]] or [[PlanAdaptiveSubqueries]] rule does the physical planning of scalar
+ *   subqueries including the ones under [[CommonScalarSubqueriesExec]] node and replaces
+ *   each [[ScalarSubqueryReference]] to their referenced physical plan in
+ *   `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() header))` form.
+ *   It is important that references pointing to the same merged subquery are replaced to the same
+ *   planned instance to make sure that each merged subquery runs only once (even without a wrapping
+ *   [[ReuseSubquery]] node).
+ *   Finally, the [[CommonScalarSubqueriesExec]] node is removed from the physical plan.
+ * - The [[ReuseExchangeAndSubquery]] rule wraps the second, third, ... instances of the same
+ *   subquery into a [[ReuseSubquery]] node, but this just a cosmetic change in the plan.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *          scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * CommonScalarSubqueries [scalar-subquery#250 []]
+ * :  +- Project [named_struct(avg(a), avg(a)#236, sum(b), sum(b)#238L) AS mergedValue#249]
+ * :     +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :        +- Project [a#233, b#234]
+ * :           +- Relation default.t[a#233,b#234] parquet
+ * +- Project [scalarsubqueryreference(0, 0, DoubleType, 231) AS scalarsubquery()#241,
+ *             scalarsubqueryreference(0, 1, LongType, 232) AS scalarsubquery()#242L]
+ *    +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    if (conf.subqueryReuseEnabled) {
+      plan match {
+        case Subquery(_: CommonScalarSubqueries, _) => plan
+        case s: Subquery => s.copy(child = extractCommonScalarSubqueries(s.child))
+        case _: CommonScalarSubqueries => plan
+        case _ => extractCommonScalarSubqueries(plan)
+      }
+    } else {
+      plan
+    }
+  }
+
+  type Header = Project
+
+  private def extractCommonScalarSubqueries(plan: LogicalPlan) = {
+    // Plan of subqueries and a flag is the plan is merged
+    val cache = ListBuffer.empty[(Header, Boolean)]
+    val newPlan = removeReferences(insertReferences(plan, cache), cache)
+    if (cache.nonEmpty) {
+      val scalarSubqueries = cache.map { case (header, _) => ScalarSubquery(header) }.toSeq
+      CommonScalarSubqueries(scalarSubqueries, newPlan)
+    } else {
+      newPlan
+    }
+  }
+
+  // First traversal builds up the cache and inserts `ScalarSubqueryReference`s to the plan.
+  private def insertReferences(
+      plan: LogicalPlan,
+      cache: ListBuffer[(Header, Boolean)]): LogicalPlan = {
+    plan.transformAllExpressionsWithPruning(_.containsAnyPattern(SCALAR_SUBQUERY)) {
+      case s: ScalarSubquery if s.children.isEmpty =>
+        val (subqueryIndex, headerIndex) = cacheSubquery(s.plan, cache)
+        ScalarSubqueryReference(subqueryIndex, headerIndex, s.dataType, s.exprId)
+    }
+  }
+
+  // Caching returns the index of the subquery in the cache and the index of scalar member in the
+  // "header", that `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in a
+  // [[Project]] node.
+  private def cacheSubquery(
+      plan: LogicalPlan,
+      cache: ListBuffer[(Header, Boolean)]): (Int, Int) = {
+    val firstOutput = plan.output.head

Review comment:
       Nit: I would avoid calling this `first` as in that case I would look for the rest but as a `ScalarSubquery` has only one single output, maybe just `output`?.

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,416 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CommonScalarSubqueries, Filter, Join, LogicalPlan, Project, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s to compute multiple scalar
+ * values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ *   During this first traversal each [[ScalarSubquery]] expression is replaced to a
+ *   [[ScalarSubqueryReference]] pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a results of merging 2 or more
+ *   plans, or it is a plan that was seen only once.
+ *   Merged plans in the cache get a "header", that is is basically
+ *   `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in new root
+ *   [[Project]] node. This expression ensures that the merged plan is a valid scalar subquery that
+ *   returns only one value.
+ * - A second traversal checks if a [[ScalarSubqueryReference]] is pointing to a merged subquery
+ *   plan or not and either keeps the reference or restores the original [[ScalarSubquery]].
+ *   If there are [[ScalarSubqueryReference]] nodes remained a [[CommonScalarSubqueries]] root node
+ *   is added to the plan with the referenced scalar subqueries.
+ * - [[PlanSubqueries]] or [[PlanAdaptiveSubqueries]] rule does the physical planning of scalar
+ *   subqueries including the ones under [[CommonScalarSubqueriesExec]] node and replaces
+ *   each [[ScalarSubqueryReference]] to their referenced physical plan in
+ *   `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() header))` form.
+ *   It is important that references pointing to the same merged subquery are replaced to the same
+ *   planned instance to make sure that each merged subquery runs only once (even without a wrapping
+ *   [[ReuseSubquery]] node).
+ *   Finally, the [[CommonScalarSubqueriesExec]] node is removed from the physical plan.
+ * - The [[ReuseExchangeAndSubquery]] rule wraps the second, third, ... instances of the same
+ *   subquery into a [[ReuseSubquery]] node, but this just a cosmetic change in the plan.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *          scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * CommonScalarSubqueries [scalar-subquery#250 []]
+ * :  +- Project [named_struct(avg(a), avg(a)#236, sum(b), sum(b)#238L) AS mergedValue#249]
+ * :     +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :        +- Project [a#233, b#234]
+ * :           +- Relation default.t[a#233,b#234] parquet
+ * +- Project [scalarsubqueryreference(0, 0, DoubleType, 231) AS scalarsubquery()#241,
+ *             scalarsubqueryreference(0, 1, LongType, 232) AS scalarsubquery()#242L]
+ *    +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    if (conf.subqueryReuseEnabled) {
+      plan match {
+        case Subquery(_: CommonScalarSubqueries, _) => plan
+        case s: Subquery => s.copy(child = extractCommonScalarSubqueries(s.child))
+        case _: CommonScalarSubqueries => plan
+        case _ => extractCommonScalarSubqueries(plan)
+      }
+    } else {
+      plan
+    }
+  }
+
+  type Header = Project
+
+  private def extractCommonScalarSubqueries(plan: LogicalPlan) = {
+    // Plan of subqueries and a flag is the plan is merged
+    val cache = ListBuffer.empty[(Header, Boolean)]
+    val newPlan = removeReferences(insertReferences(plan, cache), cache)
+    if (cache.nonEmpty) {
+      val scalarSubqueries = cache.map { case (header, _) => ScalarSubquery(header) }.toSeq
+      CommonScalarSubqueries(scalarSubqueries, newPlan)
+    } else {
+      newPlan
+    }
+  }
+
+  // First traversal builds up the cache and inserts `ScalarSubqueryReference`s to the plan.
+  private def insertReferences(
+      plan: LogicalPlan,
+      cache: ListBuffer[(Header, Boolean)]): LogicalPlan = {
+    plan.transformAllExpressionsWithPruning(_.containsAnyPattern(SCALAR_SUBQUERY)) {
+      case s: ScalarSubquery if s.children.isEmpty =>
+        val (subqueryIndex, headerIndex) = cacheSubquery(s.plan, cache)
+        ScalarSubqueryReference(subqueryIndex, headerIndex, s.dataType, s.exprId)
+    }
+  }
+
+  // Caching returns the index of the subquery in the cache and the index of scalar member in the
+  // "header", that `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in a
+  // [[Project]] node.
+  private def cacheSubquery(
+      plan: LogicalPlan,
+      cache: ListBuffer[(Header, Boolean)]): (Int, Int) = {
+    val firstOutput = plan.output.head
+    cache.zipWithIndex.collectFirst(Function.unlift { case ((header, merged), subqueryIndex) =>
+      checkIdenticalPlans(plan, header.child)
+        .map((subqueryIndex, header, header.child, _, merged))
+        .orElse(tryMergePlans(plan, header.child).map {
+          case (mergedPlan, outputMap) => (subqueryIndex, header, mergedPlan, outputMap, true)
+        })
+    }).map { case (subqueryIndex, header, mergedPlan, outputMap, merged) =>
+      val mappedFirstOutput = mapAttributes(firstOutput, outputMap)
+      val headerElements = getHeaderElements(header)
+      var headerIndex = headerElements.indexWhere {
+        case (_, attribute) => attribute.exprId == mappedFirstOutput.exprId
+      }
+      if (headerIndex == -1) {
+        val newHeaderElements = headerElements :+ (Literal(firstOutput.name) -> mappedFirstOutput)
+        cache(subqueryIndex) = createHeader(newHeaderElements, mergedPlan) -> merged
+        headerIndex = headerElements.size
+      }
+      subqueryIndex -> headerIndex
+    }.getOrElse {
+      cache += createHeader(Seq(Literal(firstOutput.name) -> firstOutput), plan) -> false
+      cache.length - 1 -> 0
+    }
+  }
+
+  // If 2 plans are identical return the attribute mapping from the new to the cached version.
+  private def checkIdenticalPlans(newPlan: LogicalPlan, cachedPlan: LogicalPlan) = {
+    if (newPlan.canonicalized == cachedPlan.canonicalized) {
+      Some(AttributeMap(newPlan.output.zip(cachedPlan.output)))
+    } else {
+      None
+    }
+  }
+
+  // Recursively traverse down and try merging 2 plans. If merge is possible then return the merged
+  // plan with the attribute mapping from the new to the merged version.
+  // Please note that merging arbitrary plans can be complicated, the current version supports only
+  // some of the most important nodes.
+  private def tryMergePlans(
+      newPlan: LogicalPlan,
+      cachedPlan: LogicalPlan): Option[(LogicalPlan, AttributeMap[Attribute])] = {
+    checkIdenticalPlans(newPlan, cachedPlan).map(cachedPlan -> _).orElse(
+      (newPlan, cachedPlan) match {
+        case (np: Project, cp: Project) =>
+          tryMergePlans(np.child, cp.child).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.projectList, outputMap, cp.projectList)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np, cp: Project) =>
+          tryMergePlans(np, cp.child).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.output, outputMap, cp.projectList)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np: Project, cp) =>
+          tryMergePlans(np.child, cp).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.projectList, outputMap, cp.output)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np: Aggregate, cp: Aggregate) if supportedAggregateMerge(np, cp) =>
+          tryMergePlans(np.child, cp.child).flatMap { case (mergedChild, outputMap) =>
+            val mappedNewGroupingExpression =
+              np.groupingExpressions.map(mapAttributes(_, outputMap))
+            if (ExpressionSet(mappedNewGroupingExpression) ==
+              ExpressionSet(cp.groupingExpressions)) {
+              val (mergedAggregateExpressions, newOutputMap) =
+                mergeNamedExpressions(np.aggregateExpressions, outputMap, cp.aggregateExpressions)
+              val mergedPlan =
+                Aggregate(cp.groupingExpressions, mergedAggregateExpressions, mergedChild)
+              Some(mergedPlan -> newOutputMap)
+            } else {
+              None
+            }
+          }
+
+        // Merging general nodes is complicated and this implementation supports only those nodes in
+        // which the order and the number of output attributes are not relevant (see
+        // `supportedMerge()` whitelist).
+        // Also, this implementation supports only those nodes in which children can be merged in
+        // the same order.
+        case (np, cp) if supportedMerge(np) && np.getClass == cp.getClass &&
+          np.children.size == cp.children.size =>
+          val merged = np.children.zip(cp.children).map {
+            case (npChild, cpChild) => tryMergePlans(npChild, cpChild)
+          }
+          if (merged.forall(_.isDefined)) {
+            val (mergedChildren, outputMaps) = merged.map(_.get).unzip
+            val outputMap = AttributeMap(outputMaps.map(_.iterator).reduce(_ ++ _).toSeq)
+            val mappedNewPlan = mapAttributes(np.withNewChildren(mergedChildren), outputMap)
+            val mergedPlan = cp.withNewChildren(mergedChildren)
+            if (mappedNewPlan.canonicalized == mergedPlan.canonicalized) {
+              Some(mergedPlan -> outputMap)
+            } else {
+              None
+            }
+          } else {
+            None
+          }
+
+        // As a follow-up, it would be possible to merge `CommonScalarSubqueries` nodes, which would
+        // allow merging subqueries with mergee subqueries.
+        // E.g. this query:
+        //
+        // SELECT
+        //   (
+        //     SELECT
+        //       (SELECT avg(a) FROM t GROUP BY b) +
+        //       (SELECT sum(b) FROM t GROUP BY b)
+        //   ),
+        //   (
+        //     SELECT
+        //       (SELECT max(a) FROM t GROUP BY b) +
+        //       (SELECT min(b) FROM t GROUP BY b)
+        //   )
+        //
+        // is currently optimized to:
+        //
+        // == Optimized Logical Plan ==
+        // Project [scalar-subquery#233 [] AS scalarsubquery()#255,
+        //          scalar-subquery#236 [] AS scalarsubquery()#256]
+        // :  :- CommonScalarSubqueries [scalar-subquery#264 []]
+        // :  :  :  +- Aggregate [b#238], [named_struct(avg(a), avg(a#237), sum(b), sum(b#238))
+        //                                 AS mergedValue#263]
+        // :  :  :     +- Relation default.t[a#237,b#238] parquet
+        // :  :  +- Project [(scalarsubqueryreference(0, 0, DoubleType, 231) +
+        //                   cast(scalarsubqueryreference(0, 1, LongType, 232) as double))
+        //                   AS (scalarsubquery() + scalarsubquery())#245]
+        // :  :     +- OneRowRelation
+        // :  +- CommonScalarSubqueries [scalar-subquery#269 []]
+        // :     :  +- Aggregate [b#254], [named_struct(min(a), min(a#253), max(b), max(b#254))
+        //                                 AS mergedValue#268]
+        // :     :     +- Relation default.t[a#253,b#254] parquet
+        // :     +- Project [(scalarsubqueryreference(0, 0, IntegerType, 234) +
+        //                   scalarsubqueryreference(0, 1, IntegerType, 235))
+        //                   AS (scalarsubquery() + scalarsubquery())#252]
+        // :        +- OneRowRelation
+        // +- OneRowRelation
+        //
+        // but if we implemented merging `CommonScalarSubqueries` nodes then the plan could be
+        // transformed further and all leaf subqueries could be merged.
+
+        // Otherwise merging is not possible.
+        case _ => None
+      })
+  }
+
+  private def createHeader(headerElements: Seq[(Literal, Attribute)], plan: LogicalPlan): Header = {
+    Project(
+      Seq(Alias(
+        CreateNamedStruct(headerElements.flatMap {
+          case (name, attribute) => Seq(name, attribute)
+        }),
+        "mergedValue")()),
+      plan)
+  }
+
+  private def getHeaderElements(header: Header) = {
+    val mergedValue =
+      header.projectList.head.asInstanceOf[Alias].child.asInstanceOf[CreateNamedStruct]
+    mergedValue.children.grouped(2).map {
+      case Seq(name: Literal, attribute: Attribute) => name -> attribute
+    }.toSeq
+  }
+
+  private def mapAttributes[T <: Expression](expr: T, outputMap: AttributeMap[Attribute]) = {
+    expr.transform {
+      case a: Attribute => outputMap.getOrElse(a, a)
+    }.asInstanceOf[T]
+  }
+
+  private def mapAttributes(plan: LogicalPlan, outputMap: AttributeMap[Attribute]) = {
+    plan.transformExpressions {
+      case a: Attribute => outputMap.getOrElse(a, a)
+    }
+  }
+
+  // Applies `outputMap` attribute mapping on elements of `newExpressions` and merges them into
+  // `cachedExpressions`. Returns the merged expressions and the attribute mapping from the new to
+  // the merged version that can be propagated up during merging nodes.
+  private def mergeNamedExpressions(
+      newExpressions: Seq[NamedExpression],
+      outputMap: AttributeMap[Attribute],
+      cachedExpressions: Seq[NamedExpression]) = {
+    val mergedExpressions = ListBuffer[NamedExpression](cachedExpressions: _*)
+    val newOutputMap = AttributeMap(newExpressions.map { ne =>
+      val mapped = mapAttributes(ne, outputMap)
+      val withoutAlias = mapped match {
+        case Alias(child, _) => child
+        case e => e
+      }
+      ne.toAttribute -> mergedExpressions.find {
+        case Alias(child, _) => child semanticEquals withoutAlias
+        case e => e semanticEquals withoutAlias
+      }.getOrElse {
+        mergedExpressions += mapped
+        mapped
+      }.toAttribute
+    })
+    (mergedExpressions.toSeq, newOutputMap)
+  }
+
+  // Merging different aggregate implementations could cause performance regression
+  private def supportedAggregateMerge(newPlan: Aggregate, cachedPlan: Aggregate) = {
+    val newPlanAggregateExpressions = newPlan.aggregateExpressions.flatMap(_.collect {
+      case a: AggregateExpression => a
+    })
+    val cachedPlanAggregateExpressions = cachedPlan.aggregateExpressions.flatMap(_.collect {
+      case a: AggregateExpression => a
+    })
+    val newPlanSupportsHashAggregate = Aggregate.supportsHashAggregate(
+      newPlanAggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes))
+    val cachedPlanSupportsHashAggregate = Aggregate.supportsHashAggregate(
+      cachedPlanAggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes))
+    newPlanSupportsHashAggregate && cachedPlanSupportsHashAggregate ||
+      !newPlanSupportsHashAggregate && !cachedPlanSupportsHashAggregate && {
+        val newPlanSupportsObjectHashAggregate =
+          Aggregate.supportsObjectHashAggregate(newPlanAggregateExpressions)
+        val cachedPlanSupportsObjectHashAggregate =
+          Aggregate.supportsObjectHashAggregate(cachedPlanAggregateExpressions)
+        newPlanSupportsObjectHashAggregate && cachedPlanSupportsObjectHashAggregate ||
+          !newPlanSupportsObjectHashAggregate && !cachedPlanSupportsObjectHashAggregate
+      }
+  }
+
+  // Whitelist of mergeable general nodes
+  private def supportedMerge(plan: LogicalPlan) = {
+    plan match {
+      case _: Filter => true
+      case _: Join => true
+      case _ => false
+    }
+  }
+
+  // Second traversal replaces `ScalarSubqueryReference`s to either
+  // `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() header))` if the plan is
+  // merged from multiple subqueries or `ScalarSubquery(original plan)` it it isn't.
+  private def removeReferences(
+      plan: LogicalPlan,
+      cache: ListBuffer[(Header, Boolean)]): LogicalPlan = {
+    val nonMergedSubqueriesBefore = cache.scanLeft(0) {
+      case (nonMergedSubqueriesBefore, (_, merged)) =>
+        nonMergedSubqueriesBefore + (if (merged) 0 else 1)
+    }.toArray
+    val newPlan =
+      plan.transformAllExpressionsWithPruning(_.containsAnyPattern(SCALAR_SUBQUERY_REFERENCE)) {
+        case ssr: ScalarSubqueryReference =>
+          val (header, merged) = cache(ssr.subqueryIndex)
+          if (merged) {
+            if (nonMergedSubqueriesBefore(ssr.subqueryIndex) > 0) {
+              ssr.copy(subqueryIndex =
+                ssr.subqueryIndex - nonMergedSubqueriesBefore(ssr.subqueryIndex))
+            } else {
+              ssr
+            }
+          } else {
+            ScalarSubquery(plan = header.child, exprId = ssr.exprId)
+          }
+    }
+    cache.zipWithIndex.collect {
+      case ((_, merged), i) if !merged => i
+    }.reverse.foreach(cache.remove)

Review comment:
       Nit: add a comment that we can replace this in Scala 2.13 with the `filterInPlace` method of `ListBuffer`




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,416 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CommonScalarSubqueries, Filter, Join, LogicalPlan, Project, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s to compute multiple scalar
+ * values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ *   During this first traversal each [[ScalarSubquery]] expression is replaced to a
+ *   [[ScalarSubqueryReference]] pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a results of merging 2 or more
+ *   plans, or it is a plan that was seen only once.
+ *   Merged plans in the cache get a "header", that is is basically
+ *   `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in new root
+ *   [[Project]] node. This expression ensures that the merged plan is a valid scalar subquery that
+ *   returns only one value.
+ * - A second traversal checks if a [[ScalarSubqueryReference]] is pointing to a merged subquery
+ *   plan or not and either keeps the reference or restores the original [[ScalarSubquery]].
+ *   If there are [[ScalarSubqueryReference]] nodes remained a [[CommonScalarSubqueries]] root node
+ *   is added to the plan with the referenced scalar subqueries.
+ * - [[PlanSubqueries]] or [[PlanAdaptiveSubqueries]] rule does the physical planning of scalar
+ *   subqueries including the ones under [[CommonScalarSubqueriesExec]] node and replaces
+ *   each [[ScalarSubqueryReference]] to their referenced physical plan in
+ *   `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() header))` form.
+ *   It is important that references pointing to the same merged subquery are replaced to the same
+ *   planned instance to make sure that each merged subquery runs only once (even without a wrapping
+ *   [[ReuseSubquery]] node).
+ *   Finally, the [[CommonScalarSubqueriesExec]] node is removed from the physical plan.
+ * - The [[ReuseExchangeAndSubquery]] rule wraps the second, third, ... instances of the same
+ *   subquery into a [[ReuseSubquery]] node, but this just a cosmetic change in the plan.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *          scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * CommonScalarSubqueries [scalar-subquery#250 []]
+ * :  +- Project [named_struct(avg(a), avg(a)#236, sum(b), sum(b)#238L) AS mergedValue#249]
+ * :     +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :        +- Project [a#233, b#234]
+ * :           +- Relation default.t[a#233,b#234] parquet
+ * +- Project [scalarsubqueryreference(0, 0, DoubleType, 231) AS scalarsubquery()#241,
+ *             scalarsubqueryreference(0, 1, LongType, 232) AS scalarsubquery()#242L]
+ *    +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    if (conf.subqueryReuseEnabled) {
+      plan match {
+        case Subquery(_: CommonScalarSubqueries, _) => plan
+        case s: Subquery => s.copy(child = extractCommonScalarSubqueries(s.child))
+        case _: CommonScalarSubqueries => plan
+        case _ => extractCommonScalarSubqueries(plan)
+      }
+    } else {
+      plan
+    }
+  }
+
+  type Header = Project
+
+  private def extractCommonScalarSubqueries(plan: LogicalPlan) = {
+    // Plan of subqueries and a flag is the plan is merged
+    val cache = ListBuffer.empty[(Header, Boolean)]
+    val newPlan = removeReferences(insertReferences(plan, cache), cache)
+    if (cache.nonEmpty) {
+      val scalarSubqueries = cache.map { case (header, _) => ScalarSubquery(header) }.toSeq
+      CommonScalarSubqueries(scalarSubqueries, newPlan)
+    } else {
+      newPlan
+    }
+  }
+
+  // First traversal builds up the cache and inserts `ScalarSubqueryReference`s to the plan.
+  private def insertReferences(
+      plan: LogicalPlan,
+      cache: ListBuffer[(Header, Boolean)]): LogicalPlan = {
+    plan.transformAllExpressionsWithPruning(_.containsAnyPattern(SCALAR_SUBQUERY)) {
+      case s: ScalarSubquery if s.children.isEmpty =>
+        val (subqueryIndex, headerIndex) = cacheSubquery(s.plan, cache)
+        ScalarSubqueryReference(subqueryIndex, headerIndex, s.dataType, s.exprId)
+    }
+  }
+
+  // Caching returns the index of the subquery in the cache and the index of scalar member in the
+  // "header", that `CreateNamedStruct(name1, attribute1, name2, attribute2, ...)` expression in a
+  // [[Project]] node.
+  private def cacheSubquery(
+      plan: LogicalPlan,
+      cache: ListBuffer[(Header, Boolean)]): (Int, Int) = {
+    val firstOutput = plan.output.head
+    cache.zipWithIndex.collectFirst(Function.unlift { case ((header, merged), subqueryIndex) =>
+      checkIdenticalPlans(plan, header.child)
+        .map((subqueryIndex, header, header.child, _, merged))
+        .orElse(tryMergePlans(plan, header.child).map {
+          case (mergedPlan, outputMap) => (subqueryIndex, header, mergedPlan, outputMap, true)
+        })
+    }).map { case (subqueryIndex, header, mergedPlan, outputMap, merged) =>
+      val mappedFirstOutput = mapAttributes(firstOutput, outputMap)
+      val headerElements = getHeaderElements(header)
+      var headerIndex = headerElements.indexWhere {
+        case (_, attribute) => attribute.exprId == mappedFirstOutput.exprId
+      }
+      if (headerIndex == -1) {
+        val newHeaderElements = headerElements :+ (Literal(firstOutput.name) -> mappedFirstOutput)
+        cache(subqueryIndex) = createHeader(newHeaderElements, mergedPlan) -> merged
+        headerIndex = headerElements.size
+      }
+      subqueryIndex -> headerIndex
+    }.getOrElse {
+      cache += createHeader(Seq(Literal(firstOutput.name) -> firstOutput), plan) -> false
+      cache.length - 1 -> 0
+    }
+  }
+
+  // If 2 plans are identical return the attribute mapping from the new to the cached version.
+  private def checkIdenticalPlans(newPlan: LogicalPlan, cachedPlan: LogicalPlan) = {
+    if (newPlan.canonicalized == cachedPlan.canonicalized) {
+      Some(AttributeMap(newPlan.output.zip(cachedPlan.output)))
+    } else {
+      None
+    }
+  }
+
+  // Recursively traverse down and try merging 2 plans. If merge is possible then return the merged
+  // plan with the attribute mapping from the new to the merged version.
+  // Please note that merging arbitrary plans can be complicated, the current version supports only
+  // some of the most important nodes.
+  private def tryMergePlans(
+      newPlan: LogicalPlan,
+      cachedPlan: LogicalPlan): Option[(LogicalPlan, AttributeMap[Attribute])] = {
+    checkIdenticalPlans(newPlan, cachedPlan).map(cachedPlan -> _).orElse(
+      (newPlan, cachedPlan) match {
+        case (np: Project, cp: Project) =>
+          tryMergePlans(np.child, cp.child).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.projectList, outputMap, cp.projectList)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np, cp: Project) =>
+          tryMergePlans(np, cp.child).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.output, outputMap, cp.projectList)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np: Project, cp) =>
+          tryMergePlans(np.child, cp).map { case (mergedChild, outputMap) =>
+            val (mergedProjectList, newOutputMap) =
+              mergeNamedExpressions(np.projectList, outputMap, cp.output)
+            val mergedPlan = Project(mergedProjectList, mergedChild)
+            mergedPlan -> newOutputMap
+          }
+        case (np: Aggregate, cp: Aggregate) if supportedAggregateMerge(np, cp) =>
+          tryMergePlans(np.child, cp.child).flatMap { case (mergedChild, outputMap) =>
+            val mappedNewGroupingExpression =
+              np.groupingExpressions.map(mapAttributes(_, outputMap))
+            if (ExpressionSet(mappedNewGroupingExpression) ==
+              ExpressionSet(cp.groupingExpressions)) {
+              val (mergedAggregateExpressions, newOutputMap) =
+                mergeNamedExpressions(np.aggregateExpressions, outputMap, cp.aggregateExpressions)
+              val mergedPlan =
+                Aggregate(cp.groupingExpressions, mergedAggregateExpressions, mergedChild)
+              Some(mergedPlan -> newOutputMap)
+            } else {
+              None
+            }
+          }
+
+        // Merging general nodes is complicated and this implementation supports only those nodes in
+        // which the order and the number of output attributes are not relevant (see
+        // `supportedMerge()` whitelist).
+        // Also, this implementation supports only those nodes in which children can be merged in
+        // the same order.
+        case (np, cp) if supportedMerge(np) && np.getClass == cp.getClass &&
+          np.children.size == cp.children.size =>
+          val merged = np.children.zip(cp.children).map {
+            case (npChild, cpChild) => tryMergePlans(npChild, cpChild)
+          }
+          if (merged.forall(_.isDefined)) {
+            val (mergedChildren, outputMaps) = merged.map(_.get).unzip
+            val outputMap = AttributeMap(outputMaps.map(_.iterator).reduce(_ ++ _).toSeq)
+            val mappedNewPlan = mapAttributes(np.withNewChildren(mergedChildren), outputMap)
+            val mergedPlan = cp.withNewChildren(mergedChildren)
+            if (mappedNewPlan.canonicalized == mergedPlan.canonicalized) {
+              Some(mergedPlan -> outputMap)
+            } else {
+              None
+            }
+          } else {
+            None
+          }
+
+        // As a follow-up, it would be possible to merge `CommonScalarSubqueries` nodes, which would
+        // allow merging subqueries with mergee subqueries.
+        // E.g. this query:
+        //
+        // SELECT
+        //   (
+        //     SELECT
+        //       (SELECT avg(a) FROM t GROUP BY b) +
+        //       (SELECT sum(b) FROM t GROUP BY b)
+        //   ),
+        //   (
+        //     SELECT
+        //       (SELECT max(a) FROM t GROUP BY b) +
+        //       (SELECT min(b) FROM t GROUP BY b)
+        //   )
+        //
+        // is currently optimized to:
+        //
+        // == Optimized Logical Plan ==
+        // Project [scalar-subquery#233 [] AS scalarsubquery()#255,
+        //          scalar-subquery#236 [] AS scalarsubquery()#256]
+        // :  :- CommonScalarSubqueries [scalar-subquery#264 []]
+        // :  :  :  +- Aggregate [b#238], [named_struct(avg(a), avg(a#237), sum(b), sum(b#238))
+        //                                 AS mergedValue#263]
+        // :  :  :     +- Relation default.t[a#237,b#238] parquet
+        // :  :  +- Project [(scalarsubqueryreference(0, 0, DoubleType, 231) +
+        //                   cast(scalarsubqueryreference(0, 1, LongType, 232) as double))
+        //                   AS (scalarsubquery() + scalarsubquery())#245]
+        // :  :     +- OneRowRelation
+        // :  +- CommonScalarSubqueries [scalar-subquery#269 []]
+        // :     :  +- Aggregate [b#254], [named_struct(min(a), min(a#253), max(b), max(b#254))
+        //                                 AS mergedValue#268]
+        // :     :     +- Relation default.t[a#253,b#254] parquet
+        // :     +- Project [(scalarsubqueryreference(0, 0, IntegerType, 234) +
+        //                   scalarsubqueryreference(0, 1, IntegerType, 235))
+        //                   AS (scalarsubquery() + scalarsubquery())#252]
+        // :        +- OneRowRelation
+        // +- OneRowRelation
+        //
+        // but if we implemented merging `CommonScalarSubqueries` nodes then the plan could be
+        // transformed further and all leaf subqueries could be merged.
+
+        // Otherwise merging is not possible.
+        case _ => None
+      })
+  }
+
+  private def createHeader(headerElements: Seq[(Literal, Attribute)], plan: LogicalPlan): Header = {
+    Project(
+      Seq(Alias(
+        CreateNamedStruct(headerElements.flatMap {
+          case (name, attribute) => Seq(name, attribute)
+        }),
+        "mergedValue")()),
+      plan)
+  }
+
+  private def getHeaderElements(header: Header) = {
+    val mergedValue =
+      header.projectList.head.asInstanceOf[Alias].child.asInstanceOf[CreateNamedStruct]
+    mergedValue.children.grouped(2).map {
+      case Seq(name: Literal, attribute: Attribute) => name -> attribute
+    }.toSeq
+  }
+
+  private def mapAttributes[T <: Expression](expr: T, outputMap: AttributeMap[Attribute]) = {
+    expr.transform {
+      case a: Attribute => outputMap.getOrElse(a, a)
+    }.asInstanceOf[T]
+  }
+
+  private def mapAttributes(plan: LogicalPlan, outputMap: AttributeMap[Attribute]) = {
+    plan.transformExpressions {
+      case a: Attribute => outputMap.getOrElse(a, a)
+    }
+  }
+
+  // Applies `outputMap` attribute mapping on elements of `newExpressions` and merges them into
+  // `cachedExpressions`. Returns the merged expressions and the attribute mapping from the new to
+  // the merged version that can be propagated up during merging nodes.
+  private def mergeNamedExpressions(
+      newExpressions: Seq[NamedExpression],
+      outputMap: AttributeMap[Attribute],
+      cachedExpressions: Seq[NamedExpression]) = {
+    val mergedExpressions = ListBuffer[NamedExpression](cachedExpressions: _*)
+    val newOutputMap = AttributeMap(newExpressions.map { ne =>
+      val mapped = mapAttributes(ne, outputMap)
+      val withoutAlias = mapped match {
+        case Alias(child, _) => child
+        case e => e
+      }
+      ne.toAttribute -> mergedExpressions.find {
+        case Alias(child, _) => child semanticEquals withoutAlias
+        case e => e semanticEquals withoutAlias
+      }.getOrElse {
+        mergedExpressions += mapped
+        mapped
+      }.toAttribute
+    })
+    (mergedExpressions.toSeq, newOutputMap)
+  }
+
+  // Merging different aggregate implementations could cause performance regression
+  private def supportedAggregateMerge(newPlan: Aggregate, cachedPlan: Aggregate) = {
+    val newPlanAggregateExpressions = newPlan.aggregateExpressions.flatMap(_.collect {
+      case a: AggregateExpression => a
+    })
+    val cachedPlanAggregateExpressions = cachedPlan.aggregateExpressions.flatMap(_.collect {
+      case a: AggregateExpression => a
+    })
+    val newPlanSupportsHashAggregate = Aggregate.supportsHashAggregate(
+      newPlanAggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes))
+    val cachedPlanSupportsHashAggregate = Aggregate.supportsHashAggregate(
+      cachedPlanAggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes))
+    newPlanSupportsHashAggregate && cachedPlanSupportsHashAggregate ||
+      !newPlanSupportsHashAggregate && !cachedPlanSupportsHashAggregate && {
+        val newPlanSupportsObjectHashAggregate =
+          Aggregate.supportsObjectHashAggregate(newPlanAggregateExpressions)
+        val cachedPlanSupportsObjectHashAggregate =
+          Aggregate.supportsObjectHashAggregate(cachedPlanAggregateExpressions)
+        newPlanSupportsObjectHashAggregate && cachedPlanSupportsObjectHashAggregate ||
+          !newPlanSupportsObjectHashAggregate && !cachedPlanSupportsObjectHashAggregate
+      }
+  }
+
+  // Whitelist of mergeable general nodes
+  private def supportedMerge(plan: LogicalPlan) = {
+    plan match {
+      case _: Filter => true
+      case _: Join => true
+      case _ => false
+    }
+  }
+
+  // Second traversal replaces `ScalarSubqueryReference`s to either
+  // `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() header))` if the plan is
+  // merged from multiple subqueries or `ScalarSubquery(original plan)` it it isn't.
+  private def removeReferences(
+      plan: LogicalPlan,
+      cache: ListBuffer[(Header, Boolean)]): LogicalPlan = {
+    val nonMergedSubqueriesBefore = cache.scanLeft(0) {
+      case (nonMergedSubqueriesBefore, (_, merged)) =>
+        nonMergedSubqueriesBefore + (if (merged) 0 else 1)
+    }.toArray
+    val newPlan =
+      plan.transformAllExpressionsWithPruning(_.containsAnyPattern(SCALAR_SUBQUERY_REFERENCE)) {
+        case ssr: ScalarSubqueryReference =>
+          val (header, merged) = cache(ssr.subqueryIndex)
+          if (merged) {
+            if (nonMergedSubqueriesBefore(ssr.subqueryIndex) > 0) {
+              ssr.copy(subqueryIndex =
+                ssr.subqueryIndex - nonMergedSubqueriesBefore(ssr.subqueryIndex))
+            } else {
+              ssr
+            }
+          } else {
+            ScalarSubquery(plan = header.child, exprId = ssr.exprId)
+          }
+    }
+    cache.zipWithIndex.collect {
+      case ((_, merged), i) if !merged => i
+    }.reverse.foreach(cache.remove)

Review comment:
       Ok, added in https://github.com/apache/spark/pull/32298/commits/63c370966577985ebfde03b4bd309fda85ca909b.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/InjectRuntimeFilterSuite.scala
##########
@@ -253,20 +253,14 @@ class InjectRuntimeFilterSuite extends QueryTest with SQLTestUtils with SharedSp
   }
 
   def getNumBloomFilters(plan: LogicalPlan): Integer = {
-    val numBloomFilterAggs = plan.collect {
-      case Filter(condition, _) => condition.collect {
-        case subquery: org.apache.spark.sql.catalyst.expressions.ScalarSubquery
-        => subquery.plan.collect {
-          case Aggregate(_, aggregateExpressions, _) =>
-            aggregateExpressions.map {
-              case Alias(AggregateExpression(bfAgg : BloomFilterAggregate, _, _, _, _),
-              _) =>
-                assert(bfAgg.estimatedNumItemsExpression.isInstanceOf[Literal])
-                assert(bfAgg.numBitsExpression.isInstanceOf[Literal])
-                1
-            }.sum
+    val numBloomFilterAggs = plan.collectWithSubqueries {

Review comment:
       Ditto. Let me know it this check is too loose.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] sigmod commented on pull request #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


   > `CTERef` are invented for 1 special purpose
   
   - (1) yes, we need CTERef for correctness when a CTE definition is non-deterministic;
   - (2) however, `CTERef` is also a primitive for de-duplicate common plan subtrees. Those plan trees to be shared do not have to be identical, e.g., one can merge filter predicates with `OR` and union needed columns into a single, shared CTE definition. Other query engines do that, even though Spark doesn't do that for now. E.g., this paper describes such optimizations: http://www.vldb.org/pvldb/vol8/p1704-elhelw.pdf. 
   - (3) I think what this PR does is a special case of (2). E.g., if you have two plan subtrees (within the same query plan, but not subqueries) run different aggregations over the same table with the same grouping exprs, we can use `CTERef` but not `CommonSubqueries` to share the scan and computation. 
   - (4) subqueries might present more optimization opportunities, but I think the additional optimizations would better come up in physical plans rather than logical plans.
   
   > I'm not sure I get this. Why ColumnPruning should consider these new nodes?
   
   There's a pattern matching for CTE:
   https://github.com/apache/spark/blob/efe43306fcab18f076f755c81c0406ebc1a5fee9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala#L874-L880
   Similarly, if a scalar subquery reference was pruned by some other optimizations, we may want to remove the subquery too.
   
   > how about combining them into one node (CommonDefinitions?) that can host CTEs and scalar subqueries as well?
   
   The difference seems minor at logical level - but the latter avoid things like SubqueryReference:
   
   CommonDef
     +- Seq(Subquery)
   
   v.s.
   CommonDef
     +- Seq(Plan)
   but wrap CTE into a scalar subquery of (Select .. FROM cte) at the place of original 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


   > * (2) however, `CTERef` is also a primitive for de-duplicate common plan subtrees. Those plan trees to be shared do not have to be identical, e.g., one can merge filter predicates with `OR` and union needed columns into a single, shared CTE definition. Other query engines do that, even though Spark doesn't do that for now. E.g., this paper describes such optimizations: http://www.vldb.org/pvldb/vol8/p1704-elhelw.pdf.
   > * (3) I think what this PR does is a special case of (2). E.g., if you have two plan subtrees (within the same query plan, but not subqueries) run different aggregations over the same table with the same grouping exprs, we can use `CTERef` but not `CommonSubqueries` to share the scan and computation.
   
   In think there are 2 different things here:
   - The merging logic (`tryMergePlans()`) in this PR is general enough to handle common plan subtrees. It can be improved of course and actually I have a follow-up PR to support merging different filter predicates with `OR`, just didn't want this PR more complex. I think the logic can be extracted from `MergeScalarSubqueries` to a common place and used in follow-up PRs for different purposes. Like, it can be used for the general (2) and in that case `CTERef` is the only viable way to reference common parts, indeed.
   - Merging subqueries seems to be much simpler to start with (than merging arbitrary common parts of a plan). It is well defined which plan should be tried to merged with another plan. This PR wants to deal with that scope only. But for this limited scope `CTERef`'s physical implementation seems to be a bit overkill, but a simple expression a right fit.
   
   > * (4) subqueries might present more optimization opportunities, but I think the additional optimizations would better come up in physical plans rather than logical plans.
   > 
   > > I'm not sure I get this. Why ColumnPruning should consider these new nodes?
   > 
   > There's a pattern matching for CTE:
   > 
   > https://github.com/apache/spark/blob/efe43306fcab18f076f755c81c0406ebc1a5fee9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala#L874-L880
   > Similarly, if a scalar subquery reference was pruned by some other optimizations, we may want to remove the subquery too.
   
   I think this code only prunes the plan child of `WithCTE` not the CTE definitions, but indeed, we need a `case p @ Project(_, w: <definition hosting node>) =>` pattern there.
   I'm not sure as in that case `PlanSubqueries` / `PlanAdaptiveSubqueries` simply don't add the physical `ScalarSubquery` to the plan. Likewise, we don't seem to remove any CTE definition that are no longer needed.
   
   > > how about combining them into one node (CommonDefinitions?) that can host CTEs and scalar subqueries as well?
   > 
   > The difference seems minor at logical level - but the latter avoid things like SubqueryReference:
   > 
   > CommonDef +- Seq(Subquery)
   > 
   > v.s. CommonDef +- Seq(Plan) but wrap CTE into a scalar subquery of (Select .. FROM cte) at the place of original subqueries.
   
   I don't feel that `SubqueryReference` is bad and we need to avoid it, but if others also suggest removing it, I can change the implementation to use only `WithCTE` + `CTERef`s.
   
   Please also consider my alternative proposal to rename `WithCTE` to `CommonDef` and keep `CTERef` for common query parts and `SubqueryReference` for common subquery expressions:
   
   ```
   CommonDef
   : +- Seq(Subquery)
   +- Seq(Plan)
   ```
   


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


   > @peter-toth you said you updated performance numbers, was there a difference after changing implementation?
   
   No, this PR still brings the same performance improvement.


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


   @tgravescs, thanks for your interesting in in this PR. I reran the tests, they seem to be fine now.
   This change adds basic query merging logic and affects only `q9` from TPCDS, but it brings considerable performance improvement to it.
   This PR has been ready for review for a while, but haven't got much attention. Your and others review would be much appreciated.
   
   I also have some follow-up PRs/plans:
   - to add more advanced logic to support merging aggregate subqueries with different `Filter` nodes
   - support DSv2
   - try merging correlated subqueries
   
   but this PR is complex enough so I would add these in subsequent PRs only.
   
   We can target 3.3, but AFAIK the branch has been cut so probably we need some kind of approval on the dev list?
   


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] tgravescs commented on pull request #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries

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


   sent email to dev for 3.3 to see if any concerns with this.  Hopefully I can finish reviewing today.  
   We ran through a bunch of tests and it looks good from that testing.}
   
   Agree anything else should be in followup PRs, you could file an issue for them if you haven't 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] zinking commented on a change in pull request #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,413 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CommonScalarSubqueries, Filter, Join, LogicalPlan, Project, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s to compute multiple scalar
+ * values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ *   During this first traversal each [[ScalarSubquery]] expression is replaced to a
+ *   [[ScalarSubqueryReference]] pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a results of merging 2 or more
+ *   plans, or it is a plan that was seen only once.
+ *   Merged plans in the cache get a "header" that is is basically

Review comment:
       can we duplicate this `header` comment to where it is actually used ? 




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


   **[Test build #138260 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138260/testReport)** for PR 32298 at commit [`1f2f75c`](https://github.com/apache/spark/commit/1f2f75c9c52f7494db4e3bb15368c87d9b8e17c7).
    * This patch **fails SparkR 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] AmplabJenkins commented on pull request #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LeafNode, LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{MULTI_SCALAR_SUBQUERY, SCALAR_SUBQUERY}
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s into a
+ * [[MultiScalarSubquery]] to compute multiple scalar values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ * - The original [[ScalarSubquery]] expression is replaced to a reference pointing to its cached
+ *   version in this form: `GetStructField(MultiScalarSubquery(SubqueryReference(...)))`.
+ * - A second traversal checks if a [[SubqueryReference]] is pointing to a subquery plan that
+ *   returns multiple values and either replaces only [[SubqueryReference]] to the cached plan or
+ *   restores the whole expression to its original [[ScalarSubquery]] form.
+ * - [[ReuseSubquery]] rule makes sure that merged subqueries are computed once.
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t GROUP BY b),
+ *   (SELECT sum(b) FROM t GROUP BY b)
+ *
+ * is optimized from:
+ *
+ * Project [scalar-subquery#231 [] AS scalarsubquery()#241,
+ *   scalar-subquery#232 [] AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
+ * :  :  +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
+ * :     +- Project [b#240]
+ * :        +- Relation default.t[a#239,b#240] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * Project [multi-scalar-subquery#231.avg(a) AS scalarsubquery()#241,
+ *   multi-scalar-subquery#232.sum(b) AS scalarsubquery()#242L]
+ * :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :  :  +- Project [a#233, b#234]
+ * :  :     +- Relation default.t[a#233,b#234] parquet
+ * :  +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
+ * :     +- Project [a#233, b#234]
+ * :        +- Relation default.t[a#233,b#234] parquet
+ * +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    if (conf.scalarSubqueryMergeEabled && conf.subqueryReuseEnabled) {
+      val mergedSubqueries = ArrayBuffer.empty[LogicalPlan]
+      removeReferences(mergeAndInsertReferences(plan, mergedSubqueries), mergedSubqueries)
+    } else {
+      plan
+    }
+  }
+
+  private def mergeAndInsertReferences(
+      plan: LogicalPlan,
+      mergedSubqueries: ArrayBuffer[LogicalPlan]): LogicalPlan = {
+    plan.transformAllExpressionsWithPruning(_.containsAnyPattern(SCALAR_SUBQUERY), ruleId) {
+      case s: ScalarSubquery if s.children.isEmpty =>
+        val (mergedPlan, ordinal) = mergeAndGetReference(s.plan, mergedSubqueries)
+        GetStructField(MultiScalarSubquery(mergedPlan, s.exprId), ordinal)
+    }
+  }
+
+  case class SubqueryReference(
+      index: Int,
+      mergedSubqueries: ArrayBuffer[LogicalPlan]) extends LeafNode {
+    override def stringArgs: Iterator[Any] = Iterator(index)
+
+    override def output: Seq[Attribute] = mergedSubqueries(index).output
+  }
+
+  private def mergeAndGetReference(
+      plan: LogicalPlan,
+      mergedSubqueries: ArrayBuffer[LogicalPlan]): (SubqueryReference, Int) = {
+    mergedSubqueries.zipWithIndex.collectFirst {
+      Function.unlift { case (s, i) => mergePlans(plan, s).map(_ -> i) }
+    }.map { case ((mergedPlan, outputMap), i) =>
+      mergedSubqueries(i) = mergedPlan
+      SubqueryReference(i, mergedSubqueries) ->
+        mergedPlan.output.indexOf(outputMap(plan.output.head))
+    }.getOrElse {
+      mergedSubqueries += plan
+      SubqueryReference(mergedSubqueries.length - 1, mergedSubqueries) -> 0
+    }
+  }
+
+  private def mergePlans(
+      newPlan: LogicalPlan,
+      existingPlan: LogicalPlan): Option[(LogicalPlan, AttributeMap[Attribute])] = {
+    (newPlan, existingPlan) match {
+      case (np, ep) if np.canonicalized == ep.canonicalized =>
+        Some(ep -> AttributeMap(np.output.zip(ep.output)))
+      case (np: Project, ep: Project) =>
+        mergePlans(np.child, ep.child).map { case (mergedChild, outputMap) =>
+          val newProjectList = replaceAttributes(np.projectList, outputMap)
+          val newOutputMap = createOutputMap(np.projectList, newProjectList)
+          Project(distinctExpressions(ep.projectList ++ newProjectList), mergedChild) ->
+            newOutputMap
+        }
+      case (np, ep: Project) =>
+        mergePlans(np, ep.child).map { case (mergedChild, outputMap) =>
+          Project(distinctExpressions(ep.projectList ++ outputMap.values), mergedChild) -> outputMap
+        }
+      case (np: Project, ep) =>
+        mergePlans(np.child, ep).map { case (mergedChild, outputMap) =>
+          val newProjectList = replaceAttributes(np.projectList, outputMap)
+          val newOutputMap = createOutputMap(np.projectList, newProjectList)
+          Project(distinctExpressions(ep.output ++ newProjectList), mergedChild) -> newOutputMap
+        }

Review comment:
       A good example is in the description of this PR:
   ```
   SELECT
     (SELECT avg(a) FROM t GROUP BY b),
     (SELECT sum(b) FROM t GROUP BY b)
   ```
   Where we have an additional `Project [b#240]` in the second subquery due to column pruning:
   ```
   Project [scalar-subquery#231 [] AS scalarsubquery()#241, scalar-subquery#232 [] AS scalarsubquery()#242L]
   :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236]
   :  :  +- Relation default.t[a#233,b#234] parquet
   :  +- Aggregate [b#240], [sum(b#240) AS sum(b)#238L]
   :     +- Project [b#240]
   :        +- Relation default.t[a#239,b#240] parquet
   +- OneRowRelation
   ```
   and this rule can merge the 2 queries as:
   ```
   Project [multi-scalar-subquery#231.avg(a) AS scalarsubquery()#241, multi-scalar-subquery#232.sum(b) AS scalarsubquery()#242L]
   :  :- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
   :  :  +- Project [a#233, b#234]
   :  :     +- Relation default.t[a#233,b#234] parquet
   :  +- Aggregate [b#234], [avg(a#233) AS avg(a)#236, sum(b#234) AS sum(b)#238L]
   :     +- Project [a#233, b#234]
   :        +- Relation default.t[a#233,b#234] parquet
   +- OneRowRelation
   ```
   IMO the above 2 cases are safe and just handle the cases when there is an extra `Project` node in one of the plans but the child plan under the `Project` and the plan of the other side are mergeable. In these cases the merged plan should contain the `Project` node but it should also contain the output of the other side transparently.
   
   I will add this test case to `SubquerySuite`.




-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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






-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


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


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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


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


-- 
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 #32298: [WIP][SPARK-34079][SQL] Merge non-correlated scalar subqueries to multi-column scalar subqueries for better reuse

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


   **[Test build #139122 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/139122/testReport)** for PR 32298 at commit [`9d8dd6b`](https://github.com/apache/spark/commit/9d8dd6bc7bca56a11878dcccb5a5186d09e9f67b).


-- 
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] zinking commented on a change in pull request #32298: [SPARK-34079][SQL] Merge non-correlated scalar subqueries for better reuse

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala
##########
@@ -0,0 +1,413 @@
+/*
+ * 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.catalyst.optimizer
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CommonScalarSubqueries, Filter, Join, LogicalPlan, Project, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * This rule tries to merge multiple non-correlated [[ScalarSubquery]]s to compute multiple scalar
+ * values once.
+ *
+ * The process is the following:
+ * - While traversing through the plan each [[ScalarSubquery]] plan is tried to merge into the cache
+ *   of already seen subquery plans. If merge is possible then cache is updated with the merged
+ *   subquery plan, if not then the new subquery plan is added to the cache.
+ *   During this first traversal each [[ScalarSubquery]] expression is replaced to a
+ *   [[ScalarSubqueryReference]] pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a results of merging 2 or more
+ *   plans, or it is a plan that was seen only once.
+ *   Merged plans in the cache get a "header" that is is basically

Review comment:
       can we duplicate this `header` comment to where it is actually used ? I mean like just above the function so that one can catch the context more easily




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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