apache / hudi

Upserts, Deletes And Incremental Processing on Big Data.
https://hudi.apache.org/
Apache License 2.0
5.21k stars 2.38k forks source link

org.apache.hudi.exception.HoodieException: org.apache.avro.AvroTypeException: Cannot encode decimal with precision 14 as max precision 13 #11335

Open KSubramanyaH opened 1 month ago

KSubramanyaH commented 1 month ago

*Hell All,

We have a issue while doing clustering for one of our job:

Caused by: org.apache.hudi.exception.HoodieException: org.apache.hudi.exception.HoodieException: org.apache.avro.AvroTypeException: Cannot encode decimal with precision 14 as max precision 13 at org.apache.hudi.table.action.commit.HoodieMergeHelper.runMerge(HoodieMergeHelper.java:149) at org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor.handleUpdateInternal(BaseSparkCommitActionExecutor.java:387) at org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor.handleUpdate(BaseSparkCommitActionExecutor.java:369) at org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor.handleUpsertPartition(BaseSparkCommitActionExecutor.java:335) ... 28 more Caused by: org.apache.hudi.exception.HoodieException: org.apache.avro.AvroTypeException: Cannot encode decimal with precision 14 as max precision 13 at org.apache.hudi.common.util.queue.SimpleExecutor.execute(SimpleExecutor.java:75) at org.apache.hudi.table.action.commit.HoodieMergeHelper.runMerge(HoodieMergeHelper.java:147) ... 31 more Caused by: org.apache.avro.AvroTypeException: Cannot encode decimal with precision 14 as max precision 13 at org.apache.avro.Conversions$DecimalConversion.validate(Conversions.java:140) at org.apache.avro.Conversions$DecimalConversion.toFixed(Conversions.java:104) at org.apache.hudi.avro.HoodieAvroUtils.rewritePrimaryTypeWithDiffSchemaType(HoodieAvroUtils.java:1077) at org.apache.hudi.avro.HoodieAvroUtils.rewritePrimaryType(HoodieAvroUtils.java:1001) at org.apache.hudi.avro.HoodieAvroUtils.rewriteRecordWithNewSchemaInternal(HoodieAvroUtils.java:946) at org.apache.hudi.avro.HoodieAvroUtils.rewriteRecordWithNewSchema(HoodieAvroUtils.java:873) at org.apache.hudi.avro.HoodieAvroUtils.rewriteRecordWithNewSchemaInternal(HoodieAvroUtils.java:944) at org.apache.hudi.avro.HoodieAvroUtils.rewriteRecordWithNewSchema(HoodieAvroUtils.java:873) at org.apache.hudi.avro.HoodieAvroUtils.rewriteRecordWithNewSchemaInternal(HoodieAvroUtils.java:894) at org.apache.hudi.avro.HoodieAvroUtils.rewriteRecordWithNewSchema(HoodieAvroUtils.java:873) at org.apache.hudi.avro.HoodieAvroUtils.rewriteRecordWithNewSchema(HoodieAvroUtils.java:843) at org.apache.hudi.common.model.HoodieAvroIndexedRecord.rewriteRecordWithNewSchema(HoodieAvroIndexedRecord.java:123) at org.apache.hudi.common.model.HoodieRecord.rewriteRecordWithNewSchema(HoodieRecord.java:382) at org.apache.hudi.table.action.commit.HoodieMergeHelper.lambda$runMerge$0(HoodieMergeHelper.java:136) at org.apache.hudi.common.util.queue.SimpleExecutor.execute(SimpleExecutor.java:68) ... 32 more

There are some fields , which are having data type decimal(13,4) and incoming data is not having more precision than prescribed one . still we are getting above issue. max we get (10,4) precision values.

is there any solution solve this . This issue occurred after we start to use clustering for our tables

Hudi configs: {'hoodie.clustering.plan.strategy.target.file.max.bytes': '525829120', 'hoodie.datasource.hive_sync.table': '', 'hoodie.datasource.write.reconcile.schema': 'true', 'hoodie.index.type': 'SIMPLE', 'hoodie.clean.automatic': 'true', 'hoodie.write.markers.type': 'direct', 'hoodie.datasource.hive_sync.partition_extractor_class': 'org.apache.hudi.hive.MultiPartKeysValueExtractor', 'hoodie.schema.on.read.enable': 'true', 'hoodie.datasource.hive_sync.mode': 'hms', 'hoodie.datasource.write.drop.partition.columns': 'true', 'hoodie.datasource.write.recordkey.field': '_partition_year,_partition_month,_partition_day,id', 'hoodie.datasource.hive_sync.support_timestamp': 'true', 'hoodie.metadata.enable': 'false', 'hoodie.datasource.write.keygenerator.class': 'org.apache.hudi.keygen.ComplexKeyGenerator', 'hoodie.datasource.write.table.type': 'COPY_ON_WRITE', 'hoodie.datasource.write.hive_style_partitioning': 'true', 'hoodie.clustering.plan.strategy.max.bytes.per.group': '525829120', 'hoodie.parquet.small.file.limit': '0', 'hoodie.datasource.hive_sync.enable': 'true', 'hoodie.cleaner.policy': 'KEEP_LATEST_FILE_VERSIONS', 'hoodie.clustering.inline.max.commits': '4', 'hoodie.clustering.inline': 'true', 'hoodie.clustering.plan.strategy.max.num.groups': '200', 'hoodie.datasource.write.schema.allow.auto.evolution.column.drop': 'false', 'hoodie.datasource.hive_sync.use_jdbc': 'false', 'hoodie.datasource.write.partitionpath.field': '_partition_year,_partition_month,_partition_day', 'hoodie.cleaner.fileversions.retained': '1', 'hoodie.table.name': '', 'hoodie.clustering.plan.strategy.small.file.limit': '512000', 'hoodie.datasource.write.payload.class': 'org.apache.hudi.common.model.DefaultHoodieRecordPayload', 'hoodie.datasource.write.precombine.field': '_acq_change_seq', 'hoodie.datasource.hive_sync.database': '***', 'hoodie.datasource.write.operation': 'upsert'}

