Skip to content

Commit

Permalink
[SPARK-38333][SQL] PlanExpression expression should skip addExprTree …
Browse files Browse the repository at this point in the history
…function in Executor

It is master branch pr [SPARK-38333](#35662)

Bug fix, it is potential issue.

No

UT

Closes #36012 from monkeyboy123/spark-38333.

Authored-by: Dereck Li <monkeyboy.ljh@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit a40acd4)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
  • Loading branch information
monkeyboy123 authored and cloud-fan committed Mar 31, 2022
1 parent 01600ae commit e78cca9
Show file tree
Hide file tree
Showing 2 changed files with 18 additions and 2 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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}

Expand Down Expand Up @@ -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")))

Expand Down

0 comments on commit e78cca9

Please sign in to comment.