parisni opened a new issue, #9005:
URL: https://github.com/apache/hudi/issues/9005
hudi 0.13.x
spark 3.x.y
When schema evolution occurs with a non primitive type (string, array....)
then the new merger fails with NPE. Other types such int, timestamp won't fail.
```python
tableName = 'test_hudi_merger'
basePath = "/tmp/{tableName}".format(tableName=tableName)
df =spark.sql("select '1' as event_id, '2' as ts, '3' as version, 'foo' as
event_date")
hudi_options = {
"hoodie.table.name": tableName,
"hoodie.datasource.write.recordkey.field": "event_id",
"hoodie.datasource.write.partitionpath.field": "version,event_date",
"hoodie.datasource.write.table.name": tableName,
"hoodie.datasource.write.operation": "insert",
"hoodie.datasource.write.precombine.field": "ts",
"hoodie.datasource.write.keygenerator.class":
"org.apache.hudi.keygen.ComplexKeyGenerator",
"hoodie.datasource.write.hive_style_partitioning": "true",
"hoodie.datasource.hive_sync.enable": "false",
"hoodie.metadata.enable": "false",
# NEW MERGER
"hoodie.datasource.write.record.merger.impls":
"org.apache.hudi.HoodieSparkRecordMerger",
}
(df.write.format("hudi").options(**hudi_options).mode("overwrite").save(basePath))
# NOW ADD A STRING COLUMN
df =spark.sql("select '2' as event_id, '2' as ts, '3' as version, 'foo' as
event_date, 'bar' as add_col")
hudi_options = {
"hoodie.table.name": tableName,
"hoodie.datasource.write.recordkey.field": "event_id",
"hoodie.datasource.write.partitionpath.field": "version,event_date",
"hoodie.datasource.write.table.name": tableName,
"hoodie.datasource.write.operation": "insert",
"hoodie.datasource.write.precombine.field": "ts",
"hoodie.datasource.write.keygenerator.class":
"org.apache.hudi.keygen.ComplexKeyGenerator",
"hoodie.datasource.write.hive_style_partitioning": "true",
"hoodie.datasource.hive_sync.enable": "false",
"hoodie.metadata.enable": "false",
# NEW MERGER
"hoodie.datasource.write.record.merger.impls":
"org.apache.hudi.HoodieSparkRecordMerger",
}
(df.write.format("hudi").options(**hudi_options).mode("append").save(basePath))
```
Fails with:
```
: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0
in stage 140.0 failed 1 times, most recent failure: Lost task 0.0 in stage
140.0 (TID 138) (192.168.1.18 executor driver):
org.apache.hudi.exception.HoodieUpsertException: Error upserting bucketType
UPDATE for partition :0
at
org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor.handleUpsertPartition(BaseSparkCommitActionExecutor.java:336)
at
org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor.handleInsertPartition(BaseSparkCommitActionExecutor.java:342)
at
org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor.lambda$mapPartitionsAsRDD$a3ab3c4$1(BaseSparkCommitActionExecutor.java:253)
at
org.apache.spark.api.java.JavaRDDLike.$anonfun$mapPartitionsWithIndex$1(JavaRDDLike.scala:102)
at
org.apache.spark.api.java.JavaRDDLike.$anonfun$mapPartitionsWithIndex$1$adapted(JavaRDDLike.scala:102)
at
org.apache.spark.rdd.RDD.$anonfun$mapPartitionsWithIndex$2(RDD.scala:915)
at
org.apache.spark.rdd.RDD.$anonfun$mapPartitionsWithIndex$2$adapted(RDD.scala:915)
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.$anonfun$getOrCompute$1(RDD.scala:386)
at
org.apache.spark.storage.BlockManager.$anonfun$doPutIterator$1(BlockManager.scala:1498)
at
org.apache.spark.storage.BlockManager.org$apache$spark$storage$BlockManager$$doPut(BlockManager.scala:1408)
at
org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:1472)
at
org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:1295)
at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:384)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:335)
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.scheduler.ResultTask.runTask(ResultTask.scala:90)
at org.apache.spark.scheduler.Task.run(Task.scala:131)
at
org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:506)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1462)
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:750)
Caused by: org.apache.hudi.exception.HoodieException:
org.apache.hudi.exception.HoodieException: java.lang.NullPointerException
at
org.apache.hudi.table.action.commit.HoodieMergeHelper.runMerge(HoodieMergeHelper.java:156)
at
org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor.handleUpdateInternal(BaseSparkCommitActionExecutor.java:372)
at
org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor.handleUpdate(BaseSparkCommitActionExecutor.java:363)
at
org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor.handleUpsertPartition(BaseSparkCommitActionExecutor.java:329)
... 29 more
Caused by: org.apache.hudi.exception.HoodieException:
java.lang.NullPointerException
at
org.apache.hudi.common.util.queue.SimpleExecutor.execute(SimpleExecutor.java:73)
at
org.apache.hudi.table.action.commit.HoodieMergeHelper.runMerge(HoodieMergeHelper.java:154)
... 32 more
Caused by: java.lang.NullPointerException
at
org.apache.spark.sql.catalyst.expressions.codegen.UnsafeWriter.write(UnsafeWriter.java:110)
at
org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.writeFields_0_1$(Unknown
Source)
at
org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown
Source)
at
org.apache.spark.sql.HoodieCatalystExpressionUtils$$anon$1.apply(HoodieCatalystExpressionUtils.scala:105)
at
org.apache.spark.sql.HoodieInternalRowUtils$.$anonfun$genUnsafeRowWriter$1(HoodieInternalRowUtils.scala:136)
at
org.apache.hudi.common.model.HoodieSparkRecord.rewriteRecordWithNewSchema(HoodieSparkRecord.java:211)
at
org.apache.hudi.common.model.HoodieRecord.rewriteRecordWithNewSchema(HoodieRecord.java:369)
at
org.apache.hudi.table.action.commit.HoodieMergeHelper.lambda$runMerge$1(HoodieMergeHelper.java:143)
at
org.apache.hudi.common.util.queue.SimpleExecutor.execute(SimpleExecutor.java:66)
... 33 more
```
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]