Zbigniew Baranowski created HUDI-7212:
-----------------------------------------
Summary: java.lang.IllegalArgumentException when writing to Hudi
with ORC base format
Key: HUDI-7212
URL: https://issues.apache.org/jira/browse/HUDI-7212
Project: Apache Hudi
Issue Type: Bug
Reporter: Zbigniew Baranowski
*What is the problem?*
I am getting java.lang.IllegalArgumentException when writing any data with
DECIMAL type with precision < 10. This is a bug related to the wrong
initialization order of a decimal precision and scale:
https://github.com/apache/hudi/blob/a7c01f6874b20ebebb24399995ed8e8aba09cb2a/hudi-common/src/main/java/org/apache/hudi/common/util/AvroOrcUtils.java#L612
As per orc lib specificity, when building decimal type, first should go scale
setting then precision, otherwise, the scale is by default to 10, so precision
cannot be set to a lower value than 10 without exception thrown by some sanity
check:
https://github.com/apache/orc/blob/ede42277e10486e4885ce8f99facd7d194a79498/java/core/src/java/org/apache/orc/TypeDescription.java#L218
For reference - the same problem was affecting orc tools in the past:
https://github.com/apache/orc/pull/127/files
*Expected behavior:*
The dataframe should be written to hudi in orc format without an error.
Version affected:
Occurred 0.14.0 and Spark 3.4.1 - but most likely all the versions with orc
support are affected. The Spark version does not have any impact here - so can
any supported
*Steps to reproduce:*
This issue can be quite easily reproduced with the spark shell
{code:java}
-- spark 3.4.1
spark-shell \
--packages org.apache.hudi:hudi-spark3.4-bundle_2.12:0.14.0 \
--conf 'spark.serializer=org.apache.spark.serializer.KryoSerializer' \
--conf
'spark.sql.catalog.spark_catalog=org.apache.spark.sql.hudi.catalog.HoodieCatalog'
\
--conf
'spark.sql.extensions=org.apache.spark.sql.hudi.HoodieSparkSessionExtension' \
--conf 'spark.kryo.registrator=org.apache.spark.HoodieSparkKryoRegistrar'
-----------------
import org.apache.spark.sql.types.{StructType, StructField, DecimalType}
import org.apache.spark.sql.{Row, SaveMode}
import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.DataSourceWriteOptions
// Define the precision and scale for the DecimalType column
val decimalPrecision = 7
val decimalScale = 0
// Define the schema with a DecimalType column
val schema = StructType(Seq(
StructField("id", org.apache.spark.sql.types.IntegerType, nullable = false),
StructField("decimalColumn", DecimalType(decimalPrecision, decimalScale),
nullable = false)
))
// Create sample data
val data = Seq(
Row(1, BigDecimal("123.45")),
Row(2, BigDecimal("678.90")),
// Add more rows as needed
)
// Create a DataFrame with the specified schema and data
val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema)
// show df
df.show(false)
+---+-------------+
|id |decimalColumn|
+---+-------------+
|1 |123 |
|2 |679 |
+---+-------------+
df.printSchema
root
|-- id: integer (nullable = false)
|-- decimalColumn: decimal(7,0) (nullable = false)
// Specify the Hudi table name and path
val tableName = "hudi_orc_bug"
val hudiPath = "/tmp/hudi_orc_bug"
// Write the DataFrame to Hudi with base ORC format
df.write.
format("org.apache.hudi").
option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "id").
option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "decimalColumn").
option(HoodieWriteConfig.TABLE_NAME, tableName).
option(HoodieWriteConfig.BASE_PATH_PROP, hudiPath).
option(DataSourceWriteOptions.STORAGE_TYPE_OPT_KEY, "COPY_ON_WRITE").
option(HoodieWriteConfig.BASE_FILE_FORMAT.key, "ORC").
mode(SaveMode.Append).
save(hudiPath)
{code}
*Stacktrace:*
{code:java}
Driver stacktrace:
at
org.apache.spark.scheduler.DAGScheduler.failJobAndIndependentStages(DAGScheduler.scala:2785)
at
org.apache.spark.scheduler.DAGScheduler.$anonfun$abortStage$2(DAGScheduler.scala:2721)
at
org.apache.spark.scheduler.DAGScheduler.$anonfun$abortStage$2$adapted(DAGScheduler.scala:2720)
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:2720)
at
org.apache.spark.scheduler.DAGScheduler.$anonfun$handleTaskSetFailed$1(DAGScheduler.scala:1206)
at
org.apache.spark.scheduler.DAGScheduler.$anonfun$handleTaskSetFailed$1$adapted(DAGScheduler.scala:1206)
at scala.Option.foreach(Option.scala:407)
at
org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:1206)
at
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:2984)
at
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2923)
at
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2912)
at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:49)
at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:971)
at org.apache.spark.SparkContext.runJob(SparkContext.scala:2263)
at org.apache.spark.SparkContext.runJob(SparkContext.scala:2284)
at org.apache.spark.SparkContext.runJob(SparkContext.scala:2303)
at org.apache.spark.SparkContext.runJob(SparkContext.scala:2328)
at org.apache.spark.rdd.RDD.count(RDD.scala:1266)
at
org.apache.hudi.HoodieSparkSqlWriter$.commitAndPerformPostOperations(HoodieSparkSqlWriter.scala:1050)
at
org.apache.hudi.HoodieSparkSqlWriter$.writeInternal(HoodieSparkSqlWriter.scala:441)
at org.apache.hudi.HoodieSparkSqlWriter$.write(HoodieSparkSqlWriter.scala:132)
at org.apache.hudi.DefaultSource.createRelation(DefaultSource.scala:150)
at
org.apache.spark.sql.execution.datasources.SaveIntoDataSourceCommand.run(SaveIntoDataSourceCommand.scala:47)
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:98)
at
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$6(SQLExecution.scala:118)
at
org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:195)
at
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:103)
at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:827)
at
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:65)
at
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:98)
at
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:94)
at
org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:512)
at
org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:104)
at
org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:512)
at
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:31)
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:31)
at
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:31)
at
org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:488)
at
org.apache.spark.sql.execution.QueryExecution.eagerlyExecuteCommands(QueryExecution.scala:94)
at
org.apache.spark.sql.execution.QueryExecution.commandExecuted$lzycompute(QueryExecution.scala:81)
at
org.apache.spark.sql.execution.QueryExecution.commandExecuted(QueryExecution.scala:79)
at
org.apache.spark.sql.execution.QueryExecution.assertCommandExecuted(QueryExecution.scala:133)
at org.apache.spark.sql.DataFrameWriter.runCommand(DataFrameWriter.scala:856)
at
org.apache.spark.sql.DataFrameWriter.saveToV1Source(DataFrameWriter.scala:387)
at
org.apache.spark.sql.DataFrameWriter.saveInternal(DataFrameWriter.scala:360)
at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:239)
... 56 elided
Caused by: java.lang.RuntimeException:
org.apache.hudi.exception.HoodieException:
org.apache.hudi.exception.HoodieException: java.lang.IllegalArgumentException:
precision 7 is out of range 1 .. 10
at
org.apache.hudi.client.utils.LazyIterableIterator.next(LazyIterableIterator.java:121)
at scala.collection.convert.Wrappers$JIteratorWrapper.next(Wrappers.scala:46)
at scala.collection.Iterator$$anon$11.nextCur(Iterator.scala:486)
at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:492)
at
org.apache.spark.storage.memory.MemoryStore.putIterator(MemoryStore.scala:223)
at
org.apache.spark.storage.memory.MemoryStore.putIteratorAsBytes(MemoryStore.scala:352)
at
org.apache.spark.storage.BlockManager.$anonfun$doPutIterator$1(BlockManager.scala:1552)
at
org.apache.spark.storage.BlockManager.org$apache$spark$storage$BlockManager$$doPut(BlockManager.scala:1462)
at
org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:1526)
at
org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:1349)
at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:375)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:326)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:364)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:328)
at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:92)
at org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:161)
at org.apache.spark.scheduler.Task.run(Task.scala:139)
at
org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:554)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1529)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:557)
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.IllegalArgumentException:
precision 7 is out of range 1 .. 10
at
org.apache.hudi.execution.SparkLazyInsertIterable.computeNext(SparkLazyInsertIterable.java:84)
at
org.apache.hudi.execution.SparkLazyInsertIterable.computeNext(SparkLazyInsertIterable.java:39)
at
org.apache.hudi.client.utils.LazyIterableIterator.next(LazyIterableIterator.java:119)
... 23 more
Caused by: org.apache.hudi.exception.HoodieException:
java.lang.IllegalArgumentException: precision 7 is out of range 1 .. 10
at
org.apache.hudi.common.util.queue.SimpleExecutor.execute(SimpleExecutor.java:75)
at
org.apache.hudi.execution.SparkLazyInsertIterable.computeNext(SparkLazyInsertIterable.java:80)
... 25 more
Caused by: java.lang.IllegalArgumentException: precision 7 is out of range 1 ..
10
at org.apache.orc.TypeDescription.withPrecision(TypeDescription.java:219)
at
org.apache.hudi.common.util.AvroOrcUtils.createOrcSchema(AvroOrcUtils.java:612)
at
org.apache.hudi.common.util.AvroOrcUtils.createOrcSchema(AvroOrcUtils.java:670)
at
org.apache.hudi.io.storage.HoodieAvroOrcWriter.<init>(HoodieAvroOrcWriter.java:77)
at
org.apache.hudi.io.storage.HoodieAvroFileWriterFactory.newOrcFileWriter(HoodieAvroFileWriterFactory.java:107)
at
org.apache.hudi.io.storage.HoodieFileWriterFactory.getFileWriterByFormat(HoodieFileWriterFactory.java:86)
at
org.apache.hudi.io.storage.HoodieFileWriterFactory.getFileWriter(HoodieFileWriterFactory.java:67)
at org.apache.hudi.io.HoodieCreateHandle.<init>(HoodieCreateHandle.java:104)
at org.apache.hudi.io.HoodieCreateHandle.<init>(HoodieCreateHandle.java:76)
at org.apache.hudi.io.CreateHandleFactory.create(CreateHandleFactory.java:45)
at
org.apache.hudi.execution.CopyOnWriteInsertHandler.consume(CopyOnWriteInsertHandler.java:85)
at
org.apache.hudi.execution.CopyOnWriteInsertHandler.consume(CopyOnWriteInsertHandler.java:42)
at
org.apache.hudi.common.util.queue.SimpleExecutor.execute(SimpleExecutor.java:69)
... 26 more
{code}
--
This message was sent by Atlassian Jira
(v8.20.10#820010)