Skip to content

[SUPPORT]spark-sql MOR query error with org.apache.avro.SchemaParseException: Cannot parse <null> schema #9016

@zyclove

Description

@zyclove

Describe the problem you faced
Write new data with config
hoodie.avro.schema.validate=true;
hoodie.datasource.write.schema.allow.auto.evolution.column.drop=true;
hoodie.datasource.write.reconcile.schema=true;

query with config :
set hoodie.schema.on.read.enable=true;
set hoodie.datasource.write.reconcile.schema=true;
set hoodie.avro.schema.validate=true;

The new data is written normally, but query is error.

Check the schema file is with the latest_schema.
extraMetadata : {
schema : "{"type":"record","name":"ods_api_test_case_record","namespace":"hoodie.ods_api_test_case","fields":[{"name":"_hoodie_commit_time","type":["null","string"],"doc":"","default":null},{"name":"_hoodie_commit_seqno","type":["null","string"],"doc":"","default":null},{"name":"_hoodie_record_key","type":["null","string"],"doc":"","default":null},{"name":"_hoodie_partition_path","type":["null","string"],"doc":"","default":null},{"name":"_hoodie_file_name","type":["null","string"],"doc":"","default":null},{"name":"gmt_create","type":["null","long"],"default":null},{"name":"owner","type":["null","string"],"default":null},{"name":"last_status","type":["null","long"],"default":null},{"name":"description","type":["null","string"],"default":null},{"name":"offset","type":["null","long"],"default":null},{"name":"last_run","type":["null","long"],"default":null},{"name":"env","type":["null","string"],"default":null},{"name":"gmt_modified","type":["null","long"],"default":null},{"name":"created_by","type":["null","string"],"default":null},{"name":"run_ignore_fail","type":["null","long"],"default":null},{"name":"test_type","type":["null","long"],"default":null},{"name":"tags","type":["null","string"],"default":null},{"name":"dt","type":["null","string"],"default":null},{"name":"ct","type":["null","long"],"default":null},{"name":"partition","type":["null","string"],"default":null},{"name":"name","type":["null","string"],"default":null},{"name":"only_me","type":["null","long"],"default":null},{"name":"id","type":["null","long"],"default":null},{"name":"region","type":["null","string"],"default":null},{"name":"status","type":["null","long"],"default":null}]}",
latest_schema: "{"max_column_id":24,"version_id":20230619064139757,"type":"record","fields":[{"id":0,"name":"_hoodie_commit_time","optional":true,"type":"string","doc":""},{"id":1,"name":"_hoodie_commit_seqno","optional":true,"type":"string","doc":""},{"id":2,"name":"_hoodie_record_key","optional":true,"type":"string","doc":""},{"id":3,"name":"_hoodie_partition_path","optional":true,"type":"string","doc":""},{"id":4,"name":"_hoodie_file_name","optional":true,"type":"string","doc":""},{"id":5,"name":"gmt_create","optional":true,"type":"long"},{"id":6,"name":"owner","optional":true,"type":"string"},{"id":7,"name":"last_status","optional":true,"type":"long"},{"id":8,"name":"description","optional":true,"type":"string"},{"id":9,"name":"offset","optional":true,"type":"long"},{"id":10,"name":"last_run","optional":true,"type":"long"},{"id":11,"name":"env","optional":true,"type":"string"},{"id":12,"name":"gmt_modified","optional":true,"type":"long"},{"id":13,"name":"created_by","optional":true,"type":"string"},{"id":14,"name":"run_ignore_fail","optional":true,"type":"long"},{"id":15,"name":"test_type","optional":true,"type":"long"},{"id":16,"name":"tags","optional":true,"type":"string"},{"id":17,"name":"dt","optional":true,"type":"string"},{"id":18,"name":"ct","optional":true,"type":"long"},{"id":19,"name":"partition","optional":true,"type":"string"},{"id":20,"name":"name","optional":true,"type":"string"},{"id":21,"name":"only_me","optional":true,"type":"long"},{"id":22,"name":"id","optional":true,"type":"long"},{"id":23,"name":"region","optional":true,"type":"string"},{"id":24,"name":"status","optional":true,"type":"long"}]}"
}`

To Reproduce

Review code and debug.
image

org.apache.hudi.internal.schema.utils.InternalSchemaUtils.searchSchema
image

image

Expected behavior

A clear and concise description of what you expected to happen.

Environment Description

  • Hudi version :0.13.1

  • Spark version :3.2.1

  • Hive version :3.2.1

  • Hadoop version :3.2.2

  • Storage (HDFS/S3/GCS..) :s3

  • Running on Docker? (yes/no) :no

Additional context

Add any other context about the problem here.

Stacktrace

Add the stacktrace of the error.
Caused by: org.apache.avro.SchemaParseException: Cannot parse schema
at org.apache.avro.Schema.parse(Schema.java:1633)
at org.apache.avro.Schema$Parser.parse(Schema.java:1430)
at org.apache.avro.Schema$Parser.parse(Schema.java:1418)
at org.apache.hudi.common.util.InternalSchemaCache.getInternalSchemaByVersionId(InternalSchemaCache.java:225)
at org.apache.spark.sql.execution.datasources.parquet.Spark32PlusHoodieParquetFileFormat.$anonfun$buildReaderWithPartitionValues$2(Spark32PlusHoodieParquetFileFormat.scala:159)
at org.apache.hudi.HoodieDataSourceHelper$.$anonfun$buildHoodieParquetReader$1(HoodieDataSourceHelper.scala:71)
at org.apache.hudi.HoodieBaseRelation.$anonfun$createBaseFileReader$2(HoodieBaseRelation.scala:569)
at org.apache.hudi.HoodieBaseRelation$BaseFileReader.apply(HoodieBaseRelation.scala:637)
at org.apache.hudi.HoodieMergeOnReadRDD.compute(HoodieMergeOnReadRDD.scala:87)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
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:133)
at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:506)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1474)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:509)
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)

Metadata

Metadata

Assignees

No one assigned

    Labels

    area:schemaSchema evolution and data typespriority:highSignificant impact; potential bugs

    Type

    No type

    Projects

    Status

    ✅ Done

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions