Details
-
Bug
-
Status: Open
-
Minor
-
Resolution: Unresolved
-
2.2.0
-
None
-
None
Description
When spark SQL is used with Avro-backed HIVE tables, intermittently getting NPE from org.apache.hadoop.hive.serde2.avro.AvroObjectInspectorGenerator.supportedCategories.
Root cause is due to race condition in hive 1.2.1 jar used in Spark SQL .
In HIVE 2.3 this issue has been fixed (HIVE JIRA: https://issues.apache.org/jira/browse/HIVE-16175. ), since Spark is still using Hive 1.2.1 jars we are still getting into a race condition.
One workaround is to run Spark with a single task per executor, however, it will slow down the jobs.
Exception stack trace
13/05/07 09:18:39 WARN scheduler.TaskSetManager: Lost task 18.0 in stage 0.0 (TID 18, aiyhyashu.dxc.com): java.lang.NullPointerException
at org.apache.hadoop.hive.serde2.avro.AvroObjectInspectorGenerator.supportedCategories(AvroObjectInspectorGenerator.java:142)
at org.apache.hadoop.hive.serde2.avro.AvroObjectInspectorGenerator.createObjectInspectorWorker(AvroObjectInspectorGenerator.java:91)
at org.apache.hadoop.hive.serde2.avro.AvroObjectInspectorGenerator.createObjectInspectorWorker(AvroObjectInspectorGenerator.java:104)
at org.apache.hadoop.hive.serde2.avro.AvroObjectInspectorGenerator.createObjectInspectorWorker(AvroObjectInspectorGenerator.java:104)
at org.apache.hadoop.hive.serde2.avro.AvroObjectInspectorGenerator.createObjectInspectorWorker(AvroObjectInspectorGenerator.java:120)
at org.apache.hadoop.hive.serde2.avro.AvroObjectInspectorGenerator.createObjectInspector(AvroObjectInspectorGenerator.java:83)
at org.apache.hadoop.hive.serde2.avro.AvroObjectInspectorGenerator.<init>(AvroObjectInspectorGenerator.java:56)
at org.apache.hadoop.hive.serde2.avro.AvroSerDe.initialize(AvroSerDe.java:124)
at org.apache.spark.sql.hive.HadoopTableReader$$anonfun$5$$anonfun$10.apply(TableReader.scala:251)
at org.apache.spark.sql.hive.HadoopTableReader$$anonfun$5$$anonfun$10.apply(TableReader.scala:239)
at org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$23.apply(RDD.scala:785)
at org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$23.apply(RDD.scala:785)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:283)
at org.apache.spark.rdd.UnionRDD.compute(UnionRDD.scala:105)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:283)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:283)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:283)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:283)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:283)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:283)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:283)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:283)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:283)
at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)
at org.apache.spark.scheduler.Task.run(Task.scala:86)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)
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)
Note: Similar issues are already reported in past but still no solution
https://www.mail-archive.com/user@spark.apache.org/msg61566.html