peter-toth commented on code in PR #32298:
URL: https://github.com/apache/spark/pull/32298#discussion_r853899050


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala:
##########
@@ -0,0 +1,357 @@
+/*
+ * 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.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:
   I've simplified `removeReferences()` in 
https://github.com/apache/spark/pull/32298/commits/1c4d14b8d759b57cca065d316337c7c9cc4d7cb7
 and fixed the failing tests.



-- 
This is an automated message from the 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