You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by we...@apache.org on 2022/03/31 13:37:25 UTC

[spark] branch branch-3.2 updated: [SPARK-38333][SQL] PlanExpression expression should skip addExprTree function in Executor

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

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


The following commit(s) were added to refs/heads/branch-3.2 by this push:
     new e78cca9  [SPARK-38333][SQL] PlanExpression expression should skip addExprTree function in Executor
e78cca9 is described below

commit e78cca96f586a7812a119d8e2e72940767e37bd9
Author: Dereck Li <mo...@gmail.com>
AuthorDate: Thu Mar 31 21:33:08 2022 +0800

    [SPARK-38333][SQL] PlanExpression expression should skip addExprTree function in Executor
    
    It is master branch pr [SPARK-38333](https://github.com/apache/spark/pull/35662)
    
    Bug fix, it is potential issue.
    
    No
    
    UT
    
    Closes #36012 from monkeyboy123/spark-38333.
    
    Authored-by: Dereck Li <mo...@gmail.com>
    Signed-off-by: Wenchen Fan <we...@databricks.com>
    (cherry picked from commit a40acd4392a8611062763ce6ec7bc853d401c646)
    Signed-off-by: Wenchen Fan <we...@databricks.com>
---
 .../catalyst/expressions/EquivalentExpressions.scala   |  2 +-
 .../expressions/SubexpressionEliminationSuite.scala    | 18 +++++++++++++++++-
 2 files changed, 18 insertions(+), 2 deletions(-)

diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala
index ef04e88..a288ea5 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala
@@ -165,7 +165,7 @@ class EquivalentExpressions {
       expr.find(_.isInstanceOf[LambdaVariable]).isDefined ||
       // `PlanExpression` wraps query plan. To compare query plans of `PlanExpression` on executor,
       // can cause error like NPE.
-      (expr.isInstanceOf[PlanExpression[_]] && TaskContext.get != null)
+      (expr.find(_.isInstanceOf[PlanExpression[_]]).isDefined && TaskContext.get != null)
 
     if (!skip && !addExprToMap(expr, map)) {
       childrenToRecurse(expr).foreach(addExprTree(_, map))
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala
index 6fc9d04..4b353cd 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala
@@ -16,8 +16,9 @@
  */
 package org.apache.spark.sql.catalyst.expressions
 
-import org.apache.spark.SparkFunSuite
+import org.apache.spark.{SparkFunSuite, TaskContext, TaskContextImpl}
 import org.apache.spark.sql.catalyst.expressions.codegen._
+import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types.{BinaryType, DataType, Decimal, IntegerType}
 
@@ -391,6 +392,21 @@ class SubexpressionEliminationSuite extends SparkFunSuite with ExpressionEvalHel
     }
   }
 
+  test("SPARK-38333: PlanExpression expression should skip addExprTree function in Executor") {
+    try {
+      // suppose we are in executor
+      val context1 = new TaskContextImpl(0, 0, 0, 0, 0, null, null, null, cpus = 0)
+      TaskContext.setTaskContext(context1)
+
+      val equivalence = new EquivalentExpressions
+      val expression = DynamicPruningExpression(Exists(LocalRelation()))
+      equivalence.addExprTree(expression)
+      assert(equivalence.getExprState(expression).isEmpty)
+    } finally {
+      TaskContext.unset()
+    }
+  }
+
   test("SPARK-35886: PromotePrecision should not overwrite genCode") {
     val p = PromotePrecision(Literal(Decimal("10.1")))
 

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