parisni opened a new issue, #8920:
URL: https://github.com/apache/hudi/issues/8920
hudi 0.13.0 / 0.13.1
When the schema of an array change (based on nullable support or not), then
the insert/upsert operation fails while trying to merge an existing parquet
file.
In the below example we first write a
```
root
|-- event_id: string (nullable = false)
|-- ts: string (nullable = false)
|-- array_col: array (nullable = false)
| |-- element: long (containsNull = false)
|-- version: string (nullable = false)
|-- event_date: string (nullable = false)
```
and then:
```
root
|-- event_id: string (nullable = false)
|-- ts: string (nullable = false)
|-- array_col: array (nullable = false)
| |-- element: long (containsNull = true)
|-- version: string (nullable = false)
|-- event_date: string (nullable = false)
````
```python
tableName = 'test_hudi_merger'
basePath = "/tmp/{tableName}".format(tableName=tableName)
df =spark.sql("select '1' as event_id, '2' as ts, array(1L,2L) as array_col,
'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))
df =spark.sql("select '2' as event_id, '2' as ts, cast(array() as
array<long>) as array_col, '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("append").save(basePath))
```
This raises an error while trying to set a value to an unsafeArray (which is
unsupported):
```
: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0
in stage 90.0 failed 1 times, most recent failure: Lost task 0.0 in stage 90.0
(TID 90) (192.168.1.11 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.UnsupportedOperationException
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.UnsupportedOperationException
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.UnsupportedOperationException
at
org.apache.spark.sql.catalyst.expressions.UnsafeArrayData.update(UnsafeArrayData.java:273)
at
org.apache.spark.sql.HoodieInternalRowUtils$ArrayDataUpdater.set(HoodieInternalRowUtils.scala:421)
at
org.apache.spark.sql.HoodieInternalRowUtils$.$anonfun$newWriterRenaming$1(HoodieInternalRowUtils.scala:206)
at
org.apache.spark.sql.HoodieInternalRowUtils$.$anonfun$newWriterRenaming$1$adapted(HoodieInternalRowUtils.scala:206)
at
org.apache.spark.sql.HoodieInternalRowUtils$.$anonfun$newWriterRenaming$4(HoodieInternalRowUtils.scala:247)
at
org.apache.spark.sql.HoodieInternalRowUtils$.$anonfun$newWriterRenaming$4$adapted(HoodieInternalRowUtils.scala:229)
at
org.apache.spark.sql.HoodieInternalRowUtils$.$anonfun$genUnsafeStructWriter$3(HoodieInternalRowUtils.scala:194)
at
org.apache.spark.sql.HoodieInternalRowUtils$.$anonfun$genUnsafeStructWriter$3$adapted(HoodieInternalRowUtils.scala:181)
at
org.apache.spark.sql.HoodieInternalRowUtils$.$anonfun$newWriterRenaming$3(HoodieInternalRowUtils.scala:220)
at
org.apache.spark.sql.HoodieInternalRowUtils$.$anonfun$newWriterRenaming$3$adapted(HoodieInternalRowUtils.scala:214)
at
org.apache.spark.sql.HoodieInternalRowUtils$.$anonfun$genUnsafeRowWriter$1(HoodieInternalRowUtils.scala:135)
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
Driver stacktrace:
at
org.apache.spark.scheduler.DAGScheduler.failJobAndIndependentStages(DAGScheduler.scala:2454)
at
org.apache.spark.scheduler.DAGScheduler.$anonfun$abortStage$2(DAGScheduler.scala:2403)
at
org.apache.spark.scheduler.DAGScheduler.$anonfun$abortStage$2$adapted(DAGScheduler.scala:2402)
at
scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
at
scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
at
org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:2402)
at
org.apache.spark.scheduler.DAGScheduler.$anonfun$handleTaskSetFailed$1(DAGScheduler.scala:1160)
at
org.apache.spark.scheduler.DAGScheduler.$anonfun$handleTaskSetFailed$1$adapted(DAGScheduler.scala:1160)
at scala.Option.foreach(Option.scala:407)
at
org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:1160)
at
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:2642)
at
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2584)
at
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2573)
at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:49)
at
org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:938)
at org.apache.spark.SparkContext.runJob(SparkContext.scala:2214)
at org.apache.spark.SparkContext.runJob(SparkContext.scala:2235)
at org.apache.spark.SparkContext.runJob(SparkContext.scala:2254)
at org.apache.spark.SparkContext.runJob(SparkContext.scala:2279)
at org.apache.spark.rdd.RDD.count(RDD.scala:1253)
at
org.apache.hudi.HoodieSparkSqlWriter$.commitAndPerformPostOperations(HoodieSparkSqlWriter.scala:945)
at
org.apache.hudi.HoodieSparkSqlWriter$.write(HoodieSparkSqlWriter.scala:381)
at
org.apache.hudi.DefaultSource.createRelation(DefaultSource.scala:150)
at
org.apache.spark.sql.execution.datasources.SaveIntoDataSourceCommand.run(SaveIntoDataSourceCommand.scala:45)
at
org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:75)
at
org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:73)
at
org.apache.spark.sql.execution.command.ExecutedCommandExec.executeCollect(commands.scala:84)
at
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.$anonfun$applyOrElse$1(QueryExecution.scala:110)
at
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:103)
at
org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:163)
at
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:90)
at
org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:775)
at
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
at
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:110)
at
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:106)
at
org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:481)
at
org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:82)
at
org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:481)
at
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:30)
at
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267)
at
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263)
at
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30)
at
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30)
at
org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:457)
at
org.apache.spark.sql.execution.QueryExecution.eagerlyExecuteCommands(QueryExecution.scala:106)
at
org.apache.spark.sql.execution.QueryExecution.commandExecuted$lzycompute(QueryExecution.scala:93)
at
org.apache.spark.sql.execution.QueryExecution.commandExecuted(QueryExecution.scala:91)
at
org.apache.spark.sql.execution.QueryExecution.assertCommandExecuted(QueryExecution.scala:128)
at
org.apache.spark.sql.DataFrameWriter.runCommand(DataFrameWriter.scala:848)
at
org.apache.spark.sql.DataFrameWriter.saveToV1Source(DataFrameWriter.scala:382)
at
org.apache.spark.sql.DataFrameWriter.saveInternal(DataFrameWriter.scala:355)
at
org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:239)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498)
at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)
at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)
at py4j.Gateway.invoke(Gateway.java:282)
at
py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
at py4j.commands.CallCommand.execute(CallCommand.java:79)
at
py4j.ClientServerConnection.waitForCommands(ClientServerConnection.java:182)
at py4j.ClientServerConnection.run(ClientServerConnection.java:106)
at java.lang.Thread.run(Thread.java:750)
Caused by: 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)
... 1 more
Caused by: org.apache.hudi.exception.HoodieException:
org.apache.hudi.exception.HoodieException:
java.lang.UnsupportedOperationException
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.UnsupportedOperationException
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.UnsupportedOperationException
at
org.apache.spark.sql.catalyst.expressions.UnsafeArrayData.update(UnsafeArrayData.java:273)
at
org.apache.spark.sql.HoodieInternalRowUtils$ArrayDataUpdater.set(HoodieInternalRowUtils.scala:421)
at
org.apache.spark.sql.HoodieInternalRowUtils$.$anonfun$newWriterRenaming$1(HoodieInternalRowUtils.scala:206)
at
org.apache.spark.sql.HoodieInternalRowUtils$.$anonfun$newWriterRenaming$1$adapted(HoodieInternalRowUtils.scala:206)
at
org.apache.spark.sql.HoodieInternalRowUtils$.$anonfun$newWriterRenaming$4(HoodieInternalRowUtils.scala:247)
at
org.apache.spark.sql.HoodieInternalRowUtils$.$anonfun$newWriterRenaming$4$adapted(HoodieInternalRowUtils.scala:229)
at
org.apache.spark.sql.HoodieInternalRowUtils$.$anonfun$genUnsafeStructWriter$3(HoodieInternalRowUtils.scala:194)
at
org.apache.spark.sql.HoodieInternalRowUtils$.$anonfun$genUnsafeStructWriter$3$adapted(HoodieInternalRowUtils.scala:181)
at
org.apache.spark.sql.HoodieInternalRowUtils$.$anonfun$newWriterRenaming$3(HoodieInternalRowUtils.scala:220)
at
org.apache.spark.sql.HoodieInternalRowUtils$.$anonfun$newWriterRenaming$3$adapted(HoodieInternalRowUtils.scala:214)
at
org.apache.spark.sql.HoodieInternalRowUtils$.$anonfun$genUnsafeRowWriter$1(HoodieInternalRowUtils.scala:135)
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]