[ 
https://issues.apache.org/jira/browse/SPARK-49460?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Yang Jie resolved SPARK-49460.
------------------------------
    Resolution: Fixed

Fixed by https://github.com/apache/spark/pull/47931

> NPE error in EmptyRelationExec.cleanupResources()
> -------------------------------------------------
>
>                 Key: SPARK-49460
>                 URL: https://issues.apache.org/jira/browse/SPARK-49460
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 4.0.0, 3.5.2, 3.5.3
>            Reporter: Ziqi Liu
>            Assignee: Ziqi Liu
>            Priority: Major
>              Labels: pull-request-available
>
> This bug was introduced in [https://github.com/apache/spark/pull/46830] : 
> *{{cleanupResources}}* might be executed on the executor where {{*logical* is 
> null.}}
>  
> A simple repro
> {code:java}
> spark.sql("create table t1left (a int, b int);")
> spark.sql("insert into t1left values (1, 1), (2,2), (3,3);")
> spark.sql("create table t1right (a int, b int);")
> spark.sql("create table t1empty (a int, b int);")
> spark.sql("insert into t1right values (2,20), (4, 40);")
> spark.sql("""
>   |with leftT as (
>   |  with erp as (
>   |    select
>   |      *
>   |    from
>   |      t1left
>   |      join t1empty on t1left.a = t1empty.a
>   |      join t1right on t1left.a = t1right.a
>   |  )
>   |  SELECT
>   |    CASE
>   |      WHEN COUNT(*) = 0 THEN 4
>   |      ELSE NULL
>   |    END AS a
>   |  FROM
>   |    erp
>   |  HAVING
>   |    COUNT(*) = 0
>   |)
>   |select
>   |  /*+ MERGEJOIN(t1right) */
>   |  *
>   |from
>   |  leftT
>   |  join t1right on leftT.a = t1right.a""").collect() {code}
>  
> error stacktrace:
> {code:java}
> 24/08/29 17:52:08 ERROR TaskSetManager: Task 0 in stage 9.0 failed 1 times; 
> aborting job
> org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in 
> stage 9.0 failed 1 times, most recent failure: Lost task 0.0 in stage 9.0 
> (TID 10) (192.168.3.181 executor driver): java.lang.NullPointerException: 
> Cannot invoke 
> "org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.foreach(scala.Function1)"
>  because the return value of 
> "org.apache.spark.sql.execution.EmptyRelationExec.logical()" is null
>         at 
> org.apache.spark.sql.execution.EmptyRelationExec.cleanupResources(EmptyRelationExec.scala:86)
>         at 
> org.apache.spark.sql.execution.SparkPlan.$anonfun$cleanupResources$1(SparkPlan.scala:571)
>         at 
> org.apache.spark.sql.execution.SparkPlan.$anonfun$cleanupResources$1$adapted(SparkPlan.scala:571)
>         at scala.collection.immutable.Vector.foreach(Vector.scala:2124)
> ....
>         at 
> org.apache.spark.sql.execution.SparkPlan.cleanupResources(SparkPlan.scala:571)
>         at 
> org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage8.processNext(Unknown
>  Source)
>         at 
> org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
>         at 
> org.apache.spark.sql.execution.WholeStageCodegenEvaluatorFactory$WholeStageCodegenPartitionEvaluator$$anon$1.hasNext(WholeStageCodegenEvaluatorFactory.scala:50)
>         at 
> org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:388)
>         at 
> org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:901)
>         at 
> org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:901)
>         at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
>         at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:374)
>         at org.apache.spark.rdd.RDD.iterator(RDD.scala:338)
>         at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:93)
>         at 
> org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:171)
>         at org.apache.spark.scheduler.Task.run(Task.scala:146)
>         at 
> org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$5(Executor.scala:644)
>  {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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

Reply via email to