[
https://issues.apache.org/jira/browse/HUDI-2105?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
pengzhiwei updated HUDI-2105:
-----------------------------
Description:
Compaction will fail when do merge-into by spark sql with the
*hoodie.compact.inline* enable*.*
This sql is as followed:
{code:java}
spark.sql(
s"""
|create table h0(
| id int,
| name string,
| price double,
| ts long
|) using hudi
| location '/tmp/hudi/h0'
| options (
| primaryKey ='id',
| type = 'mor',
| preCombineField = 'ts',
| hoodie.compact.inline = 'true'
| )
""".stripMargin)
spark.sql("insert into h0 values(1, 'a1', 10, 1000)")
spark.sql(s"insert into h0 values(2, 'a2', 10, 1000)")
spark.sql(s"insert into h0 values(3, 'a3', 10, 1000)")
spark.sql(s"insert into h0 values(4, 'a4', 10, 1000)")
spark.sql(
s"""
|merge into h0
|using (
| select 4 as id, 'a4' as name, 11 as price, 1000 as ts
| ) s0
| on h0.id = s0.id
| when matched then update set *
|""".stripMargin)
{code}
The follow exception will throw out:
{code:java}
org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in
stage 137.0 failed 1 times, most recent failure: Lost task 0.0 in stage 137.0
(TID 189, localhost, executor driver):
org.apache.hudi.exception.HoodieException: Exception when reading log file
org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in
stage 137.0 failed 1 times, most recent failure: Lost task 0.0 in stage 137.0
(TID 189, localhost, executor driver):
org.apache.hudi.exception.HoodieException: Exception when reading log file at
org.apache.hudi.common.table.log.AbstractHoodieLogRecordScanner.scan(AbstractHoodieLogRecordScanner.java:269)
at
org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner.performScan(HoodieMergedLogRecordScanner.java:94)
at
org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner.<init>(HoodieMergedLogRecordScanner.java:87)
at
org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner$Builder.build(HoodieMergedLogRecordScanner.java:238)
at
org.apache.hudi.table.action.compact.HoodieSparkMergeOnReadTableCompactor.compact(HoodieSparkMergeOnReadTableCompactor.java:150)
at
org.apache.hudi.table.action.compact.HoodieSparkMergeOnReadTableCompactor.lambda$compact$9ec9d4c7$1(HoodieSparkMergeOnReadTableCompactor.java:114)
at
org.apache.spark.api.java.JavaPairRDD$$anonfun$toScalaFunction$1.apply(JavaPairRDD.scala:1040)
at scala.collection.Iterator$$anon$11.next(Iterator.scala:410) at
scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:435) at
scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:441) at
org.apache.spark.storage.memory.MemoryStore.putIterator(MemoryStore.scala:221)
at
org.apache.spark.storage.memory.MemoryStore.putIteratorAsBytes(MemoryStore.scala:349)
at
org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1182)
at
org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1156)
at org.apache.spark.storage.BlockManager.doPut(BlockManager.scala:1091) at
org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:1156) at
org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:882)
at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:335) at
org.apache.spark.rdd.RDD.iterator(RDD.scala:286) at
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) at
org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324) at
org.apache.spark.rdd.RDD.iterator(RDD.scala:288) at
org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90) at
org.apache.spark.scheduler.Task.run(Task.scala:123) at
org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:408)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360) at
org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:414) 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)Caused by:
org.apache.avro.AvroTypeException: Found double, expecting int at
org.apache.avro.io.ResolvingDecoder.doAction(ResolvingDecoder.java:292) at
org.apache.avro.io.parsing.Parser.advance(Parser.java:88) at
org.apache.avro.io.ValidatingDecoder.readInt(ValidatingDecoder.java:82) at
org.apache.avro.generic.GenericDatumReader.readInt(GenericDatumReader.java:503)
at
org.apache.avro.generic.GenericDatumReader.readWithoutConversion(GenericDatumReader.java:183)
at
org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:153) at
org.apache.avro.generic.GenericDatumReader.readField(GenericDatumReader.java:232)
at
org.apache.avro.generic.GenericDatumReader.readRecord(GenericDatumReader.java:222)
at
org.apache.avro.generic.GenericDatumReader.readWithoutConversion(GenericDatumReader.java:175)
at
org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:153) at
org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:145) at
org.apache.hudi.common.table.log.block.HoodieAvroDataBlock.deserializeRecords(HoodieAvroDataBlock.java:157)
at
org.apache.hudi.common.table.log.block.HoodieDataBlock.createRecordsFromContentBytes(HoodieDataBlock.java:128)
at
org.apache.hudi.common.table.log.block.HoodieDataBlock.getRecords(HoodieDataBlock.java:106)
at
org.apache.hudi.common.table.log.AbstractHoodieLogRecordScanner.processDataBlock(AbstractHoodieLogRecordScanner.java:297)
at
org.apache.hudi.common.table.log.AbstractHoodieLogRecordScanner.processQueuedBlocksForInstant(AbstractHoodieLogRecordScanner.java:332)
at
org.apache.hudi.common.table.log.AbstractHoodieLogRecordScanner.scan(AbstractHoodieLogRecordScanner.java:260)
... 29 more
{code}
was:Compaction will fail when do merge-into by spark sql with the
*hoodie.compact.inline* enable*.*
> Compaction Failed For MergeInto
> -------------------------------
>
> Key: HUDI-2105
> URL: https://issues.apache.org/jira/browse/HUDI-2105
> Project: Apache Hudi
> Issue Type: Bug
> Components: Spark Integration
> Reporter: pengzhiwei
> Assignee: pengzhiwei
> Priority: Blocker
> Fix For: 0.9.0
>
>
> Compaction will fail when do merge-into by spark sql with the
> *hoodie.compact.inline* enable*.*
> This sql is as followed:
> {code:java}
> spark.sql(
> s"""
> |create table h0(
> | id int,
> | name string,
> | price double,
> | ts long
> |) using hudi
> | location '/tmp/hudi/h0'
> | options (
> | primaryKey ='id',
> | type = 'mor',
> | preCombineField = 'ts',
> | hoodie.compact.inline = 'true'
> | )
> """.stripMargin)
> spark.sql("insert into h0 values(1, 'a1', 10, 1000)")
> spark.sql(s"insert into h0 values(2, 'a2', 10, 1000)")
> spark.sql(s"insert into h0 values(3, 'a3', 10, 1000)")
> spark.sql(s"insert into h0 values(4, 'a4', 10, 1000)")
> spark.sql(
> s"""
> |merge into h0
> |using (
> | select 4 as id, 'a4' as name, 11 as price, 1000 as ts
> | ) s0
> | on h0.id = s0.id
> | when matched then update set *
> |""".stripMargin)
> {code}
>
> The follow exception will throw out:
> {code:java}
> org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in
> stage 137.0 failed 1 times, most recent failure: Lost task 0.0 in stage 137.0
> (TID 189, localhost, executor driver):
> org.apache.hudi.exception.HoodieException: Exception when reading log file
> org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in
> stage 137.0 failed 1 times, most recent failure: Lost task 0.0 in stage 137.0
> (TID 189, localhost, executor driver):
> org.apache.hudi.exception.HoodieException: Exception when reading log file
> at
> org.apache.hudi.common.table.log.AbstractHoodieLogRecordScanner.scan(AbstractHoodieLogRecordScanner.java:269)
> at
> org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner.performScan(HoodieMergedLogRecordScanner.java:94)
> at
> org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner.<init>(HoodieMergedLogRecordScanner.java:87)
> at
> org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner$Builder.build(HoodieMergedLogRecordScanner.java:238)
> at
> org.apache.hudi.table.action.compact.HoodieSparkMergeOnReadTableCompactor.compact(HoodieSparkMergeOnReadTableCompactor.java:150)
> at
> org.apache.hudi.table.action.compact.HoodieSparkMergeOnReadTableCompactor.lambda$compact$9ec9d4c7$1(HoodieSparkMergeOnReadTableCompactor.java:114)
> at
> org.apache.spark.api.java.JavaPairRDD$$anonfun$toScalaFunction$1.apply(JavaPairRDD.scala:1040)
> at scala.collection.Iterator$$anon$11.next(Iterator.scala:410) at
> scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:435) at
> scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:441) at
> org.apache.spark.storage.memory.MemoryStore.putIterator(MemoryStore.scala:221)
> at
> org.apache.spark.storage.memory.MemoryStore.putIteratorAsBytes(MemoryStore.scala:349)
> at
> org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1182)
> at
> org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1156)
> at org.apache.spark.storage.BlockManager.doPut(BlockManager.scala:1091) at
> org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:1156)
> at
> org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:882)
> at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:335) at
> org.apache.spark.rdd.RDD.iterator(RDD.scala:286) at
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) at
> org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324) at
> org.apache.spark.rdd.RDD.iterator(RDD.scala:288) at
> org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90) at
> org.apache.spark.scheduler.Task.run(Task.scala:123) at
> org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:408)
> at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360) at
> org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:414) 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)Caused by:
> org.apache.avro.AvroTypeException: Found double, expecting int at
> org.apache.avro.io.ResolvingDecoder.doAction(ResolvingDecoder.java:292) at
> org.apache.avro.io.parsing.Parser.advance(Parser.java:88) at
> org.apache.avro.io.ValidatingDecoder.readInt(ValidatingDecoder.java:82) at
> org.apache.avro.generic.GenericDatumReader.readInt(GenericDatumReader.java:503)
> at
> org.apache.avro.generic.GenericDatumReader.readWithoutConversion(GenericDatumReader.java:183)
> at
> org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:153)
> at
> org.apache.avro.generic.GenericDatumReader.readField(GenericDatumReader.java:232)
> at
> org.apache.avro.generic.GenericDatumReader.readRecord(GenericDatumReader.java:222)
> at
> org.apache.avro.generic.GenericDatumReader.readWithoutConversion(GenericDatumReader.java:175)
> at
> org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:153)
> at
> org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:145)
> at
> org.apache.hudi.common.table.log.block.HoodieAvroDataBlock.deserializeRecords(HoodieAvroDataBlock.java:157)
> at
> org.apache.hudi.common.table.log.block.HoodieDataBlock.createRecordsFromContentBytes(HoodieDataBlock.java:128)
> at
> org.apache.hudi.common.table.log.block.HoodieDataBlock.getRecords(HoodieDataBlock.java:106)
> at
> org.apache.hudi.common.table.log.AbstractHoodieLogRecordScanner.processDataBlock(AbstractHoodieLogRecordScanner.java:297)
> at
> org.apache.hudi.common.table.log.AbstractHoodieLogRecordScanner.processQueuedBlocksForInstant(AbstractHoodieLogRecordScanner.java:332)
> at
> org.apache.hudi.common.table.log.AbstractHoodieLogRecordScanner.scan(AbstractHoodieLogRecordScanner.java:260)
> ... 29 more
> {code}
>
--
This message was sent by Atlassian Jira
(v8.3.4#803005)