Skip to content

Conversation

@jaceklaskowski
Copy link
Contributor

What changes were proposed in this pull request?

Avoids ("fixes") a NullPointerException in subexpression elimination for subqueries with FileSourceScanExec.

How was this patch tested?

Local build. No new tests as I could not reproduce it other than using the query and data under NDA. Waiting for Jenkins.

@SparkQA
Copy link

SparkQA commented Mar 18, 2018

Test build #88359 has finished for PR 20856 at commit 3981421.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@kiszk
Copy link
Member

kiszk commented Mar 19, 2018

retest this please

@kiszk
Copy link
Member

kiszk commented Mar 19, 2018

I think that this problem does not depend on data. We could use different data.
Would it be possible to create a test case by modifying the query slightly with different column names?

@SparkQA
Copy link

SparkQA commented Mar 19, 2018

Test build #88362 has finished for PR 20856 at commit 3981421.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@gatorsmile
Copy link
Member

What is the test case?

@jaceklaskowski
Copy link
Contributor Author

I spent over 2 days applying different modifications to the query hoping I could cut the number of CASE WHENs and other projections, but noticed no correlation between the number or their "types". I'll see if renaming the columns leads to the issue and submit a test case. Thanks for your support!

@jaceklaskowski
Copy link
Contributor Author

jaceklaskowski commented Mar 20, 2018

BTW, I've just realized that even without the query it's clear that creating a new FileSourceScanExec with no relation will end up with a NPE from the supportsBatch field.

@SparkQA
Copy link

SparkQA commented May 11, 2018

Test build #4171 has finished for PR 20856 at commit 3981421.

  • This patch fails Spark unit tests.
  • This patch does not merge cleanly.
  • This patch adds no public classes.

@HyukjinKwon
Copy link
Member

retest this please

@gatorsmile
Copy link
Member

We are not able to merge it without a valid test case. We need to understand the root cause why relation can be null.

@HyukjinKwon If you can help, please try to create a test case.

@HyukjinKwon
Copy link
Member

Sure. Let me give a shot to reproduce it.

@SparkQA
Copy link

SparkQA commented Jul 16, 2018

Test build #93077 has finished for PR 20856 at commit 3981421.

  • This patch fails due to an unknown error code, -9.
  • This patch merges cleanly.
  • This patch adds no public classes.

@HyukjinKwon
Copy link
Member

HyukjinKwon commented Jul 18, 2018

Okay, I was investigating this and the fix itself looks quite inappropriate.

This looks what happened now. I can reproduce this by a bit of messy way:

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 8d06804ce1e..d25fc9a7ba9 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
@@ -37,7 +37,9 @@ class EquivalentExpressions {
       case _ => false
     }
    
-    override def hashCode: Int = e.semanticHash()
+    override def hashCode: Int = {
+      1
+    }
   }
spark.range(1).write.mode("overwrite").parquet("/tmp/foo")
spark.read.parquet("/tmp/foo").createOrReplaceTempView("foo")
spark.conf.set("spark.sql.codegen.wholeStage", false)
sql("SELECT (SELECT id FROM foo) == (SELECT id FROM foo)").collect()

This is what I see and think:

  1. Sub scalar query was made (for instance SELECT (SELECT id FROM foo)).

  2. Try to extract some common expressions (via CodeGenerator.subexpressionElimination) so that it can generates some common codes and can be reused.

  3. During this, seems it extracts some expressions that can be reused (via EquivalentExpressions.addExprTree)

expressions.foreach(equivalentExpressions.addExprTree(_))

  1. During this, if the hash (EquivalentExpressions.Expr.hashCode) happened to be the same at EquivalentExpressions.addExpr anyhow, EquivalentExpressions.Expr.equals is called to identify object in the same hash, which eventually calls semanticEquals in ScalarSubquery

  1. ScalarSubquery's semanticEquals needs SubqueryExec's sameResult

case s: ScalarSubquery => plan.sameResult(s.plan)

  1. SubqueryExec's sameResult requires a canonicalized plan which calls FileSourceScanExec's doCanonicalize

