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

yamamuro pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new 8207e2f  [SPARK-34781][SQL] Eliminate LEFT SEMI/ANTI joins to its left 
child side in AQE
8207e2f is described below

commit 8207e2f65cc2ce2d87ee60ee05a2c1ee896cf93e
Author: Cheng Su <chen...@fb.com>
AuthorDate: Fri Mar 19 09:41:52 2021 +0900

    [SPARK-34781][SQL] Eliminate LEFT SEMI/ANTI joins to its left child side in 
AQE
    
    ### What changes were proposed in this pull request?
    
    In `EliminateJoinToEmptyRelation.scala`, we can extend it to cover more 
cases for LEFT SEMI and LEFT ANI joins:
    
    * Join is left semi join, join right side is non-empty and condition is 
empty. Eliminate join to its left side.
    * Join is left anti join, join right side is empty. Eliminate join to its 
left side.
    
    Given we eliminate join to its left side here, renaming the current 
optimization rule to `EliminateUnnecessaryJoin` instead.
    In addition, also change to use `checkRowCount()` to check run time row 
count, instead of using `EmptyHashedRelation`. So this can cover 
`BroadcastNestedLoopJoin` as well. (`BroadcastNestedLoopJoin`'s broadcast side 
is `Array[InternalRow]`, not `HashedRelation`).
    
    ### Why are the changes needed?
    
    Cover more join cases, and improve query performance for affected queries.
    
    ### Does this PR introduce _any_ user-facing change?
    
    No.
    
    ### How was this patch tested?
    
    Added unit tests in `AdaptiveQueryExecSuite.scala`.
    
    Closes #31873 from c21/aqe-join.
    
    Authored-by: Cheng Su <chen...@fb.com>
    Signed-off-by: Takeshi Yamamuro <yamam...@apache.org>
---
 .../sql/execution/adaptive/AQEOptimizer.scala      |  2 +-
 .../adaptive/EliminateJoinToEmptyRelation.scala    | 71 -----------------
 .../adaptive/EliminateUnnecessaryJoin.scala        | 91 ++++++++++++++++++++++
 .../spark/sql/DynamicPartitionPruningSuite.scala   |  2 +-
 .../adaptive/AdaptiveQueryExecSuite.scala          | 51 ++++++++----
 5 files changed, 127 insertions(+), 90 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEOptimizer.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEOptimizer.scala
