You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by do...@apache.org on 2023/12/07 03:23:32 UTC

(spark) branch branch-3.4 updated: [SPARK-45580][SQL][3.4] Handle case where a nested subquery becomes an existence join

This is an automated email from the ASF dual-hosted git repository.

dongjoon pushed a commit to branch branch-3.4
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.4 by this push:
     new 8e40ec6fa52 [SPARK-45580][SQL][3.4] Handle case where a nested subquery becomes an existence join
8e40ec6fa52 is described below

commit 8e40ec6fa525420c1da5ce3b8846ef9f540b9d49
Author: Bruce Robbins <be...@gmail.com>
AuthorDate: Wed Dec 6 19:23:19 2023 -0800

    [SPARK-45580][SQL][3.4] Handle case where a nested subquery becomes an existence join
    
    ### What changes were proposed in this pull request?
    
    This is a back-port of https://github.com/apache/spark/pull/44193.
    
    In `RewritePredicateSubquery`, prune existence flags from the final join when `rewriteExistentialExpr` returns an existence join. This change prunes the flags (attributes with the name "exists") by adding a `Project` node.
    
    For example:
    ```
    Join LeftSemi, ((a#13 = c1#15) OR exists#19)
    :- Join ExistenceJoin(exists#19), (a#13 = col1#17)
    :  :- LocalRelation [a#13]
    :  +- LocalRelation [col1#17]
    +- LocalRelation [c1#15]
    ```
    becomes
    ```
    Project [a#13]
    +- Join LeftSemi, ((a#13 = c1#15) OR exists#19)
       :- Join ExistenceJoin(exists#19), (a#13 = col1#17)
       :  :- LocalRelation [a#13]
       :  +- LocalRelation [col1#17]
       +- LocalRelation [c1#15]
    ```
    This change always adds the `Project` node, whether `rewriteExistentialExpr` returns an existence join or not. In the case when `rewriteExistentialExpr` does not return an existence join, `RemoveNoopOperators` will remove the unneeded `Project` node.
    
    ### Why are the changes needed?
    
    This query returns an extraneous boolean column when run in spark-sql:
    ```
    create or replace temp view t1(a) as values (1), (2), (3), (7);
    create or replace temp view t2(c1) as values (1), (2), (3);
    create or replace temp view t3(col1) as values (3), (9);
    
    select *
    from t1
    where exists (
      select c1
      from t2
      where a = c1
      or a in (select col1 from t3)
    );
    
    1       false
    2       false
    3       true
    ```
    (Note: the above query will not have the extraneous boolean column when run from the Dataset API. That is because the Dataset API truncates the rows based on the schema of the analyzed plan. The bug occurs during optimization).
    
    This query fails when run in either spark-sql or using the Dataset API:
    ```
    select (
      select *
      from t1
      where exists (
        select c1
        from t2
        where a = c1
        or a in (select col1 from t3)
      )
      limit 1
    )
    from range(1);
    
    java.lang.AssertionError: assertion failed: Expects 1 field, but got 2; something went wrong in analysis
    ```
    
    ### Does this PR introduce _any_ user-facing change?
    
    No, except for the removal of the extraneous boolean flag and the fix to the error condition.
    
    ### How was this patch tested?
    
    New unit test.
    
    ### Was this patch authored or co-authored using generative AI tooling?
    
    No.
    
    Closes #44219 from bersprockets/schema_change_br34.
    
    Authored-by: Bruce Robbins <be...@gmail.com>
    Signed-off-by: Dongjoon Hyun <dh...@apple.com>
---
 .../spark/sql/catalyst/optimizer/subquery.scala    |  9 +++--
 .../scala/org/apache/spark/sql/SubquerySuite.scala | 46 ++++++++++++++++++++++
 2 files changed, 52 insertions(+), 3 deletions(-)

diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala
index 1d2f5602630..861f2f2fabf 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala
@@ -118,16 +118,19 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper {
       withSubquery.foldLeft(newFilter) {
         case (p, Exists(sub, _, _, conditions, subHint)) =>
           val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p)
-          buildJoin(outerPlan, sub, LeftSemi, joinCond, subHint)
+          val join = buildJoin(outerPlan, sub, LeftSemi, joinCond, subHint)
+          Project(p.output, join)
         case (p, Not(Exists(sub, _, _, conditions, subHint))) =>
           val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p)
-          buildJoin(outerPlan, sub, LeftAnti, joinCond, subHint)
+          val join = buildJoin(outerPlan, sub, LeftAnti, joinCond, subHint)
+          Project(p.output, join)
         case (p, InSubquery(values, ListQuery(sub, _, _, _, conditions, subHint))) =>
           // Deduplicate conflicting attributes if any.
           val newSub = dedupSubqueryOnSelfJoin(p, sub, Some(values))
           val inConditions = values.zip(newSub.output).map(EqualTo.tupled)
           val (joinCond, outerPlan) = rewriteExistentialExpr(inConditions ++ conditions, p)
-          Join(outerPlan, newSub, LeftSemi, joinCond, JoinHint(None, subHint))
+          val join = Join(outerPlan, newSub, LeftSemi, joinCond, JoinHint(None, subHint))
+          Project(p.output, join)
         case (p, Not(InSubquery(values, ListQuery(sub, _, _, _, conditions, subHint)))) =>
           // This is a NULL-aware (left) anti join (NAAJ) e.g. col NOT IN expr
           // Construct the condition. A NULL in one of the conditions is regarded as a positive
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala
index 3cfda19134a..75eabcb96f2 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala
@@ -2668,4 +2668,50 @@ class SubquerySuite extends QueryTest
         expected)
     }
   }
+
+  test("SPARK-45580: Handle case where a nested subquery becomes an existence join") {
+    withTempView("t1", "t2", "t3") {
+      Seq((1), (2), (3), (7)).toDF("a").persist().createOrReplaceTempView("t1")
+      Seq((1), (2), (3)).toDF("c1").persist().createOrReplaceTempView("t2")
+      Seq((3), (9)).toDF("col1").persist().createOrReplaceTempView("t3")
+
+      val query1 =
+        """
+          |SELECT *
+          |FROM t1
+          |WHERE EXISTS (
+          |  SELECT c1
+          |  FROM t2
+          |  WHERE a = c1
+          |  OR a IN (SELECT col1 FROM t3)
+          |)""".stripMargin
+      val df1 = sql(query1)
+      checkAnswer(df1, Row(1) :: Row(2) :: Row(3) :: Nil)
+
+      val query2 =
+        """
+          |SELECT *
+          |FROM t1
+          |WHERE a IN (
+          |  SELECT c1
+          |  FROM t2
+          |  where a IN (SELECT col1 FROM t3)
+          |)""".stripMargin
+      val df2 = sql(query2)
+      checkAnswer(df2, Row(3))
+
+      val query3 =
+        """
+          |SELECT *
+          |FROM t1
+          |WHERE NOT EXISTS (
+          |  SELECT c1
+          |  FROM t2
+          |  WHERE a = c1
+          |  OR a IN (SELECT col1 FROM t3)
+          |)""".stripMargin
+      val df3 = sql(query3)
+      checkAnswer(df3, Row(7))
+    }
+  }
 }


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