You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "allisonwang-db (via GitHub)" <gi...@apache.org> on 2023/08/10 22:30:13 UTC

[GitHub] [spark] allisonwang-db commented on a diff in pull request #41301: [SPARK-43780][SQL] Support correlated references in join predicates

allisonwang-db commented on code in PR #41301:
URL: https://github.com/apache/spark/pull/41301#discussion_r1290738214


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:
##########
@@ -4268,6 +4268,14 @@ object SQLConf {
       .checkValue(_ >= 0, "The threshold of cached local relations must not be negative")
       .createWithDefault(64 * 1024 * 1024)
 
+  val DECORRELATE_JOIN_PREDICATE_ENABLED =
+    buildConf("spark.sql.optimizer.decorrelateJoinPredicate.enabled")
+      .internal()
+      .doc("Decorrelate subqueries with correlated references in join predicates.")
+      .version("3.5.0")

Review Comment:
   Should this be `4.0.0`?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/DecorrelateInnerQuery.scala:
##########
@@ -804,18 +804,74 @@ object DecorrelateInnerQuery extends PredicateHelper {
             (d.copy(child = newChild), joinCond, outerReferenceMap)
 
           case j @ Join(left, right, joinType, condition, _) =>
-            val outerReferences = collectOuterReferences(j.expressions)
-            // Join condition containing outer references is not supported.
-            assert(outerReferences.isEmpty, s"Correlated column is not allowed in join: $j")
-            val newOuterReferences = parentOuterReferences ++ outerReferences
-            val shouldPushToLeft = joinType match {
+            // Given 'condition', computes the tuple of
+            // (correlated, uncorrelated, equalityCond, predicates, equivalences).
+            // 'correlated' and 'uncorrelated' are the conjuncts with (resp. without)
+            // outer (correlated) references. Furthermore, correlated conjuncts are split
+            // into 'equalityCond' (those that are equalities) and all rest ('predicates').
+            // 'equivalences' track equivalent attributes given 'equalityCond'.
+            def splitCorrelatedPredicate(condition: Option[Expression],
+                                         isInnerJoin: Boolean,
+                                         shouldDecorrelatePredicates: Boolean):
+            (Seq[Expression], Seq[Expression], Seq[Expression],
+              Seq[Expression], AttributeMap[Attribute]) = {

Review Comment:
   ```suggestion
               def splitCorrelatedPredicate(
                   condition: Option[Expression],
                   isInnerJoin: Boolean,
                   shouldDecorrelatePredicates: Boolean):
               (Seq[Expression], Seq[Expression], Seq[Expression],
                 Seq[Expression], AttributeMap[Attribute]) = {
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/DecorrelateInnerQuery.scala:
##########
@@ -804,18 +804,67 @@ object DecorrelateInnerQuery extends PredicateHelper {
             (d.copy(child = newChild), joinCond, outerReferenceMap)
 
           case j @ Join(left, right, joinType, condition, _) =>
-            val outerReferences = collectOuterReferences(j.expressions)
-            // Join condition containing outer references is not supported.
-            assert(outerReferences.isEmpty, s"Correlated column is not allowed in join: $j")
-            val newOuterReferences = parentOuterReferences ++ outerReferences
-            val shouldPushToLeft = joinType match {
+            def splitCorrelatedPredicate(condition: Option[Expression],
+                                         isInnerJoin: Boolean,
+                                         shouldDecorrelatePredicates: Boolean):
+            (Seq[Expression], Seq[Expression], Seq[Expression],
+              Seq[Expression], AttributeMap[Attribute]) = {

Review Comment:
   Thanks for adding the comment! If possible, can we also include a minimal example in this comment?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:
##########
@@ -4268,6 +4268,14 @@ object SQLConf {
       .checkValue(_ >= 0, "The threshold of cached local relations must not be negative")
       .createWithDefault(64 * 1024 * 1024)
 
+  val DECORRELATE_JOIN_PREDICATE_ENABLED =
+    buildConf("spark.sql.optimizer.decorrelateJoinPredicate.enabled")
+      .internal()
+      .doc("Decorrelate subqueries with correlated references in join predicates.")

Review Comment:
   We should also mention that the decorrelate inner query config



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/DecorrelateInnerQuery.scala:
##########
@@ -804,18 +804,74 @@ object DecorrelateInnerQuery extends PredicateHelper {
             (d.copy(child = newChild), joinCond, outerReferenceMap)
 
           case j @ Join(left, right, joinType, condition, _) =>
-            val outerReferences = collectOuterReferences(j.expressions)
-            // Join condition containing outer references is not supported.
-            assert(outerReferences.isEmpty, s"Correlated column is not allowed in join: $j")
-            val newOuterReferences = parentOuterReferences ++ outerReferences
-            val shouldPushToLeft = joinType match {
+            // Given 'condition', computes the tuple of
+            // (correlated, uncorrelated, equalityCond, predicates, equivalences).
+            // 'correlated' and 'uncorrelated' are the conjuncts with (resp. without)
+            // outer (correlated) references. Furthermore, correlated conjuncts are split
+            // into 'equalityCond' (those that are equalities) and all rest ('predicates').
+            // 'equivalences' track equivalent attributes given 'equalityCond'.
+            def splitCorrelatedPredicate(condition: Option[Expression],
+                                         isInnerJoin: Boolean,
+                                         shouldDecorrelatePredicates: Boolean):

Review Comment:
   Can we also mention these two parameters in the comment?



##########
sql/core/src/test/resources/sql-tests/inputs/join-lateral.sql:
##########
@@ -101,6 +101,10 @@ SELECT * FROM t1 WHERE c1 = (SELECT MIN(a) FROM t2, LATERAL (SELECT c1 AS a));
 -- lateral join inside correlated subquery
 SELECT * FROM t1 WHERE c1 = (SELECT MIN(a) FROM t2, LATERAL (SELECT c1 AS a) WHERE c1 = t1.c1);
 
+-- join condition has a correlated reference to the left side of the lateral join
+SELECT * FROM t1 JOIN lateral (SELECT * FROM t2 JOIN t4 ON t2.c1 = t4.c1 AND t2.c1 = t1.c1);
+SELECT * FROM t1 LEFT JOIN lateral (SELECT * FROM t4 LEFT JOIN t2 ON t2.c1 = t4.c1 AND t2.c1 = t1.c1);
+

Review Comment:
   Shall we add one more test for inner join with non-equality join conditions here?



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

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

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


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