index 04b8ade..901637d 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEOptimizer.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEOptimizer.scala
@@ -29,7 +29,7 @@ class AQEOptimizer(conf: SQLConf) extends 
RuleExecutor[LogicalPlan] {
   private val defaultBatches = Seq(
     Batch("Demote BroadcastHashJoin", Once,
       DemoteBroadcastHashJoin),
-    Batch("Eliminate Join to Empty Relation", Once, 
EliminateJoinToEmptyRelation)
+    Batch("Eliminate Unnecessary Join", Once, EliminateUnnecessaryJoin)
   )
 
   final override protected def batches: Seq[Batch] = {
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/EliminateJoinToEmptyRelation.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/EliminateJoinToEmptyRelation.scala
deleted file mode 100644
index d6df522..0000000
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/EliminateJoinToEmptyRelation.scala
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * 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.execution.adaptive
-
-import 
org.apache.spark.sql.catalyst.planning.ExtractSingleColumnNullAwareAntiJoin
-import org.apache.spark.sql.catalyst.plans.{Inner, LeftAnti, LeftSemi}
-import org.apache.spark.sql.catalyst.plans.logical.{Join, LocalRelation, 
LogicalPlan}
-import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.execution.joins.{EmptyHashedRelation, 
HashedRelation, HashedRelationWithAllNullKeys}
-
-/**
- * This optimization rule detects and converts a Join to an empty 
[[LocalRelation]]:
- * 1. Join is single column NULL-aware anti join (NAAJ), and broadcasted 
[[HashedRelation]]
- *    is [[HashedRelationWithAllNullKeys]].
- *
- * 2. Join is inner or left semi join, and broadcasted [[HashedRelation]]
- *    is [[EmptyHashedRelation]].
- *    This applies to all Joins (sort merge join, shuffled hash join, and 
broadcast hash join),
- *    because sort merge join and shuffled hash join will be changed to 
broadcast hash join with AQE
- *    at the first place.
- *
- * 3. Join is left anti join without condition, and join right side is 
non-empty.
- */
-object EliminateJoinToEmptyRelation extends Rule[LogicalPlan] {
-
-  private def canEliminate(plan: LogicalPlan, relation: HashedRelation): 
Boolean = plan match {
-    case LogicalQueryStage(_, stage: BroadcastQueryStageExec) if 
stage.resultOption.get().isDefined
-      && stage.broadcast.relationFuture.get().value == relation => true
-    case _ => false
-  }
-
-  def apply(plan: LogicalPlan): LogicalPlan = plan.transformDown {
-    case j @ ExtractSingleColumnNullAwareAntiJoin(_, _)
-        if canEliminate(j.right, HashedRelationWithAllNullKeys) =>
-      LocalRelation(j.output, data = Seq.empty, isStreaming = j.isStreaming)
-
-    case j @ Join(_, _, Inner, _, _) if canEliminate(j.left, 
EmptyHashedRelation) ||
-        canEliminate(j.right, EmptyHashedRelation) =>
-      LocalRelation(j.output, data = Seq.empty, isStreaming = j.isStreaming)
-
-    case j @ Join(_, _, LeftSemi, _, _) if canEliminate(j.right, 
EmptyHashedRelation) =>
-      LocalRelation(j.output, data = Seq.empty, isStreaming = j.isStreaming)
-
-    case j @ Join(_, _, LeftAnti, None, _) =>
-      val isNonEmptyRightSide = j.right match {
-        case LogicalQueryStage(_, stage: QueryStageExec) if 
stage.resultOption.get().isDefined =>
-          stage.getRuntimeStatistics.rowCount.exists(_ > 0)
-        case _ => false
-      }
-      if (isNonEmptyRightSide) {
-        LocalRelation(j.output, data = Seq.empty, isStreaming = j.isStreaming)
-      } else {
-        j
-      }
-  }
-}
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/EliminateUnnecessaryJoin.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/EliminateUnnecessaryJoin.scala
new file mode 100644
index 0000000..fca5698
--- /dev/null
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/EliminateUnnecessaryJoin.scala
@@ -0,0 +1,91 @@
+/*
+ * 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.execution.adaptive
+
+import 
org.apache.spark.sql.catalyst.planning.ExtractSingleColumnNullAwareAntiJoin
+import org.apache.spark.sql.catalyst.plans.{Inner, LeftAnti, LeftSemi}
+import org.apache.spark.sql.catalyst.plans.logical.{Join, LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.joins.HashedRelationWithAllNullKeys
+
+/**
+ * This optimization rule detects and eliminates unnecessary Join:
+ * 1. Join is single column NULL-aware anti join (NAAJ), and broadcasted 
[[HashedRelation]]
+ *    is [[HashedRelationWithAllNullKeys]]. Eliminate join to an empty 
[[LocalRelation]].
+ *
+ * 2. Join is inner join, and either side of join is empty. Eliminate join to 
an empty
+ *    [[LocalRelation]].
+ *
+ * 3. Join is left semi join
+ *    3.1. Join right side is empty. Eliminate join to an empty 
[[LocalRelation]].
+ *    3.2. Join right side is non-empty and condition is empty. Eliminate join 
to its left side.
+ *
+ * 4. Join is left anti join
+ *    4.1. Join right side is empty. Eliminate join to its left side.
+ *    4.2. Join right side is non-empty and condition is empty. Eliminate join 
to an empty
+ *         [[LocalRelation]].
+ *
+ * This applies to all joins (sort merge join, shuffled hash join, broadcast 
hash join, and
+ * broadcast nested loop join), because sort merge join and shuffled hash join 
will be changed
+ * to broadcast hash join with AQE at the first place.
+ */
+object EliminateUnnecessaryJoin extends Rule[LogicalPlan] {
+
+  private def isRelationWithAllNullKeys(plan: LogicalPlan) = plan match {
+    case LogicalQueryStage(_, stage: BroadcastQueryStageExec)
+      if stage.resultOption.get().isDefined =>
+      stage.broadcast.relationFuture.get().value == 
HashedRelationWithAllNullKeys
+    case _ => false
+  }
+
+  private def checkRowCount(plan: LogicalPlan, hasRow: Boolean): Boolean = 
plan match {
+    case LogicalQueryStage(_, stage: QueryStageExec) if 
stage.resultOption.get().isDefined =>
+      stage.getRuntimeStatistics.rowCount match {
+        case Some(count) => hasRow == (count > 0)
+        case _ => false
+      }
+    case _ => false
+  }
+
+  def apply(plan: LogicalPlan): LogicalPlan = plan.transformDown {
+    case j @ ExtractSingleColumnNullAwareAntiJoin(_, _) if 
isRelationWithAllNullKeys(j.right) =>
+      LocalRelation(j.output, data = Seq.empty, isStreaming = j.isStreaming)
+
+    case j @ Join(_, _, Inner, _, _) if checkRowCount(j.left, hasRow = false) 
||
+      checkRowCount(j.right, hasRow = false) =>
+      LocalRelation(j.output, data = Seq.empty, isStreaming = j.isStreaming)
+
+    case j @ Join(_, _, LeftSemi, condition, _) =>
+      if (checkRowCount(j.right, hasRow = false)) {
+        LocalRelation(j.output, data = Seq.empty, isStreaming = j.isStreaming)
+      } else if (condition.isEmpty && checkRowCount(j.right, hasRow = true)) {
+        j.left
+      } else {
+        j
+      }
+
+    case j @ Join(_, _, LeftAnti, condition, _) =>
+      if (checkRowCount(j.right, hasRow = false)) {
+        j.left
+      } else if (condition.isEmpty && checkRowCount(j.right, hasRow = true)) {
+        LocalRelation(j.output, data = Seq.empty, isStreaming = j.isStreaming)
+      } else {
+        j
+      }
+  }
+}
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala
index 3163cb5..4bd5e64 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala
@@ -1370,7 +1370,7 @@ abstract class DynamicPartitionPruningSuiteBase
     withSQLConf(
       SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true",
       SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true",
-      SQLConf.ADAPTIVE_OPTIMIZER_EXCLUDED_RULES.key -> 
EliminateJoinToEmptyRelation.ruleName) {
+      SQLConf.ADAPTIVE_OPTIMIZER_EXCLUDED_RULES.key -> 
EliminateUnnecessaryJoin.ruleName) {
       val df = sql(
         """
           |SELECT * FROM fact_sk f
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala
index 598484f..d658c0e 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala
@@ -1216,14 +1216,14 @@ class AdaptiveQueryExecSuite
       SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> Long.MaxValue.toString,
       // This test is a copy of test(SPARK-32573), in order to test the 
configuration
       // `spark.sql.adaptive.optimizer.excludedRules` works as expect.
-      SQLConf.ADAPTIVE_OPTIMIZER_EXCLUDED_RULES.key -> 
EliminateJoinToEmptyRelation.ruleName) {
+      SQLConf.ADAPTIVE_OPTIMIZER_EXCLUDED_RULES.key -> 
EliminateUnnecessaryJoin.ruleName) {
       val (plan, adaptivePlan) = runAdaptiveAndVerifyResult(
         "SELECT * FROM testData2 t1 WHERE t1.b NOT IN (SELECT b FROM 
testData3)")
       val bhj = findTopLevelBroadcastHashJoin(plan)
       assert(bhj.size == 1)
       val join = findTopLevelBaseJoin(adaptivePlan)
       // this is different compares to test(SPARK-32573) due to the rule
-      // `EliminateJoinToEmptyRelation` has been excluded.
+      // `EliminateUnnecessaryJoin` has been excluded.
       assert(join.nonEmpty)
       checkNumLocalShuffleReaders(adaptivePlan)
     }
@@ -1254,21 +1254,38 @@ class AdaptiveQueryExecSuite
   test("SPARK-34533: Eliminate left anti join to empty relation") {
     withSQLConf(
       SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") {
-      withTable("emptyTestData") {
-        spark.range(0).write.saveAsTable("emptyTestData")
-        Seq(
-          // broadcast non-empty right side
-          ("SELECT /*+ broadcast(testData3) */ * FROM testData LEFT ANTI JOIN 
testData3", true),
-          // broadcast empty right side
-          ("SELECT /*+ broadcast(emptyTestData) */ * FROM testData LEFT ANTI 
JOIN emptyTestData",
-            false),
-          // broadcast left side
-          ("SELECT /*+ broadcast(testData) */ * FROM testData LEFT ANTI JOIN 
testData3", false)
-        ).foreach { case (query, isEliminated) =>
-          val (plan, adaptivePlan) = runAdaptiveAndVerifyResult(query)
-          assert(findTopLevelBaseJoin(plan).size == 1)
-          assert(findTopLevelBaseJoin(adaptivePlan).isEmpty == isEliminated)
-        }
+      Seq(
+        // broadcast non-empty right side
+        ("SELECT /*+ broadcast(testData3) */ * FROM testData LEFT ANTI JOIN 
testData3", true),
+        // broadcast empty right side
+        ("SELECT /*+ broadcast(emptyTestData) */ * FROM testData LEFT ANTI 
JOIN emptyTestData",
+          true),
+        // broadcast left side
+        ("SELECT /*+ broadcast(testData) */ * FROM testData LEFT ANTI JOIN 
testData3", false)
+      ).foreach { case (query, isEliminated) =>
+        val (plan, adaptivePlan) = runAdaptiveAndVerifyResult(query)
+        assert(findTopLevelBaseJoin(plan).size == 1)
+        assert(findTopLevelBaseJoin(adaptivePlan).isEmpty == isEliminated)
+      }
+    }
+  }
+
+  test("SPARK-34781: Eliminate left semi/anti join to its left side") {
+    withSQLConf(
+      SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") {
+      Seq(
+        // left semi join and non-empty right side
+        ("SELECT * FROM testData LEFT SEMI JOIN testData3", true),
+        // left semi join, non-empty right side and non-empty join condition
+        ("SELECT * FROM testData t1 LEFT SEMI JOIN testData3 t2 ON t1.key = 
t2.a", false),
+        // left anti join and empty right side
+        ("SELECT * FROM testData LEFT ANTI JOIN emptyTestData", true),
+        // left anti join, empty right side and non-empty join condition
+        ("SELECT * FROM testData t1 LEFT ANTI JOIN emptyTestData t2 ON t1.key 
= t2.key", true)
+      ).foreach { case (query, isEliminated) =>
+        val (plan, adaptivePlan) = runAdaptiveAndVerifyResult(query)
+        assert(findTopLevelBaseJoin(plan).size == 1)
+        assert(findTopLevelBaseJoin(adaptivePlan).isEmpty == isEliminated)
       }
     }
   }

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

Reply via email to