final def sameResult(other: PlanType): Boolean = this.canonicalized == other.canonicalized

  1. In FileSourceScanExec's doCanonicalize, FileSourceScanExec's relation is required but seems @transient so it becomes null.

  1. NPE is thrown:
java.lang.NullPointerException
	at org.apache.spark.sql.execution.FileSourceScanExec.<init>(DataSourceScanExec.scala:169)
	at org.apache.spark.sql.execution.FileSourceScanExec.doCanonicalize(DataSourceScanExec.scala:526)
	at org.apache.spark.sql.execution.FileSourceScanExec.doCanonicalize(DataSourceScanExec.scala:159)
	at org.apache.spark.sql.catalyst.plans.QueryPlan.canonicalized$lzycompute(QueryPlan.scala:211)
	at org.apache.spark.sql.catalyst.plans.QueryPlan.canonicalized(QueryPlan.scala:210)
	at org.apache.spark.sql.catalyst.plans.QueryPlan$$anonfun$3.apply(QueryPlan.scala:225)
	at org.apache.spark.sql.catalyst.plans.QueryPlan$$anonfun$3.apply(QueryPlan.scala:225)
	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
	at scala.collection.immutable.List.foreach(List.scala:392)
	at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
	at scala.collection.immutable.List.map(List.scala:296)
	at org.apache.spark.sql.catalyst.plans.QueryPlan.doCanonicalize(QueryPlan.scala:225)
	at org.apache.spark.sql.catalyst.plans.QueryPlan.canonicalized$lzycompute(QueryPlan.scala:211)
	at org.apache.spark.sql.catalyst.plans.QueryPlan.canonicalized(QueryPlan.scala:210)
	at org.apache.spark.sql.catalyst.plans.QueryPlan.sameResult(QueryPlan.scala:258)
	at org.apache.spark.sql.execution.ScalarSubquery.semanticEquals(subquery.scala:58)
	at org.apache.spark.sql.catalyst.expressions.EquivalentExpressions$Expr.equals(EquivalentExpressions.scala:36)
	at scala.collection.mutable.HashTable$class.elemEquals(HashTable.scala:364)
	at scala.collection.mutable.HashMap.elemEquals(HashMap.scala:40)
	at scala.collection.mutable.HashTable$class.scala$collection$mutable$HashTable$$findEntry0(HashTable.scala:139)
	at scala.collection.mutable.HashTable$class.findEntry(HashTable.scala:135)
	at scala.collection.mutable.HashMap.findEntry(HashMap.scala:40)
	at scala.collection.mutable.HashMap.get(HashMap.scala:70)
	at org.apache.spark.sql.catalyst.expressions.EquivalentExpressions.addExpr(EquivalentExpressions.scala:56)
	at org.apache.spark.sql.catalyst.expressions.EquivalentExpressions.addExprTree(EquivalentExpressions.scala:97)
	at org.apache.spark.sql.catalyst.expressions.EquivalentExpressions$$anonfun$addExprTree$1.apply(EquivalentExpressions.scala:98)
	at org.apache.spark.sql.catalyst.expressions.EquivalentExpressions$$anonfun$addExprTree$1.apply(EquivalentExpressions.scala:98)
	at scala.collection.immutable.List.foreach(List.scala:392)
	at org.apache.spark.sql.catalyst.expressions.EquivalentExpressions.addExprTree(EquivalentExpressions.scala:98)
	at org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext$$anonfun$subexpressionElimination$1.apply(CodeGenerator.scala:1102)
	at org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext$$anonfun$subexpressionElimination$1.apply(CodeGenerator.scala:1102)
	at scala.collection.immutable.List.foreach(List.scala:392)
	at org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext.subexpressionElimination(CodeGenerator.scala:1102)
	at org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext.generateExpressions(CodeGenerator.scala:1154)
	at org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.createCode(GenerateUnsafeProjection.scala:270)
	at org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.create(GenerateUnsafeProjection.scala:319)
	at org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.generate(GenerateUnsafeProjection.scala:308)
	at org.apache.spark.sql.catalyst.expressions.UnsafeProjection$.create(Projection.scala:181)
	at org.apache.spark.sql.execution.ProjectExec$$anonfun$9.apply(basicPhysicalOperators.scala:71)
	at org.apache.spark.sql.execution.ProjectExec$$anonfun$9.apply(basicPhysicalOperators.scala:70)
	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsWithIndexInternal$1$$anonfun$apply$24.apply(RDD.scala:818)
	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsWithIndexInternal$1$$anonfun$apply$24.apply(RDD.scala:818)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
	at org.apache.spark.scheduler.Task.run(Task.scala:109)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:367)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)

*1. driver side
*2., 3., 4., 5., 6., 7., 8. executor side

Note that most of cases, it looks fine because we will usually call:

which make a canonicalized plan via:

val canonicalizedChildren = children.map(_.canonicalized)

override def children: Seq[Expression] = Nil

I currently faced a bigger question. These canonicalized plans are meant to be made in executor sides? cc @cloud-fan, @viirya , @mgaido91 as well for clarification.

Looks we are already doing so. Now looks we have incomplete FileSourceScanExec on the executors.

Please correct me if I am competely mistaken somewhere.

@mgaido91
Copy link
Contributor

@HyukjinKwon thanks for your great analysis. I agree with you that the proposed fix is more a "workaround" than a real fix for the issue we have here.

The main problem here as you pointed out is that we have a bad (invalid?) FileSourceScanExec on the executors. Probably this has never been an issue as on the executors we accessed only some properties which were correctly populated and we assumed that the other operation would have been performed only on driver side.

I think the cleanest approach (not sure it is entirely feasible) would be to choose one of the following option:

  • check that all exec expression (in this case FileSourceScanExec) are working properly both on driver and executor side;
  • define which operation/attributes can be accessed on executor side too and which only on driver side, document it and enforce it (if feasible).

What do you think?

@HyukjinKwon
Copy link
Member

cc @hvanhovell too

@cloud-fan
Copy link
Contributor

cloud-fan commented Jul 19, 2018

@HyukjinKwon good analysis!

Currently Spark is a little messy about what shall be serialized and sent to executors. Sometimes we just send an entire query tree but only read a few properties of it.

It seems to me it would be better to always do codegen at driver side, to avoid complex expression/plan operations at executor side.(not sure if it's possible, cc @viirya @rednaxelafx @kiszk).

For this particular problem, I think we can just change these vals to lazy val or def in FileSourceScanExec, with a unit test. We should assume that any query plan may be serialized and sent to executors.

@mgaido91
Copy link
Contributor

@cloud-fan I do like the idea of generating the code on driver side. The main issues I see are:

  • In the generated code we reference external objects. We have to send this array of references to all the executors;
  • We should implement a mechanism for identifying which code runs which part of the plan in order for the executors to be able to retrieve it... This is probably feasible but I think we have to be very careful at it.
  • nit: the generated code may be big, may sending it to all executors cause performance issues?

asfgit pushed a commit that referenced this pull request Jul 20, 2018
…g (de)serialized

## What changes were proposed in this pull request?

### What's problem?

In some cases, sub scalar query could throw a NPE, which is caused in execution side.

```
java.lang.NullPointerException
	at org.apache.spark.sql.execution.FileSourceScanExec.<init>(DataSourceScanExec.scala:169)
	at org.apache.spark.sql.execution.FileSourceScanExec.doCanonicalize(DataSourceScanExec.scala:526)
	at org.apache.spark.sql.execution.FileSourceScanExec.doCanonicalize(DataSourceScanExec.scala:159)
	at org.apache.spark.sql.catalyst.plans.QueryPlan.canonicalized$lzycompute(QueryPlan.scala:211)
	at org.apache.spark.sql.catalyst.plans.QueryPlan.canonicalized(QueryPlan.scala:210)
	at org.apache.spark.sql.catalyst.plans.QueryPlan$$anonfun$3.apply(QueryPlan.scala:225)
	at org.apache.spark.sql.catalyst.plans.QueryPlan$$anonfun$3.apply(QueryPlan.scala:225)
	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
	at scala.collection.immutable.List.foreach(List.scala:392)
	at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
	at scala.collection.immutable.List.map(List.scala:296)
	at org.apache.spark.sql.catalyst.plans.QueryPlan.doCanonicalize(QueryPlan.scala:225)
	at org.apache.spark.sql.catalyst.plans.QueryPlan.canonicalized$lzycompute(QueryPlan.scala:211)
	at org.apache.spark.sql.catalyst.plans.QueryPlan.canonicalized(QueryPlan.scala:210)
	at org.apache.spark.sql.catalyst.plans.QueryPlan.sameResult(QueryPlan.scala:258)
	at org.apache.spark.sql.execution.ScalarSubquery.semanticEquals(subquery.scala:58)
	at org.apache.spark.sql.catalyst.expressions.EquivalentExpressions$Expr.equals(EquivalentExpressions.scala:36)
	at scala.collection.mutable.HashTable$class.elemEquals(HashTable.scala:364)
	at scala.collection.mutable.HashMap.elemEquals(HashMap.scala:40)
	at scala.collection.mutable.HashTable$class.scala$collection$mutable$HashTable$$findEntry0(HashTable.scala:139)
	at scala.collection.mutable.HashTable$class.findEntry(HashTable.scala:135)
	at scala.collection.mutable.HashMap.findEntry(HashMap.scala:40)
	at scala.collection.mutable.HashMap.get(HashMap.scala:70)
	at org.apache.spark.sql.catalyst.expressions.EquivalentExpressions.addExpr(EquivalentExpressions.scala:56)
	at org.apache.spark.sql.catalyst.expressions.EquivalentExpressions.addExprTree(EquivalentExpressions.scala:97)
	at org.apache.spark.sql.catalyst.expressions.EquivalentExpressions$$anonfun$addExprTree$1.apply(EquivalentExpressions.scala:98)
	at org.apache.spark.sql.catalyst.expressions.EquivalentExpressions$$anonfun$addExprTree$1.apply(EquivalentExpressions.scala:98)
	at scala.collection.immutable.List.foreach(List.scala:392)
	at org.apache.spark.sql.catalyst.expressions.EquivalentExpressions.addExprTree(EquivalentExpressions.scala:98)
	at org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext$$anonfun$subexpressionElimination$1.apply(CodeGenerator.scala:1102)
	at org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext$$anonfun$subexpressionElimination$1.apply(CodeGenerator.scala:1102)
	at scala.collection.immutable.List.foreach(List.scala:392)
	at org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext.subexpressionElimination(CodeGenerator.scala:1102)
	at org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext.generateExpressions(CodeGenerator.scala:1154)
	at org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.createCode(GenerateUnsafeProjection.scala:270)
	at org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.create(GenerateUnsafeProjection.scala:319)
	at org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.generate(GenerateUnsafeProjection.scala:308)
	at org.apache.spark.sql.catalyst.expressions.UnsafeProjection$.create(Projection.scala:181)
	at org.apache.spark.sql.execution.ProjectExec$$anonfun$9.apply(basicPhysicalOperators.scala:71)
	at org.apache.spark.sql.execution.ProjectExec$$anonfun$9.apply(basicPhysicalOperators.scala:70)
	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsWithIndexInternal$1$$anonfun$apply$24.apply(RDD.scala:818)
	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsWithIndexInternal$1$$anonfun$apply$24.apply(RDD.scala:818)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
	at org.apache.spark.scheduler.Task.run(Task.scala:109)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:367)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
```

### How does this happen?

Here looks what happen now:

1. Sub scalar query was made (for instance `SELECT (SELECT id FROM foo)`).

2. Try to extract some common expressions (via `CodeGenerator.subexpressionElimination`) so that it can generates some common codes and can be reused.

3. During this, seems it extracts some expressions that can be reused (via `EquivalentExpressions.addExprTree`)

  https://github.com/apache/spark/blob/b2deef64f604ddd9502a31105ed47cb63470ec85/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala#L1102

4. During this, if the hash (`EquivalentExpressions.Expr.hashCode`) happened to be the same at `EquivalentExpressions.addExpr` anyhow, `EquivalentExpressions.Expr.equals` is called to identify object in the same hash, which eventually calls `semanticEquals` in `ScalarSubquery`

  https://github.com/apache/spark/blob/087879a77acb37b790c36f8da67355b90719c2dc/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala#L54

  https://github.com/apache/spark/blob/087879a77acb37b790c36f8da67355b90719c2dc/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala#L36

5. `ScalarSubquery`'s `semanticEquals` needs `SubqueryExec`'s `sameResult`

  https://github.com/apache/spark/blob/77a2fc5b521788b406bb32bcc3c637c1d7406e58/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala#L58

6. `SubqueryExec`'s `sameResult` requires a canonicalized plan which calls `FileSourceScanExec`'s `doCanonicalize`

  https://github.com/apache/spark/blob/e008ad175256a3192fdcbd2c4793044d52f46d57/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala#L258

7. In `FileSourceScanExec`'s `doCanonicalize`, `FileSourceScanExec`'s `relation` is required but seems `transient` so it becomes `null`.

  https://github.com/apache/spark/blob/e76b0124fbe463def00b1dffcfd8fd47e04772fe/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala#L527

  https://github.com/apache/spark/blob/e76b0124fbe463def00b1dffcfd8fd47e04772fe/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala#L160

8. NPE is thrown.

\*1. driver side
\*2., 3., 4., 5., 6., 7., 8. executor side

Note that most of cases, it looks fine because we will usually call:

https://github.com/apache/spark/blob/087879a77acb37b790c36f8da67355b90719c2dc/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala#L40

which make a canonicalized plan via:

https://github.com/apache/spark/blob/b045315e5d87b7ea3588436053aaa4d5a7bd103f/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala#L192

https://github.com/apache/spark/blob/77a2fc5b521788b406bb32bcc3c637c1d7406e58/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala#L52

### How to reproduce?

This looks what happened now. I can reproduce this by a bit of messy way:

```diff
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 8d06804..d25fc9a7ba9 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
 -37,7 +37,9  class EquivalentExpressions {
       case _ => false
     }

-    override def hashCode: Int = e.semanticHash()
+    override def hashCode: Int = {
+      1
+    }
   }
```

```scala
spark.range(1).write.mode("overwrite").parquet("/tmp/foo")
spark.read.parquet("/tmp/foo").createOrReplaceTempView("foo")
spark.conf.set("spark.sql.codegen.wholeStage", false)
sql("SELECT (SELECT id FROM foo) == (SELECT id FROM foo)").collect()
```

### How does this PR fix?

- Make all variables that access to `FileSourceScanExec`'s `relation` as `lazy val` so that we avoid NPE. This is a temporary fix.

- Allow `makeCopy` in `SparkPlan` without Spark session too. This looks still able to be accessed within executor side. For instance:

  ```
	at org.apache.spark.sql.execution.SparkPlan.makeCopy(SparkPlan.scala:70)
	at org.apache.spark.sql.execution.SparkPlan.makeCopy(SparkPlan.scala:47)
	at org.apache.spark.sql.catalyst.trees.TreeNode.withNewChildren(TreeNode.scala:233)
	at org.apache.spark.sql.catalyst.plans.QueryPlan.doCanonicalize(QueryPlan.scala:243)
	at org.apache.spark.sql.catalyst.plans.QueryPlan.canonicalized$lzycompute(QueryPlan.scala:211)
	at org.apache.spark.sql.catalyst.plans.QueryPlan.canonicalized(QueryPlan.scala:210)
	at org.apache.spark.sql.catalyst.plans.QueryPlan.sameResult(QueryPlan.scala:258)
	at org.apache.spark.sql.execution.ScalarSubquery.semanticEquals(subquery.scala:58)
	at org.apache.spark.sql.catalyst.expressions.EquivalentExpressions$Expr.equals(EquivalentExpressions.scala:36)
	at scala.collection.mutable.HashTable$class.elemEquals(HashTable.scala:364)
	at scala.collection.mutable.HashMap.elemEquals(HashMap.scala:40)
	at scala.collection.mutable.HashTable$class.scala$collection$mutable$HashTable$$findEntry0(HashTable.scala:139)
	at scala.collection.mutable.HashTable$class.findEntry(HashTable.scala:135)
	at scala.collection.mutable.HashMap.findEntry(HashMap.scala:40)
	at scala.collection.mutable.HashMap.get(HashMap.scala:70)
	at org.apache.spark.sql.catalyst.expressions.EquivalentExpressions.addExpr(EquivalentExpressions.scala:54)
	at org.apache.spark.sql.catalyst.expressions.EquivalentExpressions.addExprTree(EquivalentExpressions.scala:95)
	at org.apache.spark.sql.catalyst.expressions.EquivalentExpressions$$anonfun$addExprTree$1.apply(EquivalentExpressions.scala:96)
	at org.apache.spark.sql.catalyst.expressions.EquivalentExpressions$$anonfun$addExprTree$1.apply(EquivalentExpressions.scala:96)
	at scala.collection.immutable.List.foreach(List.scala:392)
	at org.apache.spark.sql.catalyst.expressions.EquivalentExpressions.addExprTree(EquivalentExpressions.scala:96)
	at org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext$$anonfun$subexpressionElimination$1.apply(CodeGenerator.scala:1102)
	at org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext$$anonfun$subexpressionElimination$1.apply(CodeGenerator.scala:1102)
	at scala.collection.immutable.List.foreach(List.scala:392)
	at org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext.subexpressionElimination(CodeGenerator.scala:1102)
	at org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext.generateExpressions(CodeGenerator.scala:1154)
	at org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.createCode(GenerateUnsafeProjection.scala:270)
	at org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.create(GenerateUnsafeProjection.scala:319)
	at org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.generate(GenerateUnsafeProjection.scala:308)
	at org.apache.spark.sql.catalyst.expressions.UnsafeProjection$.create(Projection.scala:181)
	at org.apache.spark.sql.execution.ProjectExec$$anonfun$9.apply(basicPhysicalOperators.scala:71)
	at org.apache.spark.sql.execution.ProjectExec$$anonfun$9.apply(basicPhysicalOperators.scala:70)
	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsWithIndexInternal$1$$anonfun$apply$24.apply(RDD.scala:818)
	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsWithIndexInternal$1$$anonfun$apply$24.apply(RDD.scala:818)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
	at org.apache.spark.scheduler.Task.run(Task.scala:109)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:367)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
  ```

This PR takes over #20856.

## How was this patch tested?

Manually tested and unit test was added.

Closes #20856

Author: hyukjinkwon <[email protected]>

Closes #21815 from HyukjinKwon/SPARK-23731.

(cherry picked from commit e0b6383)

Signed-off-by: Wenchen Fan <[email protected]>

Signed-off-by: Wenchen Fan <[email protected]>
@asfgit asfgit closed this in e0b6383 Jul 20, 2018
@viirya
Copy link
Member

viirya commented Jul 20, 2018

@HyukjinKwon @cloud-fan Thanks for pinging me, sorry for replying late. Yeah I looked at the final fixing at #21815, it looks good for a fixing at this particular problem.

It seems to me it would be better to always do codegen at driver side, to avoid complex expression/plan operations at executor side.(not sure if it's possible, cc ...).

I do agree that this sounds better. A major part of executor codegen is unsafe codegen classes such as unsafe projection. Most of them if not all are not serializable for now. In order to do codegen at driver side at all, we may need to make them serializable. Is it worth doing this?

Another approach is we generate code in driver side, but do compiling at executor, it sounds more feasible to me. WDYT?

But there is possible that the generating code needs some references e.g. ctx.references.toArray, so it might not completely guaranteed that we can do codegen in driver side and compile at executor.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

9 participants