Sagar Sumit created HUDI-7966:
---------------------------------

             Summary: NPE from 
AvroSchemaUtils.createNewSchemaFromFieldsWithReference
                 Key: HUDI-7966
                 URL: https://issues.apache.org/jira/browse/HUDI-7966
             Project: Apache Hudi
          Issue Type: Task
            Reporter: Sagar Sumit
             Fix For: 1.0.0


Running 
[long-running|https://github.com/apache/hudi/blob/dbfe8b23c0b4f160b26379053873cfc2a46acef4/docker/demo/config/test-suite/spark-long-running-non-partitioned.yaml]
 deltastreamer with following properties: 
[https://github.com/apache/hudi/blob/dbfe8b23c0b4f160b26379053873cfc2a46acef4/docker/demo/config/test-suite/test-nonpartitioned.properties]

The job throws NPE during validation phase:
{code:java}
Caused by: org.apache.spark.SparkException: Job aborted due to stage failure: 
Task 0 in stage 69.0 failed 4 times, most recent failure: Lost task 0.3 in 
stage 69.0 (TID 345) (10.0.103.207 executor 1): java.lang.NullPointerException  
at org.apache.avro.JsonProperties$2$1$1.<init>(JsonProperties.java:175)  at 
org.apache.avro.JsonProperties$2$1.iterator(JsonProperties.java:174)  at 
org.apache.avro.JsonProperties.getObjectProps(JsonProperties.java:305)  at 
org.apache.hudi.avro.AvroSchemaUtils.createNewSchemaFromFieldsWithReference(AvroSchemaUtils.java:306)
  at 
org.apache.hudi.avro.AvroSchemaUtils.appendFieldsToSchemaBase(AvroSchemaUtils.java:293)
  at 
org.apache.hudi.avro.AvroSchemaUtils.appendFieldsToSchemaDedupNested(AvroSchemaUtils.java:245)
  at 
org.apache.hudi.common.table.read.HoodieFileGroupReaderSchemaHandler.generateRequiredSchema(HoodieFileGroupReaderSchemaHandler.java:146)
  at 
org.apache.hudi.common.table.read.HoodieFileGroupReaderSchemaHandler.prepareRequiredSchema(HoodieFileGroupReaderSchemaHandler.java:150)
  at 
org.apache.hudi.common.table.read.HoodieFileGroupReaderSchemaHandler.<init>(HoodieFileGroupReaderSchemaHandler.java:84)
  at 
org.apache.hudi.common.table.read.HoodieFileGroupReader.<init>(HoodieFileGroupReader.java:113)
  at 
org.apache.spark.sql.execution.datasources.parquet.HoodieFileGroupReaderBasedParquetFileFormat.$anonfun$buildReaderWithPartitionValues$3(HoodieFileGroupReaderBasedParquetFileFormat.scala:170)
  at 
org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.org$apache$spark$sql$execution$datasources$FileScanRDD$$anon$$readCurrentFile(FileScanRDD.scala:209)
  at 
org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:270)
  at 
org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:116)
  at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)  at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.hashAgg_doAggregateWithoutKey_0$(Unknown
 Source)  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:760)
  at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)  at 
org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:140)
  at 
org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59)
  at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99) 
 at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52)  
at org.apache.spark.scheduler.Task.run(Task.scala:136)  at 
org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:548)
  at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1504)  at 
org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:551)  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:750) {code}
It seems like the code assumes that all schema must have properties, which may 
not necessaily be true.



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

Reply via email to