You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2021/04/07 23:58:24 UTC

[GitHub] [spark] allisonwang-db commented on a change in pull request #32054: [SPARK-34946][SQL] Block unsupported correlated scalar subquery in Aggregate

allisonwang-db commented on a change in pull request #32054:
URL: https://github.com/apache/spark/pull/32054#discussion_r609148331



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
##########
@@ -305,6 +305,12 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog {
                     s"nor is it an aggregate function. " +
                     "Add to group by or wrap in first() (or first_value) if you don't care " +
                     "which value you get.")
+              case s: ScalarSubquery
+                  if s.children.nonEmpty && !groupingExprs.exists(_.semanticEquals(s)) =>
+                failAnalysis(s"Correlated scalar subquery '${s.sql}' is neither " +
+                  s"present in the group by, nor in an aggregate function. Add it to group by " +
+                  s"using ordinal position or wrap it in first() (or first_value) if you don't " +
+                  s"care which value you get.")

Review comment:
       This is a good point, but there is actually another bug that is harder to resolve right now (explained below). We can add an assertion in this PR so the code will fail during query optimization instead of execution time.
   ```sql
   CREATE VIEW t(c1, c2) AS VALUES (0,1), (0,2);
   SELECT (SELECT SUM(c2) FROM t t2 WHERE t1.c1 = t2.c1) FROM t t1 GROUP BY 1
   ```
   This will fail even if the query can pass CheckAnalysis because the subquery in the grouping expressions and in the aggregate expressions are the same before the optimization phase.
   ```scala
   Aggregate [scalar-subquery#18 [c1#19]], [scalar-subquery#18 [c1#19] AS scalarsubquery(c1)#27L]
   ```
    But after the `PullupCorrelatedPredicates` rule, and the subquery outputs are deduplicated, these two subqueries are no longer semantically equal:
   ```scala
   Aggregate [scalar-subquery#18 [(c1#19 = c1#19#31)]], [scalar-subquery#18 [(c1#19 = c1#19#32)] AS scalarsubquery(c1)#27L]
   ``` 
   Notice the expression `c1#19 = c1#19#31` differs from `c1#19 = c1#19#32`.
   So the optimized plan still has an un-rewritten correlated scalar subquery.
   ```scala
   == Optimized Logical Plan ==
   Aggregate [scalar-subquery#18 [(c1#19 = c1#19#31)]], [sum(c2)#26L AS scalarsubquery(c1)#27L]
   :  +- Aggregate [c1#19], [sum(c2#20) AS sum(c2)#26L, c1#19 AS c1#19#31]
   :     +- LocalRelation [c1#19, c2#20]
   +- Project [c1#19, sum(c2)#26L]
      +- Join LeftOuter, (c1#19 = c1#19#32)
         :- LocalRelation [c1#19]
         +- Aggregate [c1#19], [sum(c2#20) AS sum(c2)#26L, c1#19 AS c1#19#32]
            +- LocalRelation [c1#19, c2#20]
   ```
   Note we cannot just compare the ID of the subquery expression because this case won't work:
   ```sql
   SELECT (SELECT SUM(c2) FROM t t2 WHERE t1.c1 = t2.c1) FROM t t1
   GROUP BY (SELECT SUM(c2) FROM t t2 WHERE t1.c1 = t2.c1)
   ```




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

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



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