sigmod commented on code in PR #32298:
URL: https://github.com/apache/spark/pull/32298#discussion_r651499875


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala:
##########
@@ -0,0 +1,401 @@
+/*
+ * 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, 
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]] rules do the physical 
planning of scalar
+ *   subqueries including the ones under [[CommonScalarSubqueriesExec]] node 
and replace
+ *   each [[ScalarSubqueryReference]] to their referenced physical plan in
+ *   `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() 
header))` form. Finally,
+ *   the [[CommonScalarSubqueriesExec]] node is removed from the physical plan.
+ * - [[ReuseSubquery]] rule makes sure that merged subqueries are computed 
only 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 [scalar-subquery#231 [].avg(a) AS scalarsubquery()#241,
+ *          scalar-subquery#232 [].sum(b) AS scalarsubquery()#242L]
+ * :  :- 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 [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
+ * +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    if (conf.scalarSubqueryMergeEabled && conf.subqueryReuseEnabled) {

Review Comment:
   Flip if condition and reduce nesting?
   https://github.com/databricks/scala-style-guide#return-statements



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala:
##########
@@ -0,0 +1,401 @@
+/*
+ * 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, 
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]] rules do the physical 
planning of scalar
+ *   subqueries including the ones under [[CommonScalarSubqueriesExec]] node 
and replace
+ *   each [[ScalarSubqueryReference]] to their referenced physical plan in
+ *   `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() 
header))` form. Finally,
+ *   the [[CommonScalarSubqueriesExec]] node is removed from the physical plan.
+ * - [[ReuseSubquery]] rule makes sure that merged subqueries are computed 
only 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 [scalar-subquery#231 [].avg(a) AS scalarsubquery()#241,

Review Comment:
   Can you update the resulting plan?



##########
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueriesSuite.scala:
##########
@@ -0,0 +1,355 @@
+/*
+ * 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 org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.expressions.{Attribute, 
CreateNamedStruct, GetStructField, Literal, ScalarSubquery}
+import org.apache.spark.sql.catalyst.expressions.aggregate.{CollectList, 
CollectSet}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules._
+
+class MergeScalarSubqueriesSuite extends PlanTest {
+
+  override def beforeEach(): Unit = {
+    CTERelationDef.curId.set(0)
+  }
+
+  private object Optimize extends RuleExecutor[LogicalPlan] {
+    val batches = Batch("MergeScalarSubqueries", Once, MergeScalarSubqueries) 
:: Nil
+  }
+
+  val testRelation = LocalRelation('a.int, 'b.int, 'c.string)
+
+  private def extractorExpression(cteIndex: Int, output: Seq[Attribute], 
fieldIndex: Int) = {
+    GetStructField(
+      ScalarSubquery(CTERelationRef(cteIndex, _resolved = true, output, 
subquery = true)),
+      fieldIndex).as("scalarsubquery()")
+  }
+
+  test("Merging subqueries with projects") {
+    val subquery1 = ScalarSubquery(testRelation.select(('a + 1).as("a_plus1")))
+    val subquery2 = ScalarSubquery(testRelation.select(('a + 2).as("a_plus2")))
+    val subquery3 = ScalarSubquery(testRelation.select('b))
+    val subquery4 = ScalarSubquery(testRelation.select(('a + 
1).as("a_plus1_2")))
+    val subquery5 = ScalarSubquery(testRelation.select(('a + 
2).as("a_plus2_2")))
+    val subquery6 = ScalarSubquery(testRelation.select('b.as("b_2")))
+    val originalQuery = testRelation
+      .select(
+        subquery1,
+        subquery2,
+        subquery3,
+        subquery4,
+        subquery5,
+        subquery6)
+
+    val mergedSubquery = testRelation
+      .select(
+        ('a + 1).as("a_plus1"),
+        ('a + 2).as("a_plus2"),
+        'b)
+      .select(
+        CreateNamedStruct(Seq(
+          Literal("a_plus1"), 'a_plus1,
+          Literal("a_plus2"), 'a_plus2,
+          Literal("b"), 'b
+        )).as("mergedValue"))
+    val analyzedMergedSubquery = mergedSubquery.analyze
+    val correctAnswer = WithCTE(
+      testRelation
+        .select(
+          extractorExpression(0, analyzedMergedSubquery.output, 0),
+          extractorExpression(0, analyzedMergedSubquery.output, 1),
+          extractorExpression(0, analyzedMergedSubquery.output, 2),
+          extractorExpression(0, analyzedMergedSubquery.output, 0),
+          extractorExpression(0, analyzedMergedSubquery.output, 1),
+          extractorExpression(0, analyzedMergedSubquery.output, 2)),
+      Seq(CTERelationDef(analyzedMergedSubquery, 0)))
+
+    comparePlans(Optimize.execute(originalQuery.analyze), 
correctAnswer.analyze)
+  }
+
+  test("Merging subqueries with aggregates") {
+    val subquery1 = 
ScalarSubquery(testRelation.groupBy('b)(max('a).as("max_a")))
+    val subquery2 = 
ScalarSubquery(testRelation.groupBy('b)(sum('a).as("sum_a")))
+    val subquery3 = ScalarSubquery(testRelation.groupBy('b)('b))
+    val subquery4 = 
ScalarSubquery(testRelation.groupBy('b)(max('a).as("max_a_2")))
+    val subquery5 = 
ScalarSubquery(testRelation.groupBy('b)(sum('a).as("sum_a_2")))
+    val subquery6 = ScalarSubquery(testRelation.groupBy('b)('b.as("b_2")))
+    val originalQuery = testRelation
+      .select(
+        subquery1,
+        subquery2,
+        subquery3,
+        subquery4,
+        subquery5,
+        subquery6)
+
+    val mergedSubquery = testRelation
+      .groupBy('b)(
+        max('a).as("max_a"),
+        sum('a).as("sum_a"),
+        'b)
+      .select(CreateNamedStruct(Seq(
+        Literal("max_a"), 'max_a,
+        Literal("sum_a"), 'sum_a,
+        Literal("b"), 'b
+      )).as("mergedValue"))
+    val analyzedMergedSubquery = mergedSubquery.analyze
+    val correctAnswer = WithCTE(
+      testRelation
+        .select(
+          extractorExpression(0, analyzedMergedSubquery.output, 0),
+          extractorExpression(0, analyzedMergedSubquery.output, 1),
+          extractorExpression(0, analyzedMergedSubquery.output, 2),
+          extractorExpression(0, analyzedMergedSubquery.output, 0),
+          extractorExpression(0, analyzedMergedSubquery.output, 1),
+          extractorExpression(0, analyzedMergedSubquery.output, 2)),
+      Seq(CTERelationDef(analyzedMergedSubquery, 0)))
+
+    comparePlans(Optimize.execute(originalQuery.analyze), 
correctAnswer.analyze)
+  }
+
+  test("Merging subqueries with filters") {
+    val subquery1 = ScalarSubquery(testRelation.where('a > 1).select('a))
+    // Despite having an extra Project node, `subquery2` is mergeable with 
`subquery1`
+    val subquery2 = ScalarSubquery(testRelation.where('a > 
1).select('b.as("b_1")).select('b_1))
+    // Despite lacking a Project node, `subquery3` is mergeable with the 
result of merging
+    // `subquery1` and `subquery2`
+    val subquery3 = 
ScalarSubquery(testRelation.select('a.as("a_2")).where('a_2 > 1).select('a_2))
+    val subquery4 = ScalarSubquery(
+      testRelation.select('a.as("a_2"), 'b).where('a_2 > 
1).select('b.as("b_2")))
+    val originalQuery = testRelation
+      .select(
+        subquery1,
+        subquery2,
+        subquery3,
+        subquery4)
+
+    val mergedSubquery = testRelation
+      .select('a, 'b, 'c)
+      .where('a > 1)
+      .select('a, 'b, 'c)
+      .select('a, 'b)
+      .select(CreateNamedStruct(Seq(
+        Literal("a"), 'a,
+        Literal("b_1"), 'b
+      )).as("mergedValue"))
+    val analyzedMergedSubquery = mergedSubquery.analyze
+    val correctAnswer = WithCTE(
+      testRelation
+        .select(
+          extractorExpression(0, analyzedMergedSubquery.output, 0),
+          extractorExpression(0, analyzedMergedSubquery.output, 1),
+          extractorExpression(0, analyzedMergedSubquery.output, 0),
+          extractorExpression(0, analyzedMergedSubquery.output, 1)),
+      Seq(CTERelationDef(analyzedMergedSubquery, 0)))
+
+    comparePlans(Optimize.execute(originalQuery.analyze), 
correctAnswer.analyze)
+  }
+
+  test("Merging subqueries with aggregate filters") {
+    val subquery1 = ScalarSubquery(
+      testRelation.having('b)(max('a).as("max_a"))(max('a) > 1))
+    val subquery2 = ScalarSubquery(
+      testRelation.having('b)(sum('a).as("sum_a"))(max('a) > 1))
+    val originalQuery = testRelation.select(
+      subquery1,
+      subquery2)
+
+    val mergedSubquery = testRelation
+      .having('b)(
+        max('a).as("max_a"),
+        sum('a).as("sum_a"))('max_a > 1)
+      .select(
+        'max_a,
+        'sum_a)
+      .select(CreateNamedStruct(Seq(
+        Literal("max_a"), 'max_a,
+        Literal("sum_a"), 'sum_a
+      )).as("mergedValue"))
+    val analyzedMergedSubquery = mergedSubquery.analyze
+    val correctAnswer = WithCTE(
+      testRelation
+        .select(
+          extractorExpression(0, analyzedMergedSubquery.output, 0),
+          extractorExpression(0, analyzedMergedSubquery.output, 1)),
+      Seq(CTERelationDef(analyzedMergedSubquery, 0)))
+
+    comparePlans(Optimize.execute(originalQuery.analyze), 
correctAnswer.analyze)
+  }
+
+  test("Merging subqueries with joins") {
+    val subquery1 = ScalarSubquery(testRelation.as("t1")
+      .join(
+        testRelation.as("t2"),
+        Inner,
+        Some($"t1.b" === $"t2.b"))
+      .select($"t1.a").analyze)
+    val subquery2 = ScalarSubquery(testRelation.as("t1")
+      .select('a.as("a_1"), 'b.as("b_1"), 'c.as("c_1"))
+      .join(
+        testRelation.as("t2").select('a.as("a_2"), 'b.as("b_2"), 'c.as("c_2")),
+        Inner,
+        Some('b_1 === 'b_2))
+      .select('c_2).analyze)
+    val originalQuery = testRelation.select(
+      subquery1,
+      subquery2)
+
+    val mergedSubquery = testRelation.as("t1")
+      .select('a, 'b, 'c)
+      .join(
+        testRelation.as("t2").select('a, 'b, 'c),
+        Inner,
+        Some($"t1.b" === $"t2.b"))
+      .select($"t1.a", $"t2.c")
+      .select(CreateNamedStruct(Seq(
+        Literal("a"), 'a,
+        Literal("c_2"), 'c
+      )).as("mergedValue"))
+    val analyzedMergedSubquery = mergedSubquery.analyze
+    val correctAnswer = WithCTE(
+      testRelation
+        .select(
+          extractorExpression(0, analyzedMergedSubquery.output, 0),
+          extractorExpression(0, analyzedMergedSubquery.output, 1)),
+      Seq(CTERelationDef(analyzedMergedSubquery, 0)))
+
+    comparePlans(Optimize.execute(originalQuery.analyze), 
correctAnswer.analyze)
+  }
+
+  test("Merging subqueries with joins but different join types") {

Review Comment:
   Nit: "Merging" -> "Do not merge"?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala:
##########
@@ -687,7 +687,8 @@ case class CTERelationRef(
     cteId: Long,
     _resolved: Boolean,
     override val output: Seq[Attribute],
-    statsOpt: Option[Statistics] = None) extends LeafNode with 
MultiInstanceRelation {
+    statsOpt: Option[Statistics] = None,
+    subquery: Boolean = false) extends LeafNode with MultiInstanceRelation {

Review Comment:
   Update case class comment for the new member?
   "Nit": subquery => "mergedScalarSubquery"



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala:
##########
@@ -0,0 +1,370 @@
+/*
+ * 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
+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, CTERelationDef, 
CTERelationRef, Filter, Join, LogicalPlan, Project, Subquery, WithCTE}
+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 temporal
+ *   [[ScalarSubqueryReference]] reference pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a result 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 contains the list of 
attributes form the scalar
+ *   return value of a merged subquery.
+ * - A second traversal checks if there are merged subqueries in the cache and 
builds a `WithCTE`
+ *   node from these queries. The `CTERelationDef` nodes contain the merged 
subquery in the
+ *   following form:
+ *   `Project(Seq(CreateNamedStruct(name1, attribute1, ...) AS mergedValue), 
mergedSubqueryPlan)`
+ *   and the definitions are flagged that they host a subquery, that can 
return maximum one row.
+ *   During the second traversal [[ScalarSubqueryReference]] expressions that 
pont to a merged
+ *   subquery is either transformed to a 
`GetStructField(ScalarSubquery(CTERelationRef(...)))`
+ *   expression or restored to the original [[ScalarSubquery]].
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t),
+ *   (SELECT sum(b) FROM t)
+ *
+ * is optimized from:
+ *
+ * == Optimized Logical Plan ==
+ * Project [scalar-subquery#242 [] AS scalarsubquery()#253,
+ *          scalar-subquery#243 [] AS scalarsubquery()#254L]
+ * :  :- Aggregate [avg(a#244) AS avg(a)#247]
+ * :  :  +- Project [a#244]
+ * :  :     +- Relation default.t[a#244,b#245] parquet
+ * :  +- Aggregate [sum(a#251) AS sum(a)#250L]
+ * :     +- Project [a#251]
+ * :        +- Relation default.t[a#251,b#252] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * WithCTE
+ * :- CTERelationDef 0
+ * :  +- Project [named_struct(avg(a), avg(a)#247, sum(a), sum(a)#250L) AS 
mergedValue#260]
+ * :     +- Aggregate [avg(a#244) AS avg(a)#247, sum(a#244) AS sum(a)#250L]
+ * :        +- Project [a#244]
+ * :           +- Relation default.t[a#244,b#245] parquet
+ * +- Project [scalar-subquery#242 [].avg(a) AS scalarsubquery()#253,
+ *             scalar-subquery#243 [].sum(a) AS scalarsubquery()#254L]
+ *    :  :- CTERelationRef 0, true, [mergedValue#260], true
+ *    :  +- CTERelationRef 0, true, [mergedValue#260], true
+ *    +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    plan match {
+      case s: Subquery => s.copy(child = 
extractCommonScalarSubqueries(s.child))

Review Comment:
   Do we have a test case for merged CTE inside a subquery?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala:
##########
@@ -0,0 +1,370 @@
+/*
+ * 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
+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, CTERelationDef, 
CTERelationRef, Filter, Join, LogicalPlan, Project, Subquery, WithCTE}
+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 temporal
+ *   [[ScalarSubqueryReference]] reference pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a result 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 contains the list of 
attributes form the scalar
+ *   return value of a merged subquery.
+ * - A second traversal checks if there are merged subqueries in the cache and 
builds a `WithCTE`
+ *   node from these queries. The `CTERelationDef` nodes contain the merged 
subquery in the
+ *   following form:
+ *   `Project(Seq(CreateNamedStruct(name1, attribute1, ...) AS mergedValue), 
mergedSubqueryPlan)`
+ *   and the definitions are flagged that they host a subquery, that can 
return maximum one row.
+ *   During the second traversal [[ScalarSubqueryReference]] expressions that 
pont to a merged
+ *   subquery is either transformed to a 
`GetStructField(ScalarSubquery(CTERelationRef(...)))`
+ *   expression or restored to the original [[ScalarSubquery]].
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t),
+ *   (SELECT sum(b) FROM t)
+ *
+ * is optimized from:
+ *
+ * == Optimized Logical Plan ==
+ * Project [scalar-subquery#242 [] AS scalarsubquery()#253,
+ *          scalar-subquery#243 [] AS scalarsubquery()#254L]
+ * :  :- Aggregate [avg(a#244) AS avg(a)#247]
+ * :  :  +- Project [a#244]
+ * :  :     +- Relation default.t[a#244,b#245] parquet
+ * :  +- Aggregate [sum(a#251) AS sum(a)#250L]
+ * :     +- Project [a#251]
+ * :        +- Relation default.t[a#251,b#252] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * WithCTE
+ * :- CTERelationDef 0
+ * :  +- Project [named_struct(avg(a), avg(a)#247, sum(a), sum(a)#250L) AS 
mergedValue#260]
+ * :     +- Aggregate [avg(a#244) AS avg(a)#247, sum(a#244) AS sum(a)#250L]
+ * :        +- Project [a#244]
+ * :           +- Relation default.t[a#244,b#245] parquet
+ * +- Project [scalar-subquery#242 [].avg(a) AS scalarsubquery()#253,
+ *             scalar-subquery#243 [].sum(a) AS scalarsubquery()#254L]
+ *    :  :- CTERelationRef 0, true, [mergedValue#260], true
+ *    :  +- CTERelationRef 0, true, [mergedValue#260], true
+ *    +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    plan match {
+      case s: Subquery => s.copy(child = 
extractCommonScalarSubqueries(s.child))
+      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)

Review Comment:
   Should we use ordinal instead of names? 
   What would happen if the select columns have identical column names in two 
subqueries?



##########
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueriesSuite.scala:
##########
@@ -0,0 +1,355 @@
+/*
+ * 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 org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.expressions.{Attribute, 
CreateNamedStruct, GetStructField, Literal, ScalarSubquery}
+import org.apache.spark.sql.catalyst.expressions.aggregate.{CollectList, 
CollectSet}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules._
+
+class MergeScalarSubqueriesSuite extends PlanTest {
+
+  override def beforeEach(): Unit = {
+    CTERelationDef.curId.set(0)
+  }
+
+  private object Optimize extends RuleExecutor[LogicalPlan] {
+    val batches = Batch("MergeScalarSubqueries", Once, MergeScalarSubqueries) 
:: Nil
+  }
+
+  val testRelation = LocalRelation('a.int, 'b.int, 'c.string)
+
+  private def extractorExpression(cteIndex: Int, output: Seq[Attribute], 
fieldIndex: Int) = {
+    GetStructField(
+      ScalarSubquery(CTERelationRef(cteIndex, _resolved = true, output, 
subquery = true)),
+      fieldIndex).as("scalarsubquery()")
+  }
+
+  test("Merging subqueries with projects") {
+    val subquery1 = ScalarSubquery(testRelation.select(('a + 1).as("a_plus1")))
+    val subquery2 = ScalarSubquery(testRelation.select(('a + 2).as("a_plus2")))
+    val subquery3 = ScalarSubquery(testRelation.select('b))
+    val subquery4 = ScalarSubquery(testRelation.select(('a + 
1).as("a_plus1_2")))
+    val subquery5 = ScalarSubquery(testRelation.select(('a + 
2).as("a_plus2_2")))
+    val subquery6 = ScalarSubquery(testRelation.select('b.as("b_2")))
+    val originalQuery = testRelation
+      .select(
+        subquery1,
+        subquery2,
+        subquery3,
+        subquery4,
+        subquery5,
+        subquery6)
+
+    val mergedSubquery = testRelation
+      .select(
+        ('a + 1).as("a_plus1"),
+        ('a + 2).as("a_plus2"),
+        'b)
+      .select(
+        CreateNamedStruct(Seq(
+          Literal("a_plus1"), 'a_plus1,
+          Literal("a_plus2"), 'a_plus2,
+          Literal("b"), 'b
+        )).as("mergedValue"))
+    val analyzedMergedSubquery = mergedSubquery.analyze
+    val correctAnswer = WithCTE(
+      testRelation
+        .select(
+          extractorExpression(0, analyzedMergedSubquery.output, 0),
+          extractorExpression(0, analyzedMergedSubquery.output, 1),
+          extractorExpression(0, analyzedMergedSubquery.output, 2),
+          extractorExpression(0, analyzedMergedSubquery.output, 0),
+          extractorExpression(0, analyzedMergedSubquery.output, 1),
+          extractorExpression(0, analyzedMergedSubquery.output, 2)),
+      Seq(CTERelationDef(analyzedMergedSubquery, 0)))
+
+    comparePlans(Optimize.execute(originalQuery.analyze), 
correctAnswer.analyze)
+  }
+
+  test("Merging subqueries with aggregates") {
+    val subquery1 = 
ScalarSubquery(testRelation.groupBy('b)(max('a).as("max_a")))
+    val subquery2 = 
ScalarSubquery(testRelation.groupBy('b)(sum('a).as("sum_a")))
+    val subquery3 = ScalarSubquery(testRelation.groupBy('b)('b))
+    val subquery4 = 
ScalarSubquery(testRelation.groupBy('b)(max('a).as("max_a_2")))
+    val subquery5 = 
ScalarSubquery(testRelation.groupBy('b)(sum('a).as("sum_a_2")))
+    val subquery6 = ScalarSubquery(testRelation.groupBy('b)('b.as("b_2")))
+    val originalQuery = testRelation
+      .select(
+        subquery1,
+        subquery2,
+        subquery3,
+        subquery4,
+        subquery5,
+        subquery6)
+
+    val mergedSubquery = testRelation
+      .groupBy('b)(
+        max('a).as("max_a"),
+        sum('a).as("sum_a"),
+        'b)
+      .select(CreateNamedStruct(Seq(
+        Literal("max_a"), 'max_a,
+        Literal("sum_a"), 'sum_a,
+        Literal("b"), 'b
+      )).as("mergedValue"))
+    val analyzedMergedSubquery = mergedSubquery.analyze
+    val correctAnswer = WithCTE(
+      testRelation
+        .select(
+          extractorExpression(0, analyzedMergedSubquery.output, 0),
+          extractorExpression(0, analyzedMergedSubquery.output, 1),
+          extractorExpression(0, analyzedMergedSubquery.output, 2),
+          extractorExpression(0, analyzedMergedSubquery.output, 0),
+          extractorExpression(0, analyzedMergedSubquery.output, 1),
+          extractorExpression(0, analyzedMergedSubquery.output, 2)),
+      Seq(CTERelationDef(analyzedMergedSubquery, 0)))
+
+    comparePlans(Optimize.execute(originalQuery.analyze), 
correctAnswer.analyze)
+  }
+
+  test("Merging subqueries with filters") {
+    val subquery1 = ScalarSubquery(testRelation.where('a > 1).select('a))
+    // Despite having an extra Project node, `subquery2` is mergeable with 
`subquery1`
+    val subquery2 = ScalarSubquery(testRelation.where('a > 
1).select('b.as("b_1")).select('b_1))
+    // Despite lacking a Project node, `subquery3` is mergeable with the 
result of merging
+    // `subquery1` and `subquery2`
+    val subquery3 = 
ScalarSubquery(testRelation.select('a.as("a_2")).where('a_2 > 1).select('a_2))
+    val subquery4 = ScalarSubquery(
+      testRelation.select('a.as("a_2"), 'b).where('a_2 > 
1).select('b.as("b_2")))
+    val originalQuery = testRelation
+      .select(
+        subquery1,
+        subquery2,
+        subquery3,
+        subquery4)
+
+    val mergedSubquery = testRelation
+      .select('a, 'b, 'c)
+      .where('a > 1)
+      .select('a, 'b, 'c)
+      .select('a, 'b)
+      .select(CreateNamedStruct(Seq(
+        Literal("a"), 'a,
+        Literal("b_1"), 'b
+      )).as("mergedValue"))
+    val analyzedMergedSubquery = mergedSubquery.analyze
+    val correctAnswer = WithCTE(
+      testRelation
+        .select(
+          extractorExpression(0, analyzedMergedSubquery.output, 0),
+          extractorExpression(0, analyzedMergedSubquery.output, 1),
+          extractorExpression(0, analyzedMergedSubquery.output, 0),
+          extractorExpression(0, analyzedMergedSubquery.output, 1)),
+      Seq(CTERelationDef(analyzedMergedSubquery, 0)))
+
+    comparePlans(Optimize.execute(originalQuery.analyze), 
correctAnswer.analyze)
+  }
+
+  test("Merging subqueries with aggregate filters") {
+    val subquery1 = ScalarSubquery(
+      testRelation.having('b)(max('a).as("max_a"))(max('a) > 1))
+    val subquery2 = ScalarSubquery(
+      testRelation.having('b)(sum('a).as("sum_a"))(max('a) > 1))
+    val originalQuery = testRelation.select(
+      subquery1,
+      subquery2)
+
+    val mergedSubquery = testRelation
+      .having('b)(
+        max('a).as("max_a"),
+        sum('a).as("sum_a"))('max_a > 1)
+      .select(
+        'max_a,
+        'sum_a)
+      .select(CreateNamedStruct(Seq(
+        Literal("max_a"), 'max_a,
+        Literal("sum_a"), 'sum_a
+      )).as("mergedValue"))
+    val analyzedMergedSubquery = mergedSubquery.analyze
+    val correctAnswer = WithCTE(
+      testRelation
+        .select(
+          extractorExpression(0, analyzedMergedSubquery.output, 0),
+          extractorExpression(0, analyzedMergedSubquery.output, 1)),
+      Seq(CTERelationDef(analyzedMergedSubquery, 0)))
+
+    comparePlans(Optimize.execute(originalQuery.analyze), 
correctAnswer.analyze)
+  }
+
+  test("Merging subqueries with joins") {
+    val subquery1 = ScalarSubquery(testRelation.as("t1")
+      .join(
+        testRelation.as("t2"),
+        Inner,
+        Some($"t1.b" === $"t2.b"))
+      .select($"t1.a").analyze)
+    val subquery2 = ScalarSubquery(testRelation.as("t1")
+      .select('a.as("a_1"), 'b.as("b_1"), 'c.as("c_1"))
+      .join(
+        testRelation.as("t2").select('a.as("a_2"), 'b.as("b_2"), 'c.as("c_2")),
+        Inner,
+        Some('b_1 === 'b_2))
+      .select('c_2).analyze)
+    val originalQuery = testRelation.select(
+      subquery1,
+      subquery2)
+
+    val mergedSubquery = testRelation.as("t1")
+      .select('a, 'b, 'c)
+      .join(
+        testRelation.as("t2").select('a, 'b, 'c),
+        Inner,
+        Some($"t1.b" === $"t2.b"))
+      .select($"t1.a", $"t2.c")
+      .select(CreateNamedStruct(Seq(
+        Literal("a"), 'a,
+        Literal("c_2"), 'c
+      )).as("mergedValue"))
+    val analyzedMergedSubquery = mergedSubquery.analyze
+    val correctAnswer = WithCTE(
+      testRelation
+        .select(
+          extractorExpression(0, analyzedMergedSubquery.output, 0),
+          extractorExpression(0, analyzedMergedSubquery.output, 1)),
+      Seq(CTERelationDef(analyzedMergedSubquery, 0)))
+
+    comparePlans(Optimize.execute(originalQuery.analyze), 
correctAnswer.analyze)
+  }
+
+  test("Merging subqueries with joins but different join types") {
+    val subquery1 = ScalarSubquery(testRelation.as("t1")
+      .join(
+        testRelation.as("t2"),
+        Inner,
+        Some($"t1.b" === $"t2.b"))
+      .select($"t1.a"))
+    val subquery2 = ScalarSubquery(testRelation.as("t1")
+      .select('a.as("a_1"), 'b.as("b_1"), 'c.as("c_1"))
+      .join(
+        testRelation.as("t2").select('a.as("a_2"), 'b.as("b_2"), 'c.as("c_2")),
+        LeftOuter,
+        Some('b_1 === 'b_2))
+      .select('c_2))
+    val originalQuery = testRelation.select(
+      subquery1,
+      subquery2)
+
+    comparePlans(Optimize.execute(originalQuery.analyze), 
originalQuery.analyze)
+  }
+
+  test("Merging subqueries with nondeterministic elements") {

Review Comment:
   Nit: "Merging" -> "Do not merge"



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala:
##########
@@ -615,4 +603,23 @@ object QueryPlan extends PredicateHelper {
       case e: AnalysisException => append(e.toString)
     }
   }
+
+  /**
+   * Extracts expressions from an object.
+   */
+  def extractExpressions(o: Any): Seq[Expression] = {

Review Comment:
   why do we need this new public function?



##########
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueriesSuite.scala:
##########
@@ -0,0 +1,355 @@
+/*
+ * 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 org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.expressions.{Attribute, 
CreateNamedStruct, GetStructField, Literal, ScalarSubquery}
+import org.apache.spark.sql.catalyst.expressions.aggregate.{CollectList, 
CollectSet}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules._
+
+class MergeScalarSubqueriesSuite extends PlanTest {
+
+  override def beforeEach(): Unit = {
+    CTERelationDef.curId.set(0)
+  }
+
+  private object Optimize extends RuleExecutor[LogicalPlan] {
+    val batches = Batch("MergeScalarSubqueries", Once, MergeScalarSubqueries) 
:: Nil
+  }
+
+  val testRelation = LocalRelation('a.int, 'b.int, 'c.string)
+
+  private def extractorExpression(cteIndex: Int, output: Seq[Attribute], 
fieldIndex: Int) = {
+    GetStructField(
+      ScalarSubquery(CTERelationRef(cteIndex, _resolved = true, output, 
subquery = true)),
+      fieldIndex).as("scalarsubquery()")
+  }
+
+  test("Merging subqueries with projects") {
+    val subquery1 = ScalarSubquery(testRelation.select(('a + 1).as("a_plus1")))
+    val subquery2 = ScalarSubquery(testRelation.select(('a + 2).as("a_plus2")))
+    val subquery3 = ScalarSubquery(testRelation.select('b))
+    val subquery4 = ScalarSubquery(testRelation.select(('a + 
1).as("a_plus1_2")))
+    val subquery5 = ScalarSubquery(testRelation.select(('a + 
2).as("a_plus2_2")))
+    val subquery6 = ScalarSubquery(testRelation.select('b.as("b_2")))
+    val originalQuery = testRelation
+      .select(
+        subquery1,
+        subquery2,
+        subquery3,
+        subquery4,
+        subquery5,
+        subquery6)
+
+    val mergedSubquery = testRelation
+      .select(
+        ('a + 1).as("a_plus1"),
+        ('a + 2).as("a_plus2"),
+        'b)
+      .select(
+        CreateNamedStruct(Seq(
+          Literal("a_plus1"), 'a_plus1,
+          Literal("a_plus2"), 'a_plus2,
+          Literal("b"), 'b
+        )).as("mergedValue"))
+    val analyzedMergedSubquery = mergedSubquery.analyze
+    val correctAnswer = WithCTE(
+      testRelation
+        .select(
+          extractorExpression(0, analyzedMergedSubquery.output, 0),
+          extractorExpression(0, analyzedMergedSubquery.output, 1),
+          extractorExpression(0, analyzedMergedSubquery.output, 2),
+          extractorExpression(0, analyzedMergedSubquery.output, 0),
+          extractorExpression(0, analyzedMergedSubquery.output, 1),
+          extractorExpression(0, analyzedMergedSubquery.output, 2)),
+      Seq(CTERelationDef(analyzedMergedSubquery, 0)))
+
+    comparePlans(Optimize.execute(originalQuery.analyze), 
correctAnswer.analyze)
+  }
+
+  test("Merging subqueries with aggregates") {
+    val subquery1 = 
ScalarSubquery(testRelation.groupBy('b)(max('a).as("max_a")))
+    val subquery2 = 
ScalarSubquery(testRelation.groupBy('b)(sum('a).as("sum_a")))
+    val subquery3 = ScalarSubquery(testRelation.groupBy('b)('b))
+    val subquery4 = 
ScalarSubquery(testRelation.groupBy('b)(max('a).as("max_a_2")))
+    val subquery5 = 
ScalarSubquery(testRelation.groupBy('b)(sum('a).as("sum_a_2")))
+    val subquery6 = ScalarSubquery(testRelation.groupBy('b)('b.as("b_2")))
+    val originalQuery = testRelation
+      .select(
+        subquery1,
+        subquery2,
+        subquery3,
+        subquery4,
+        subquery5,
+        subquery6)
+
+    val mergedSubquery = testRelation
+      .groupBy('b)(
+        max('a).as("max_a"),
+        sum('a).as("sum_a"),
+        'b)
+      .select(CreateNamedStruct(Seq(
+        Literal("max_a"), 'max_a,
+        Literal("sum_a"), 'sum_a,
+        Literal("b"), 'b
+      )).as("mergedValue"))
+    val analyzedMergedSubquery = mergedSubquery.analyze
+    val correctAnswer = WithCTE(
+      testRelation
+        .select(
+          extractorExpression(0, analyzedMergedSubquery.output, 0),
+          extractorExpression(0, analyzedMergedSubquery.output, 1),
+          extractorExpression(0, analyzedMergedSubquery.output, 2),
+          extractorExpression(0, analyzedMergedSubquery.output, 0),
+          extractorExpression(0, analyzedMergedSubquery.output, 1),
+          extractorExpression(0, analyzedMergedSubquery.output, 2)),
+      Seq(CTERelationDef(analyzedMergedSubquery, 0)))
+
+    comparePlans(Optimize.execute(originalQuery.analyze), 
correctAnswer.analyze)
+  }
+
+  test("Merging subqueries with filters") {
+    val subquery1 = ScalarSubquery(testRelation.where('a > 1).select('a))
+    // Despite having an extra Project node, `subquery2` is mergeable with 
`subquery1`
+    val subquery2 = ScalarSubquery(testRelation.where('a > 
1).select('b.as("b_1")).select('b_1))
+    // Despite lacking a Project node, `subquery3` is mergeable with the 
result of merging
+    // `subquery1` and `subquery2`
+    val subquery3 = 
ScalarSubquery(testRelation.select('a.as("a_2")).where('a_2 > 1).select('a_2))
+    val subquery4 = ScalarSubquery(
+      testRelation.select('a.as("a_2"), 'b).where('a_2 > 
1).select('b.as("b_2")))
+    val originalQuery = testRelation
+      .select(
+        subquery1,
+        subquery2,
+        subquery3,
+        subquery4)
+
+    val mergedSubquery = testRelation
+      .select('a, 'b, 'c)
+      .where('a > 1)
+      .select('a, 'b, 'c)
+      .select('a, 'b)
+      .select(CreateNamedStruct(Seq(
+        Literal("a"), 'a,
+        Literal("b_1"), 'b
+      )).as("mergedValue"))
+    val analyzedMergedSubquery = mergedSubquery.analyze
+    val correctAnswer = WithCTE(
+      testRelation
+        .select(
+          extractorExpression(0, analyzedMergedSubquery.output, 0),
+          extractorExpression(0, analyzedMergedSubquery.output, 1),
+          extractorExpression(0, analyzedMergedSubquery.output, 0),
+          extractorExpression(0, analyzedMergedSubquery.output, 1)),
+      Seq(CTERelationDef(analyzedMergedSubquery, 0)))
+
+    comparePlans(Optimize.execute(originalQuery.analyze), 
correctAnswer.analyze)
+  }
+
+  test("Merging subqueries with aggregate filters") {
+    val subquery1 = ScalarSubquery(
+      testRelation.having('b)(max('a).as("max_a"))(max('a) > 1))
+    val subquery2 = ScalarSubquery(
+      testRelation.having('b)(sum('a).as("sum_a"))(max('a) > 1))
+    val originalQuery = testRelation.select(
+      subquery1,
+      subquery2)
+
+    val mergedSubquery = testRelation
+      .having('b)(
+        max('a).as("max_a"),
+        sum('a).as("sum_a"))('max_a > 1)
+      .select(
+        'max_a,
+        'sum_a)
+      .select(CreateNamedStruct(Seq(
+        Literal("max_a"), 'max_a,
+        Literal("sum_a"), 'sum_a
+      )).as("mergedValue"))
+    val analyzedMergedSubquery = mergedSubquery.analyze
+    val correctAnswer = WithCTE(
+      testRelation
+        .select(
+          extractorExpression(0, analyzedMergedSubquery.output, 0),
+          extractorExpression(0, analyzedMergedSubquery.output, 1)),
+      Seq(CTERelationDef(analyzedMergedSubquery, 0)))
+
+    comparePlans(Optimize.execute(originalQuery.analyze), 
correctAnswer.analyze)
+  }
+
+  test("Merging subqueries with joins") {
+    val subquery1 = ScalarSubquery(testRelation.as("t1")
+      .join(
+        testRelation.as("t2"),
+        Inner,
+        Some($"t1.b" === $"t2.b"))
+      .select($"t1.a").analyze)
+    val subquery2 = ScalarSubquery(testRelation.as("t1")
+      .select('a.as("a_1"), 'b.as("b_1"), 'c.as("c_1"))
+      .join(
+        testRelation.as("t2").select('a.as("a_2"), 'b.as("b_2"), 'c.as("c_2")),
+        Inner,
+        Some('b_1 === 'b_2))
+      .select('c_2).analyze)
+    val originalQuery = testRelation.select(
+      subquery1,
+      subquery2)
+
+    val mergedSubquery = testRelation.as("t1")
+      .select('a, 'b, 'c)
+      .join(
+        testRelation.as("t2").select('a, 'b, 'c),
+        Inner,
+        Some($"t1.b" === $"t2.b"))
+      .select($"t1.a", $"t2.c")
+      .select(CreateNamedStruct(Seq(
+        Literal("a"), 'a,
+        Literal("c_2"), 'c
+      )).as("mergedValue"))
+    val analyzedMergedSubquery = mergedSubquery.analyze
+    val correctAnswer = WithCTE(
+      testRelation
+        .select(
+          extractorExpression(0, analyzedMergedSubquery.output, 0),
+          extractorExpression(0, analyzedMergedSubquery.output, 1)),
+      Seq(CTERelationDef(analyzedMergedSubquery, 0)))
+
+    comparePlans(Optimize.execute(originalQuery.analyze), 
correctAnswer.analyze)
+  }
+

Review Comment:
   Add a test case in which scalar subqueries to be merged appear in different 
LogicalPlan nodes? 
   E.g., filter, project, aggregate.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala:
##########
@@ -0,0 +1,370 @@
+/*
+ * 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
+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, CTERelationDef, 
CTERelationRef, Filter, Join, LogicalPlan, Project, Subquery, WithCTE}
+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 temporal
+ *   [[ScalarSubqueryReference]] reference pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a result 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 contains the list of 
attributes form the scalar
+ *   return value of a merged subquery.
+ * - A second traversal checks if there are merged subqueries in the cache and 
builds a `WithCTE`
+ *   node from these queries. The `CTERelationDef` nodes contain the merged 
subquery in the
+ *   following form:
+ *   `Project(Seq(CreateNamedStruct(name1, attribute1, ...) AS mergedValue), 
mergedSubqueryPlan)`
+ *   and the definitions are flagged that they host a subquery, that can 
return maximum one row.
+ *   During the second traversal [[ScalarSubqueryReference]] expressions that 
pont to a merged
+ *   subquery is either transformed to a 
`GetStructField(ScalarSubquery(CTERelationRef(...)))`
+ *   expression or restored to the original [[ScalarSubquery]].
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t),
+ *   (SELECT sum(b) FROM t)
+ *
+ * is optimized from:
+ *
+ * == Optimized Logical Plan ==
+ * Project [scalar-subquery#242 [] AS scalarsubquery()#253,
+ *          scalar-subquery#243 [] AS scalarsubquery()#254L]
+ * :  :- Aggregate [avg(a#244) AS avg(a)#247]
+ * :  :  +- Project [a#244]
+ * :  :     +- Relation default.t[a#244,b#245] parquet
+ * :  +- Aggregate [sum(a#251) AS sum(a)#250L]
+ * :     +- Project [a#251]
+ * :        +- Relation default.t[a#251,b#252] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * WithCTE
+ * :- CTERelationDef 0
+ * :  +- Project [named_struct(avg(a), avg(a)#247, sum(a), sum(a)#250L) AS 
mergedValue#260]
+ * :     +- Aggregate [avg(a#244) AS avg(a)#247, sum(a#244) AS sum(a)#250L]
+ * :        +- Project [a#244]
+ * :           +- Relation default.t[a#244,b#245] parquet
+ * +- Project [scalar-subquery#242 [].avg(a) AS scalarsubquery()#253,
+ *             scalar-subquery#243 [].sum(a) AS scalarsubquery()#254L]
+ *    :  :- CTERelationRef 0, true, [mergedValue#260], true
+ *    :  +- CTERelationRef 0, true, [mergedValue#260], true
+ *    +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    plan match {
+      case s: Subquery => s.copy(child = 
extractCommonScalarSubqueries(s.child))
+      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, subqueryCTEs) = removeReferences(insertReferences(plan, 
cache), cache)
+    if (subqueryCTEs.nonEmpty) {
+      WithCTE(newPlan, subqueryCTEs)
+    } 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".
+  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. This implementation:
+        // - Supports only a whitelist of nodes (see `supportedMerge()`).
+        // - All children need to be direct fields of the tree node case 
class. Tree nodes where a
+        //   child is wrapped into a `Seq` or `Option` (e.g. `Union`) is not 
supported.
+        // - Expressions can be wrapped.
+        // - Children are tried to 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
+          }
+
+        // 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)

Review Comment:
   Could there be name conflict here? E.g.,
   
   ```
   SELECT
       (SELECT avg(a) AS x FROM t),
       (SELECT sum(b) AS x FROM t)
   ```
   
   Add a test case for it?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala:
##########
@@ -0,0 +1,401 @@
+/*
+ * 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, 
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]] rules do the physical 
planning of scalar
+ *   subqueries including the ones under [[CommonScalarSubqueriesExec]] node 
and replace
+ *   each [[ScalarSubqueryReference]] to their referenced physical plan in
+ *   `GetStructField(ScalarSubquery(merged plan with CreateNamedStruct() 
header))` form. Finally,
+ *   the [[CommonScalarSubqueriesExec]] node is removed from the physical plan.
+ * - [[ReuseSubquery]] rule makes sure that merged subqueries are computed 
only 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 [scalar-subquery#231 [].avg(a) AS scalarsubquery()#241,
+ *          scalar-subquery#232 [].sum(b) AS scalarsubquery()#242L]
+ * :  :- 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 [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
+ * +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    if (conf.scalarSubqueryMergeEabled && conf.subqueryReuseEnabled) {

Review Comment:
   Do you still need conf.subqueryReuseEnabled?



##########
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueriesSuite.scala:
##########
@@ -0,0 +1,355 @@
+/*
+ * 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 org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.expressions.{Attribute, 
CreateNamedStruct, GetStructField, Literal, ScalarSubquery}
+import org.apache.spark.sql.catalyst.expressions.aggregate.{CollectList, 
CollectSet}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules._
+
+class MergeScalarSubqueriesSuite extends PlanTest {
+
+  override def beforeEach(): Unit = {
+    CTERelationDef.curId.set(0)
+  }
+
+  private object Optimize extends RuleExecutor[LogicalPlan] {
+    val batches = Batch("MergeScalarSubqueries", Once, MergeScalarSubqueries) 
:: Nil
+  }
+
+  val testRelation = LocalRelation('a.int, 'b.int, 'c.string)
+
+  private def extractorExpression(cteIndex: Int, output: Seq[Attribute], 
fieldIndex: Int) = {
+    GetStructField(
+      ScalarSubquery(CTERelationRef(cteIndex, _resolved = true, output, 
subquery = true)),
+      fieldIndex).as("scalarsubquery()")
+  }
+
+  test("Merging subqueries with projects") {
+    val subquery1 = ScalarSubquery(testRelation.select(('a + 1).as("a_plus1")))
+    val subquery2 = ScalarSubquery(testRelation.select(('a + 2).as("a_plus2")))
+    val subquery3 = ScalarSubquery(testRelation.select('b))
+    val subquery4 = ScalarSubquery(testRelation.select(('a + 
1).as("a_plus1_2")))
+    val subquery5 = ScalarSubquery(testRelation.select(('a + 
2).as("a_plus2_2")))
+    val subquery6 = ScalarSubquery(testRelation.select('b.as("b_2")))
+    val originalQuery = testRelation
+      .select(
+        subquery1,
+        subquery2,
+        subquery3,
+        subquery4,
+        subquery5,
+        subquery6)
+
+    val mergedSubquery = testRelation
+      .select(
+        ('a + 1).as("a_plus1"),
+        ('a + 2).as("a_plus2"),
+        'b)
+      .select(
+        CreateNamedStruct(Seq(
+          Literal("a_plus1"), 'a_plus1,
+          Literal("a_plus2"), 'a_plus2,
+          Literal("b"), 'b
+        )).as("mergedValue"))
+    val analyzedMergedSubquery = mergedSubquery.analyze
+    val correctAnswer = WithCTE(
+      testRelation
+        .select(
+          extractorExpression(0, analyzedMergedSubquery.output, 0),
+          extractorExpression(0, analyzedMergedSubquery.output, 1),
+          extractorExpression(0, analyzedMergedSubquery.output, 2),
+          extractorExpression(0, analyzedMergedSubquery.output, 0),
+          extractorExpression(0, analyzedMergedSubquery.output, 1),
+          extractorExpression(0, analyzedMergedSubquery.output, 2)),
+      Seq(CTERelationDef(analyzedMergedSubquery, 0)))
+
+    comparePlans(Optimize.execute(originalQuery.analyze), 
correctAnswer.analyze)
+  }
+
+  test("Merging subqueries with aggregates") {
+    val subquery1 = 
ScalarSubquery(testRelation.groupBy('b)(max('a).as("max_a")))
+    val subquery2 = 
ScalarSubquery(testRelation.groupBy('b)(sum('a).as("sum_a")))
+    val subquery3 = ScalarSubquery(testRelation.groupBy('b)('b))
+    val subquery4 = 
ScalarSubquery(testRelation.groupBy('b)(max('a).as("max_a_2")))
+    val subquery5 = 
ScalarSubquery(testRelation.groupBy('b)(sum('a).as("sum_a_2")))
+    val subquery6 = ScalarSubquery(testRelation.groupBy('b)('b.as("b_2")))
+    val originalQuery = testRelation
+      .select(
+        subquery1,
+        subquery2,
+        subquery3,
+        subquery4,
+        subquery5,
+        subquery6)
+
+    val mergedSubquery = testRelation
+      .groupBy('b)(
+        max('a).as("max_a"),
+        sum('a).as("sum_a"),
+        'b)
+      .select(CreateNamedStruct(Seq(
+        Literal("max_a"), 'max_a,
+        Literal("sum_a"), 'sum_a,
+        Literal("b"), 'b
+      )).as("mergedValue"))
+    val analyzedMergedSubquery = mergedSubquery.analyze
+    val correctAnswer = WithCTE(
+      testRelation
+        .select(
+          extractorExpression(0, analyzedMergedSubquery.output, 0),
+          extractorExpression(0, analyzedMergedSubquery.output, 1),
+          extractorExpression(0, analyzedMergedSubquery.output, 2),
+          extractorExpression(0, analyzedMergedSubquery.output, 0),
+          extractorExpression(0, analyzedMergedSubquery.output, 1),
+          extractorExpression(0, analyzedMergedSubquery.output, 2)),
+      Seq(CTERelationDef(analyzedMergedSubquery, 0)))
+
+    comparePlans(Optimize.execute(originalQuery.analyze), 
correctAnswer.analyze)
+  }
+
+  test("Merging subqueries with filters") {
+    val subquery1 = ScalarSubquery(testRelation.where('a > 1).select('a))
+    // Despite having an extra Project node, `subquery2` is mergeable with 
`subquery1`
+    val subquery2 = ScalarSubquery(testRelation.where('a > 
1).select('b.as("b_1")).select('b_1))
+    // Despite lacking a Project node, `subquery3` is mergeable with the 
result of merging
+    // `subquery1` and `subquery2`

Review Comment:
   Add a test case to let subquery3 and subquery4 have different filter 
conditions?



##########
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueriesSuite.scala:
##########
@@ -0,0 +1,355 @@
+/*
+ * 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 org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.expressions.{Attribute, 
CreateNamedStruct, GetStructField, Literal, ScalarSubquery}
+import org.apache.spark.sql.catalyst.expressions.aggregate.{CollectList, 
CollectSet}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules._
+
+class MergeScalarSubqueriesSuite extends PlanTest {
+
+  override def beforeEach(): Unit = {
+    CTERelationDef.curId.set(0)
+  }
+
+  private object Optimize extends RuleExecutor[LogicalPlan] {
+    val batches = Batch("MergeScalarSubqueries", Once, MergeScalarSubqueries) 
:: Nil
+  }
+
+  val testRelation = LocalRelation('a.int, 'b.int, 'c.string)
+
+  private def extractorExpression(cteIndex: Int, output: Seq[Attribute], 
fieldIndex: Int) = {
+    GetStructField(
+      ScalarSubquery(CTERelationRef(cteIndex, _resolved = true, output, 
subquery = true)),
+      fieldIndex).as("scalarsubquery()")
+  }
+
+  test("Merging subqueries with projects") {
+    val subquery1 = ScalarSubquery(testRelation.select(('a + 1).as("a_plus1")))
+    val subquery2 = ScalarSubquery(testRelation.select(('a + 2).as("a_plus2")))
+    val subquery3 = ScalarSubquery(testRelation.select('b))
+    val subquery4 = ScalarSubquery(testRelation.select(('a + 
1).as("a_plus1_2")))
+    val subquery5 = ScalarSubquery(testRelation.select(('a + 
2).as("a_plus2_2")))
+    val subquery6 = ScalarSubquery(testRelation.select('b.as("b_2")))
+    val originalQuery = testRelation
+      .select(
+        subquery1,
+        subquery2,
+        subquery3,
+        subquery4,
+        subquery5,
+        subquery6)
+
+    val mergedSubquery = testRelation
+      .select(
+        ('a + 1).as("a_plus1"),
+        ('a + 2).as("a_plus2"),
+        'b)
+      .select(
+        CreateNamedStruct(Seq(
+          Literal("a_plus1"), 'a_plus1,
+          Literal("a_plus2"), 'a_plus2,
+          Literal("b"), 'b
+        )).as("mergedValue"))
+    val analyzedMergedSubquery = mergedSubquery.analyze
+    val correctAnswer = WithCTE(
+      testRelation
+        .select(
+          extractorExpression(0, analyzedMergedSubquery.output, 0),
+          extractorExpression(0, analyzedMergedSubquery.output, 1),
+          extractorExpression(0, analyzedMergedSubquery.output, 2),
+          extractorExpression(0, analyzedMergedSubquery.output, 0),
+          extractorExpression(0, analyzedMergedSubquery.output, 1),
+          extractorExpression(0, analyzedMergedSubquery.output, 2)),
+      Seq(CTERelationDef(analyzedMergedSubquery, 0)))
+
+    comparePlans(Optimize.execute(originalQuery.analyze), 
correctAnswer.analyze)
+  }
+
+  test("Merging subqueries with aggregates") {
+    val subquery1 = 
ScalarSubquery(testRelation.groupBy('b)(max('a).as("max_a")))
+    val subquery2 = 
ScalarSubquery(testRelation.groupBy('b)(sum('a).as("sum_a")))
+    val subquery3 = ScalarSubquery(testRelation.groupBy('b)('b))
+    val subquery4 = 
ScalarSubquery(testRelation.groupBy('b)(max('a).as("max_a_2")))
+    val subquery5 = 
ScalarSubquery(testRelation.groupBy('b)(sum('a).as("sum_a_2")))
+    val subquery6 = ScalarSubquery(testRelation.groupBy('b)('b.as("b_2")))
+    val originalQuery = testRelation
+      .select(
+        subquery1,
+        subquery2,
+        subquery3,
+        subquery4,
+        subquery5,
+        subquery6)
+
+    val mergedSubquery = testRelation
+      .groupBy('b)(
+        max('a).as("max_a"),
+        sum('a).as("sum_a"),
+        'b)
+      .select(CreateNamedStruct(Seq(
+        Literal("max_a"), 'max_a,
+        Literal("sum_a"), 'sum_a,
+        Literal("b"), 'b
+      )).as("mergedValue"))
+    val analyzedMergedSubquery = mergedSubquery.analyze
+    val correctAnswer = WithCTE(
+      testRelation
+        .select(
+          extractorExpression(0, analyzedMergedSubquery.output, 0),
+          extractorExpression(0, analyzedMergedSubquery.output, 1),
+          extractorExpression(0, analyzedMergedSubquery.output, 2),
+          extractorExpression(0, analyzedMergedSubquery.output, 0),
+          extractorExpression(0, analyzedMergedSubquery.output, 1),
+          extractorExpression(0, analyzedMergedSubquery.output, 2)),
+      Seq(CTERelationDef(analyzedMergedSubquery, 0)))
+
+    comparePlans(Optimize.execute(originalQuery.analyze), 
correctAnswer.analyze)
+  }
+
+  test("Merging subqueries with filters") {
+    val subquery1 = ScalarSubquery(testRelation.where('a > 1).select('a))
+    // Despite having an extra Project node, `subquery2` is mergeable with 
`subquery1`
+    val subquery2 = ScalarSubquery(testRelation.where('a > 
1).select('b.as("b_1")).select('b_1))
+    // Despite lacking a Project node, `subquery3` is mergeable with the 
result of merging
+    // `subquery1` and `subquery2`
+    val subquery3 = 
ScalarSubquery(testRelation.select('a.as("a_2")).where('a_2 > 1).select('a_2))
+    val subquery4 = ScalarSubquery(
+      testRelation.select('a.as("a_2"), 'b).where('a_2 > 
1).select('b.as("b_2")))
+    val originalQuery = testRelation
+      .select(
+        subquery1,
+        subquery2,
+        subquery3,
+        subquery4)
+
+    val mergedSubquery = testRelation
+      .select('a, 'b, 'c)
+      .where('a > 1)
+      .select('a, 'b, 'c)
+      .select('a, 'b)
+      .select(CreateNamedStruct(Seq(
+        Literal("a"), 'a,
+        Literal("b_1"), 'b
+      )).as("mergedValue"))
+    val analyzedMergedSubquery = mergedSubquery.analyze
+    val correctAnswer = WithCTE(
+      testRelation
+        .select(
+          extractorExpression(0, analyzedMergedSubquery.output, 0),
+          extractorExpression(0, analyzedMergedSubquery.output, 1),
+          extractorExpression(0, analyzedMergedSubquery.output, 0),
+          extractorExpression(0, analyzedMergedSubquery.output, 1)),
+      Seq(CTERelationDef(analyzedMergedSubquery, 0)))
+
+    comparePlans(Optimize.execute(originalQuery.analyze), 
correctAnswer.analyze)
+  }
+
+  test("Merging subqueries with aggregate filters") {
+    val subquery1 = ScalarSubquery(
+      testRelation.having('b)(max('a).as("max_a"))(max('a) > 1))
+    val subquery2 = ScalarSubquery(
+      testRelation.having('b)(sum('a).as("sum_a"))(max('a) > 1))
+    val originalQuery = testRelation.select(
+      subquery1,
+      subquery2)
+
+    val mergedSubquery = testRelation
+      .having('b)(
+        max('a).as("max_a"),
+        sum('a).as("sum_a"))('max_a > 1)
+      .select(
+        'max_a,
+        'sum_a)
+      .select(CreateNamedStruct(Seq(
+        Literal("max_a"), 'max_a,
+        Literal("sum_a"), 'sum_a
+      )).as("mergedValue"))
+    val analyzedMergedSubquery = mergedSubquery.analyze
+    val correctAnswer = WithCTE(
+      testRelation
+        .select(
+          extractorExpression(0, analyzedMergedSubquery.output, 0),
+          extractorExpression(0, analyzedMergedSubquery.output, 1)),
+      Seq(CTERelationDef(analyzedMergedSubquery, 0)))
+
+    comparePlans(Optimize.execute(originalQuery.analyze), 
correctAnswer.analyze)
+  }
+
+  test("Merging subqueries with joins") {
+    val subquery1 = ScalarSubquery(testRelation.as("t1")
+      .join(
+        testRelation.as("t2"),
+        Inner,
+        Some($"t1.b" === $"t2.b"))
+      .select($"t1.a").analyze)
+    val subquery2 = ScalarSubquery(testRelation.as("t1")
+      .select('a.as("a_1"), 'b.as("b_1"), 'c.as("c_1"))
+      .join(
+        testRelation.as("t2").select('a.as("a_2"), 'b.as("b_2"), 'c.as("c_2")),
+        Inner,
+        Some('b_1 === 'b_2))

Review Comment:
   Add a test case with different join conditions in subquery1 and subquery2?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala:
##########
@@ -0,0 +1,370 @@
+/*
+ * 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
+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, CTERelationDef, 
CTERelationRef, Filter, Join, LogicalPlan, Project, Subquery, WithCTE}
+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 temporal
+ *   [[ScalarSubqueryReference]] reference pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a result 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 contains the list of 
attributes form the scalar
+ *   return value of a merged subquery.
+ * - A second traversal checks if there are merged subqueries in the cache and 
builds a `WithCTE`
+ *   node from these queries. The `CTERelationDef` nodes contain the merged 
subquery in the
+ *   following form:
+ *   `Project(Seq(CreateNamedStruct(name1, attribute1, ...) AS mergedValue), 
mergedSubqueryPlan)`
+ *   and the definitions are flagged that they host a subquery, that can 
return maximum one row.
+ *   During the second traversal [[ScalarSubqueryReference]] expressions that 
pont to a merged
+ *   subquery is either transformed to a 
`GetStructField(ScalarSubquery(CTERelationRef(...)))`
+ *   expression or restored to the original [[ScalarSubquery]].
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t),
+ *   (SELECT sum(b) FROM t)
+ *
+ * is optimized from:
+ *
+ * == Optimized Logical Plan ==
+ * Project [scalar-subquery#242 [] AS scalarsubquery()#253,
+ *          scalar-subquery#243 [] AS scalarsubquery()#254L]
+ * :  :- Aggregate [avg(a#244) AS avg(a)#247]
+ * :  :  +- Project [a#244]
+ * :  :     +- Relation default.t[a#244,b#245] parquet
+ * :  +- Aggregate [sum(a#251) AS sum(a)#250L]
+ * :     +- Project [a#251]
+ * :        +- Relation default.t[a#251,b#252] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * WithCTE
+ * :- CTERelationDef 0
+ * :  +- Project [named_struct(avg(a), avg(a)#247, sum(a), sum(a)#250L) AS 
mergedValue#260]
+ * :     +- Aggregate [avg(a#244) AS avg(a)#247, sum(a#244) AS sum(a)#250L]
+ * :        +- Project [a#244]
+ * :           +- Relation default.t[a#244,b#245] parquet
+ * +- Project [scalar-subquery#242 [].avg(a) AS scalarsubquery()#253,
+ *             scalar-subquery#243 [].sum(a) AS scalarsubquery()#254L]
+ *    :  :- CTERelationRef 0, true, [mergedValue#260], true
+ *    :  +- CTERelationRef 0, true, [mergedValue#260], true
+ *    +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    plan match {
+      case s: Subquery => s.copy(child = 
extractCommonScalarSubqueries(s.child))
+      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

Review Comment:
   Add comment for the string field in elements?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala:
##########
@@ -0,0 +1,370 @@
+/*
+ * 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
+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, CTERelationDef, 
CTERelationRef, Filter, Join, LogicalPlan, Project, Subquery, WithCTE}
+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 temporal
+ *   [[ScalarSubqueryReference]] reference pointing to its cached version.
+ *   The cache uses a flag to keep track of if a cache entry is a result 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 contains the list of 
attributes form the scalar
+ *   return value of a merged subquery.
+ * - A second traversal checks if there are merged subqueries in the cache and 
builds a `WithCTE`
+ *   node from these queries. The `CTERelationDef` nodes contain the merged 
subquery in the
+ *   following form:
+ *   `Project(Seq(CreateNamedStruct(name1, attribute1, ...) AS mergedValue), 
mergedSubqueryPlan)`
+ *   and the definitions are flagged that they host a subquery, that can 
return maximum one row.
+ *   During the second traversal [[ScalarSubqueryReference]] expressions that 
pont to a merged
+ *   subquery is either transformed to a 
`GetStructField(ScalarSubquery(CTERelationRef(...)))`
+ *   expression or restored to the original [[ScalarSubquery]].
+ *
+ * Eg. the following query:
+ *
+ * SELECT
+ *   (SELECT avg(a) FROM t),
+ *   (SELECT sum(b) FROM t)
+ *
+ * is optimized from:
+ *
+ * == Optimized Logical Plan ==
+ * Project [scalar-subquery#242 [] AS scalarsubquery()#253,
+ *          scalar-subquery#243 [] AS scalarsubquery()#254L]
+ * :  :- Aggregate [avg(a#244) AS avg(a)#247]
+ * :  :  +- Project [a#244]
+ * :  :     +- Relation default.t[a#244,b#245] parquet
+ * :  +- Aggregate [sum(a#251) AS sum(a)#250L]
+ * :     +- Project [a#251]
+ * :        +- Relation default.t[a#251,b#252] parquet
+ * +- OneRowRelation
+ *
+ * to:
+ *
+ * WithCTE
+ * :- CTERelationDef 0
+ * :  +- Project [named_struct(avg(a), avg(a)#247, sum(a), sum(a)#250L) AS 
mergedValue#260]
+ * :     +- Aggregate [avg(a#244) AS avg(a)#247, sum(a#244) AS sum(a)#250L]
+ * :        +- Project [a#244]
+ * :           +- Relation default.t[a#244,b#245] parquet
+ * +- Project [scalar-subquery#242 [].avg(a) AS scalarsubquery()#253,
+ *             scalar-subquery#243 [].sum(a) AS scalarsubquery()#254L]
+ *    :  :- CTERelationRef 0, true, [mergedValue#260], true
+ *    :  +- CTERelationRef 0, true, [mergedValue#260], true
+ *    +- OneRowRelation
+ */
+object MergeScalarSubqueries extends Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    plan match {
+      case s: Subquery => s.copy(child = 
extractCommonScalarSubqueries(s.child))
+      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, subqueryCTEs) = removeReferences(insertReferences(plan, 
cache), cache)
+    if (subqueryCTEs.nonEmpty) {
+      WithCTE(newPlan, subqueryCTEs)
+    } 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".
+  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. This implementation:
+        // - Supports only a whitelist of nodes (see `supportedMerge()`).
+        // - All children need to be direct fields of the tree node case 
class. Tree nodes where a
+        //   child is wrapped into a `Seq` or `Option` (e.g. `Union`) is not 
supported.
+        // - Expressions can be wrapped.
+        // - Children are tried to be merged in the same order.
+        case (np, cp) if supportedMerge(np) && np.getClass == cp.getClass &&
+            np.children.size == cp.children.size &&

Review Comment:
   can we have two explicit branches for supported cases:
   
   case (np: Filter, cp: Filter) => ...
   
   case (np: Join, cp: Join) => ...
   
   The code seems simpler to read and might have fewer surprises? 
   It's not very obvious to me how different Filter/Join conditions get merged?



-- 
This is an automated message from the 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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to