Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-37196

NPE in org.apache.spark.sql.hive.HiveShim$.toCatalystDecimal(HiveShim.scala:106)

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 3.0.1
    • 3.1.3, 3.0.4, 3.2.1, 3.3.0
    • Input/Output
    • None

    Description

      Still reproducible in Spark 3.0.1.

      How to reproduce:

      SHELL

      > export SPARK_MAJOR_VERSION=3

      SPARK PART 1

      > spark-shell --master local
      SPARK_MAJOR_VERSION is set to 3, using Spark3
      Setting default log level to "WARN".
      To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
      21/10/25 11:43:58 WARN Utils: Service 'SparkUI' could not bind on port 4040. Attempting port 4041.
      21/10/25 11:43:58 WARN Utils: Service 'SparkUI' could not bind on port 4041. Attempting port 4042.
      Spark context Web UI available at <server>:4042
      Spark context available as 'sc' (master = local, app id = local-1635151438558).
      Spark session available as 'spark'.
      Welcome to
      ____ __
      / _/_ ___ ____/ /_
      \ \/ _ \/ _ `/ __/ '/
      /__/ ./_,// //_\ version 3.0.1
      /_/

      Using Scala version 2.12.10 (OpenJDK 64-Bit Server VM, Java 1.8.0_302)
      Type in expressions to have them evaluated.
      Type :help for more information.

      scala> val df = spark.sql("select 'dummy' as name, 1000000000000000000010.7000000000000010 as value")
      df: org.apache.spark.sql.DataFrame = [name: string, value: decimal(38,16)]

      scala> df.write.mode("Overwrite").parquet("test.parquet")

      HIVE

      hive> create external table test_precision(name string, value Decimal(18,6)) STORED AS PARQUET LOCATION 'test';
      OK
      Time taken: 0.067 seconds

      SPARK PART 2

      scala> spark.conf.set("spark.sql.hive.convertMetastoreParquet","false");

      scala> val df_hive = spark.sql("select * from test_precision");
      df_hive: org.apache.spark.sql.DataFrame = [name: string, value: decimal(18,6)]

      scala> df_hive.show;
      21/10/25 12:04:51 ERROR Executor: Exception in task 0.0 in stage 2.0 (TID 2)/ 1]
      java.lang.NullPointerException
      at org.apache.spark.sql.hive.HiveShim$.toCatalystDecimal(HiveShim.scala:106)

      at org.apache.spark.sql.hive.HadoopTableReader$.$anonfun$fillObject$13(TableReader.scala:465)
      at org.apache.spark.sql.hive.HadoopTableReader$.$anonfun$fillObject$13$adapted(TableReader.scala:464)
      at org.apache.spark.sql.hive.HadoopTableReader$.$anonfun$fillObject$18(TableReader.scala:493)
      at scala.collection.Iterator$$anon$10.next(Iterator.scala:459)
      at scala.collection.Iterator$$anon$10.next(Iterator.scala:459)
      at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source)
      at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
      at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:729)
      at org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:340)
      at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:872)
      at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:872)
      at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
      at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
      at org.apache.spark.rdd.RDD.iterator(RDD.scala:313)
      at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
      at org.apache.spark.scheduler.Task.run(Task.scala:127)
      at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:446)
      at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377)
      at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:449)
      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)
      21/10/25 12:04:51 WARN TaskSetManager: Lost task 0.0 in stage 2.0 (TID 2, tklis-kappd0005.dev.df.sbrf.ru, executor driver): java.lang.NullPointerException
      at org.apache.spark.sql.hive.HiveShim$.toCatalystDecimal(HiveShim.scala:106)
      at org.apache.spark.sql.hive.HadoopTableReader$.$anonfun$fillObject$13(TableReader.scala:465)
      at org.apache.spark.sql.hive.HadoopTableReader$.$anonfun$fillObject$13$adapted(TableReader.scala:464)
      at org.apache.spark.sql.hive.HadoopTableReader$.$anonfun$fillObject$18(TableReader.scala:493)
      at scala.collection.Iterator$$anon$10.next(Iterator.scala:459)
      at scala.collection.Iterator$$anon$10.next(Iterator.scala:459)
      at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source)
      at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
      at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:729)
      at org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:340)
      at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:872)
      at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:872)
      at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
      at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
      at org.apache.spark.rdd.RDD.iterator(RDD.scala:313)
      at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
      at org.apache.spark.scheduler.Task.run(Task.scala:127)
      at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:446)
      at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377)
      at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:449)
      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)

      21/10/25 12:04:51 ERROR TaskSetManager: Task 0 in stage 2.0 failed 1 times; aborting job
      org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 2.0 failed 1 times, most recent failure: Lost task 0.0 in stage 2.0 (TID 2, tklis-kappd0005.dev.df.sbrf.ru, executor driver): java.lang.NullPointerException
      at org.apache.spark.sql.hive.HiveShim$.toCatalystDecimal(HiveShim.scala:106)
      at org.apache.spark.sql.hive.HadoopTableReader$.$anonfun$fillObject$13(TableReader.scala:465)
      at org.apache.spark.sql.hive.HadoopTableReader$.$anonfun$fillObject$13$adapted(TableReader.scala:464)
      at org.apache.spark.sql.hive.HadoopTableReader$.$anonfun$fillObject$18(TableReader.scala:493)
      at scala.collection.Iterator$$anon$10.next(Iterator.scala:459)
      at scala.collection.Iterator$$anon$10.next(Iterator.scala:459)
      at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source)
      at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
      at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:729)
      at org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:340)
      at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:872)
      at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:872)
      at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
      at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
      at org.apache.spark.rdd.RDD.iterator(RDD.scala:313)
      at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
      at org.apache.spark.scheduler.Task.run(Task.scala:127)
      at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:446)
      at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377)
      at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:449)
      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)

      Driver stacktrace:
      at org.apache.spark.scheduler.DAGScheduler.failJobAndIndependentStages(DAGScheduler.scala:2059)
      at org.apache.spark.scheduler.DAGScheduler.$anonfun$abortStage$2(DAGScheduler.scala:2008)
      at org.apache.spark.scheduler.DAGScheduler.$anonfun$abortStage$2$adapted(DAGScheduler.scala:2007)
      at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
      at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
      at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
      at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:2007)
      at org.apache.spark.scheduler.DAGScheduler.$anonfun$handleTaskSetFailed$1(DAGScheduler.scala:973)
      at org.apache.spark.scheduler.DAGScheduler.$anonfun$handleTaskSetFailed$1$adapted(DAGScheduler.scala:973)
      at scala.Option.foreach(Option.scala:407)
      at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:973)
      at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:2239)
      at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2188)
      at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2177)
      at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:49)
      at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:775)
      at org.apache.spark.SparkContext.runJob(SparkContext.scala:2099)
      at org.apache.spark.SparkContext.runJob(SparkContext.scala:2120)
      at org.apache.spark.SparkContext.runJob(SparkContext.scala:2139)
      at org.apache.spark.sql.execution.SparkPlan.executeTake(SparkPlan.scala:467)
      at org.apache.spark.sql.execution.SparkPlan.executeTake(SparkPlan.scala:420)
      at org.apache.spark.sql.execution.CollectLimitExec.executeCollect(limit.scala:47)
      at org.apache.spark.sql.Dataset.collectFromPlan(Dataset.scala:3627)
      at org.apache.spark.sql.Dataset.$anonfun$head$1(Dataset.scala:2697)
      at org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:3618)
      at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:100)
      at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:160)
      at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:87)
      at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:764)
      at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
      at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3616)
      at org.apache.spark.sql.Dataset.head(Dataset.scala:2697)
      at org.apache.spark.sql.Dataset.take(Dataset.scala:2904)
      at org.apache.spark.sql.Dataset.getRows(Dataset.scala:300)
      at org.apache.spark.sql.Dataset.showString(Dataset.scala:337)
      at org.apache.spark.sql.Dataset.show(Dataset.scala:824)
      at org.apache.spark.sql.Dataset.show(Dataset.scala:783)
      at org.apache.spark.sql.Dataset.show(Dataset.scala:792)
      ... 47 elided
      Caused by: java.lang.NullPointerException
      at org.apache.spark.sql.hive.HiveShim$.toCatalystDecimal(HiveShim.scala:106)
      at org.apache.spark.sql.hive.HadoopTableReader$.$anonfun$fillObject$13(TableReader.scala:465)
      at org.apache.spark.sql.hive.HadoopTableReader$.$anonfun$fillObject$13$adapted(TableReader.scala:464)
      at org.apache.spark.sql.hive.HadoopTableReader$.$anonfun$fillObject$18(TableReader.scala:493)
      at scala.collection.Iterator$$anon$10.next(Iterator.scala:459)
      at scala.collection.Iterator$$anon$10.next(Iterator.scala:459)
      at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source)
      at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
      at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:729)
      at org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:340)
      at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:872)
      at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:872)
      at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
      at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
      at org.apache.spark.rdd.RDD.iterator(RDD.scala:313)
      at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
      at org.apache.spark.scheduler.Task.run(Task.scala:127)
      at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:446)
      at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377)
      at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:449)
      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)

      Attachments

        Issue Links

          Activity

            People

              angerszhuuu angerszhu
              Yavorski_S Sergey Yavorski
              Votes:
              0 Vote for this issue
              Watchers:
              5 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: