You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by li...@apache.org on 2018/01/21 06:39:54 UTC

spark git commit: [SPARK-23087][SQL] CheckCartesianProduct too restrictive when condition is false/null

Repository: spark
Updated Branches:
  refs/heads/master 00d169156 -> 121dc96f0


[SPARK-23087][SQL] CheckCartesianProduct too restrictive when condition is false/null

## What changes were proposed in this pull request?

CheckCartesianProduct raises an AnalysisException also when the join condition is always false/null. In this case, we shouldn't raise it, since the result will not be a cartesian product.

## How was this patch tested?

added UT

Author: Marco Gaido <ma...@gmail.com>

Closes #20333 from mgaido91/SPARK-23087.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/121dc96f
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/121dc96f
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/121dc96f

Branch: refs/heads/master
Commit: 121dc96f088a7b157d5b2cffb626b0e22d1fc052
Parents: 00d1691
Author: Marco Gaido <ma...@gmail.com>
Authored: Sat Jan 20 22:39:49 2018 -0800
Committer: gatorsmile <ga...@gmail.com>
Committed: Sat Jan 20 22:39:49 2018 -0800

----------------------------------------------------------------------
 .../spark/sql/catalyst/optimizer/Optimizer.scala      | 10 +++++++---
 .../org/apache/spark/sql/DataFrameJoinSuite.scala     | 14 ++++++++++++++
 2 files changed, 21 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/121dc96f/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
index c794ba8..0f9daa5 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
@@ -1108,15 +1108,19 @@ object CheckCartesianProducts extends Rule[LogicalPlan] with PredicateHelper {
    */
   def isCartesianProduct(join: Join): Boolean = {
     val conditions = join.condition.map(splitConjunctivePredicates).getOrElse(Nil)
-    !conditions.map(_.references).exists(refs => refs.exists(join.left.outputSet.contains)
-        && refs.exists(join.right.outputSet.contains))
+
+    conditions match {
+      case Seq(Literal.FalseLiteral) | Seq(Literal(null, BooleanType)) => false
+      case _ => !conditions.map(_.references).exists(refs =>
+        refs.exists(join.left.outputSet.contains) && refs.exists(join.right.outputSet.contains))
+    }
   }
 
   def apply(plan: LogicalPlan): LogicalPlan =
     if (SQLConf.get.crossJoinEnabled) {
       plan
     } else plan transform {
-      case j @ Join(left, right, Inner | LeftOuter | RightOuter | FullOuter, condition)
+      case j @ Join(left, right, Inner | LeftOuter | RightOuter | FullOuter, _)
         if isCartesianProduct(j) =>
           throw new AnalysisException(
             s"""Detected cartesian product for ${j.joinType.sql} join between logical plans

http://git-wip-us.apache.org/repos/asf/spark/blob/121dc96f/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala
index aef0d7f..1656f29 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala
@@ -274,4 +274,18 @@ class DataFrameJoinSuite extends QueryTest with SharedSQLContext {
     checkAnswer(innerJoin, Row(1) :: Nil)
   }
 
+  test("SPARK-23087: don't throw Analysis Exception in CheckCartesianProduct when join condition " +
+    "is false or null") {
+    val df = spark.range(10)
+    val dfNull = spark.range(10).select(lit(null).as("b"))
+    val planNull = df.join(dfNull, $"id" === $"b", "left").queryExecution.analyzed
+
+    spark.sessionState.executePlan(planNull).optimizedPlan
+
+    val dfOne = df.select(lit(1).as("a"))
+    val dfTwo = spark.range(10).select(lit(2).as("b"))
+    val planFalse = dfOne.join(dfTwo, $"a" === $"b", "left").queryExecution.analyzed
+
+    spark.sessionState.executePlan(planFalse).optimizedPlan
+  }
 }


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