Hudi version : 0.14.0

Also i tried , 'spark.sql.storeAssignmentPolicy'='legacy'

Below is spark session spark = ( SparkSession.builder.appName("test") .config('spark.hadoop.hive.metastore.client.factory.class', 'com.amazonaws.glue.catalog.metastore.AWSGlueDataCatalogHiveClientFactory') .config('spark.serializer', 'org.apache.spark.serializer.KryoSerializer') .config('spark.sql.extensions', 'org.apache.spark.sql.hudi.HoodieSparkSessionExtension') .config('spark.hadoop.spark.sql.legacy.parquet.nanosAsLong', 'false') .config('spark.hadoop.spark.sql.parquet.binaryAsString', 'false') .config('spark.hadoop.spark.sql.parquet.int96AsTimestamp', 'true') .config('spark.hadoop.spark.sql.caseSensitive', 'false') .config('spark.sql.parquet.datetimeRebaseModeInWrite','CORRECTED') .config('spark.sql.parquet.datetimeRebaseModeInRead','CORRECTED') .config('spark.sql.parquet.int96RebaseModeInWrite', 'CORRECTED') .config('spark.sql.storeAssignmentPolicy', 'legacy') .config("spark.executor.extraJavaOptions", "-XX:+UseG1GC -XX:+UnlockDiagnosticVMOptions -XX:+G1SummarizeConcMark -XX:InitiatingHeapOccupancyPercent=35 -verbose:gc -XX:+PrintGCDetails -XX:+PrintGCDateStamps -XX:OnOutOfMemoryError='kill -9 %p'") .config("spark.driver.extraJavaOptions", "-XX:+UseG1GC -XX:+UnlockDiagnosticVMOptions -XX:+G1SummarizeConcMark -XX:InitiatingHeapOccupancyPercent=35 -verbose:gc -XX:+PrintGCDetails -XX:+PrintGCDateStamps -XX:OnOutOfMemoryError='kill -9 %p'") .config('spark.sql.hive.convertMetastoreParquet', 'false') .config('conf spark.kryoserializer.buffer.max', '2040M') .config('fs.s3.maxRetries', '5') .enableHiveSupport() .getOrCreate() )

Could anybody help here . It is a production error

ad1happy2go commented 1 month ago

@KSubramanyaH Can you try to print schema for the table and check the same. Are you sure there are no changes in the incoming data for this.

Can you try to create a reproducible script for the same.

KSubramanyaH commented 1 month ago

Hello @ad1happy2go ...

below is printschema just before running Hudi load

root |-- op: string (nullable = true) |-- _acq_load_ts: string (nullable = true) |-- id: decimal(20,0) (nullable = true) |-- dataset_id: decimal(20,0) (nullable = true) |-- deleted_id: decimal(20,0) (nullable = true) |-- user_modified_at: timestamp (nullable = true) |-- created_at: timestamp (nullable = true) |-- modified_at: timestamp (nullable = true) |-- client_id: string (nullable = true) |-- opt_lock: long (nullable = true) |-- parent_id: decimal(20,0) (nullable = true) |-- name: string (nullable = true) |-- target_amount: decimal(13,4) (nullable = true) |-- overwritten_target_amount: decimal(13,4) (nullable = true) |-- calculated_spent_amount: decimal(13,4) (nullable = true) |-- filter_id: string (nullable = true) |-- recurring: byte (nullable = true) |-- frequency: string (nullable = true) |-- txn_ids: string (nullable = true) |-- excluded_txn_ids: string (nullable = true) |-- rollover_amount: decimal(13,4) (nullable = true) |-- reset_rollover_amount: decimal(13,4) (nullable = true) |-- cumulative_rollover_amount: decimal(13,4) (nullable = true) |-- rollover_type: string (nullable = true) |-- _acq_stream_position: string (nullable = true) |-- _acq_change_seq: string (nullable = true) |-- _job_run_id: string (nullable = true) |-- _s3_intermediate_location: string (nullable = true) |-- _partition_year: string (nullable = true) |-- _partition_month: string (nullable = true)

One more note : We are loading 2 hudi tables with same schema . 1 table is just load all cdc data with bulk insert and is working fine 2 table load is upsert and delete . But it is failing with decimal encode issue

I am very much sure that incoming data has not crossed prescribed schema (.i.e decimal(13,4)) .. Maximum that we receive decimal(10,4)

KSubramanyaH commented 1 month ago

Hi .. Any update on this , It is impacting our production tables

danny0405 commented 1 month ago

I see quite a few columns are declared as precision 13, do we have schema evolution on them?

|-- target_amount: decimal(13,4) (nullable = true)
|-- overwritten_target_amount: decimal(13,4) (nullable = true)
|-- calculated_spent_amount: decimal(13,4) (nullable = true)
KSubramanyaH commented 1 month ago

@danny0405 No, It has not evolved . But schema evolution is not required as incoming data is compatible with 13,4

We are loading 2 hudi tables with same schema . 1 table is just load all cdc data with bulk insert and is working fine 2 table load is upsert and delete . But it is failing with decimal encode issue

I am very much sure that incoming data has not crossed prescribed schema (.i.e decimal(13,4)) .. Maximum that we receive decimal(10,4)

danny0405 commented 4 weeks ago

Cannot encode decimal with precision 14 as max precision 13

But from the error msg there looks like a decimal with precision 14.