3806887afb
## What changes were proposed in this pull request? When query returns zero rows, the HiveUDAFFunction throws NPE ## CASE 1: create table abc(a int) select histogram_numeric(a,2) from abc // NPE ``` Job aborted due to stage failure: Task 0 in stage 1.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1.0 (TID 0, localhost, executor driver): java.lang.NullPointerException at org.apache.spark.sql.hive.HiveUDAFFunction.eval(hiveUDFs.scala:471) at org.apache.spark.sql.hive.HiveUDAFFunction.eval(hiveUDFs.scala:315) at org.apache.spark.sql.catalyst.expressions.aggregate.TypedImperativeAggregate.eval(interfaces.scala:543) at org.apache.spark.sql.execution.aggregate.AggregationIterator.$anonfun$generateResultProjection$5(AggregationIterator.scala:231) at org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.outputForEmptyGroupingKeyWithoutInput(ObjectAggregationIterator.scala:97) at org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec.$anonfun$doExecute$2(ObjectHashAggregateExec.scala:132) at org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec.$anonfun$doExecute$2$adapted(ObjectHashAggregateExec.scala:107) at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsWithIndexInternal$2(RDD.scala:839) at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsWithIndexInternal$2$adapted(RDD.scala:839) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:327) at org.apache.spark.rdd.RDD.iterator(RDD.scala:291) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:327) at org.apache.spark.rdd.RDD.iterator(RDD.scala:291) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:327) at org.apache.spark.rdd.RDD.iterator(RDD.scala:291) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90) at org.apache.spark.scheduler.Task.run(Task.scala:122) at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:425) at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1350) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:428) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) ``` ## CASE 2: create table abc(a int) insert into abc values (1) select histogram_numeric(a,2) from abc where a=3 // NPE ``` Job aborted due to stage failure: Task 0 in stage 4.0 failed 1 times, most recent failure: Lost task 0.0 in stage 4.0 (TID 5, localhost, executor driver): java.lang.NullPointerException at org.apache.spark.sql.hive.HiveUDAFFunction.serialize(hiveUDFs.scala:477) at org.apache.spark.sql.hive.HiveUDAFFunction.serialize(hiveUDFs.scala:315) at org.apache.spark.sql.catalyst.expressions.aggregate.TypedImperativeAggregate.serializeAggregateBufferInPlace(interfaces.scala:570) at org.apache.spark.sql.execution.aggregate.AggregationIterator.$anonfun$generateResultProjection$6(AggregationIterator.scala:254) at org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.outputForEmptyGroupingKeyWithoutInput(ObjectAggregationIterator.scala:97) at org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec.$anonfun$doExecute$2(ObjectHashAggregateExec.scala:132) at org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec.$anonfun$doExecute$2$adapted(ObjectHashAggregateExec.scala:107) at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsWithIndexInternal$2(RDD.scala:839) at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsWithIndexInternal$2$adapted(RDD.scala:839) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:327) at org.apache.spark.rdd.RDD.iterator(RDD.scala:291) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:327) at org.apache.spark.rdd.RDD.iterator(RDD.scala:291) at org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:94) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52) at org.apache.spark.scheduler.Task.run(Task.scala:122) at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:425) at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1350) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:428) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) ``` Hence add a check not avoid NPE ## How was this patch tested? Added new UT case Closes #24762 from ajithme/hiveudaf. Authored-by: Ajith <ajith2489@gmail.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> |
||
---|---|---|
.. | ||
benchmarks | ||
compatibility/src/test/scala/org/apache/spark/sql/hive/execution | ||
src | ||
pom.xml |