Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

The file being written is in an invalid state. Probably caused by an error thrown previously. Current state: COLUMN #5867

Closed
hellochueng opened this issue Jun 14, 2022 · 7 comments
Labels
flink Issues related to flink priority:major degraded perf; unable to move forward; potential bugs writer-core Issues relating to core transactions/write actions

Comments

@hellochueng
Copy link

2022-06-14 19:58:19,560 ERROR org.apache.hudi.io.HoodieMergeHandle [] - Error writing record HoodieRecord{key=HoodieKey { recordKey=fdbid:79505959536,fbillid:79505959731,fentryid:16,dim:hz partitionPath=fdatemonth=202203}, currentLocation='null', newLocation='null'}
java.io.IOException: The file being written is in an invalid state. Probably caused by an error thrown previously. Current state: COLUMN
at org.apache.hudi.org.apache.parquet.hadoop.ParquetFileWriter$STATE.error(ParquetFileWriter.java:192) ~[hudi-flink-bundle_2.11-0.10.1.jar:0.10.1]
at org.apache.hudi.org.apache.parquet.hadoop.ParquetFileWriter$STATE.startBlock(ParquetFileWriter.java:184) ~[hudi-flink-bundle_2.11-0.10.1.jar:0.10.1]
at org.apache.hudi.org.apache.parquet.hadoop.ParquetFileWriter.startBlock(ParquetFileWriter.java:348) ~[hudi-flink-bundle_2.11-0.10.1.jar:0.10.1]
at org.apache.hudi.org.apache.parquet.hadoop.InternalParquetRecordWriter.flushRowGroupToStore(InternalParquetRecordWriter.java:171) ~[hudi-flink-bundle_2.11-0.10.1.jar:0.10.1]
at org.apache.hudi.org.apache.parquet.hadoop.InternalParquetRecordWriter.checkBlockSizeReached(InternalParquetRecordWriter.java:148) ~[hudi-flink-bundle_2.11-0.10.1.jar:0.10.1]
at org.apache.hudi.org.apache.parquet.hadoop.InternalParquetRecordWriter.write(InternalParquetRecordWriter.java:130) ~[hudi-flink-bundle_2.11-0.10.1.jar:0.10.1]
at org.apache.hudi.org.apache.parquet.hadoop.ParquetWriter.write(ParquetWriter.java:301) ~[hudi-flink-bundle_2.11-0.10.1.jar:0.10.1]
at org.apache.hudi.io.storage.HoodieParquetWriter.writeAvroWithMetadata(HoodieParquetWriter.java:81) ~[hudi-flink-bundle_2.11-0.10.1.jar:0.10.1]
at org.apache.hudi.io.HoodieMergeHandle.writeRecord(HoodieMergeHandle.java:294) ~[hudi-flink-bundle_2.11-0.10.1.jar:0.10.1]
at org.apache.hudi.io.HoodieMergeHandle.writeInsertRecord(HoodieMergeHandle.java:273) ~[hudi-flink-bundle_2.11-0.10.1.jar:0.10.1]
at org.apache.hudi.io.HoodieMergeHandle.writeIncomingRecords(HoodieMergeHandle.java:369) ~[hudi-flink-bundle_2.11-0.10.1.jar:0.10.1]
at org.apache.hudi.io.HoodieMergeHandle.close(HoodieMergeHandle.java:377) ~[hudi-flink-bundle_2.11-0.10.1.jar:0.10.1]
at org.apache.hudi.table.action.commit.FlinkMergeHelper.runMerge(FlinkMergeHelper.java:108) ~[hudi-flink-bundle_2.11-0.10.1.jar:0.10.1]
at org.apache.hudi.table.HoodieFlinkCopyOnWriteTable.handleUpdateInternal(HoodieFlinkCopyOnWriteTable.java:368) ~[hudi-flink-bundle_2.11-0.10.1.jar:0.10.1]
at org.apache.hudi.table.HoodieFlinkCopyOnWriteTable.handleUpdate(HoodieFlinkCopyOnWriteTable.java:359) ~[hudi-flink-bundle_2.11-0.10.1.jar:0.10.1]
at org.apache.hudi.table.action.compact.HoodieCompactor.compact(HoodieCompactor.java:197) ~[hudi-flink-bundle_2.11-0.10.1.jar:0.10.1]
at org.apache.hudi.sink.compact.CompactFunction.doCompaction(CompactFunction.java:104) ~[hudi-flink-bundle_2.11-0.10.1.jar:0.10.1]
at org.apache.hudi.sink.compact.CompactFunction.lambda$processElement$0(CompactFunction.java:92) ~[hudi-flink-bundle_2.11-0.10.1.jar:0.10.1]
at org.apache.hudi.sink.utils.NonThrownExecutor.lambda$execute$0(NonThrownExecutor.java:93) ~[hudi-flink-bundle_2.11-0.10.1.jar:0.10.1]
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_281]
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_281]
at java.lang.Thread.run(Thread.java:748) [?:1.8.0_281]
image
mor upsert
image

@codope
Copy link
Member

codope commented Jun 14, 2022

@hellochueng Can you give us more details about your setup? Is it multi-writer or single writer? Is it consistently reproducible?
The stacktrace suggests that the HoodieCompactor was attempting ParquetFileWriter write/close which was simultaneously being attempted by another writer. If it's a multi-writer setup, have you configured conurrency mode and lock provider?

@codope
Copy link
Member

codope commented Jun 14, 2022

cc @danny0405
Have you come across such an issue with filnk compactor?

@danny0405
Copy link
Contributor

You mean the error throws because of multi components were trying to modify the same parquet file ? In flink write pipeline, the only component that may modify the parquet files is the CompactFunction, in theory, it does not expect to be in concurrency modification.

@codope
Copy link
Member

codope commented Jun 15, 2022

@hellochueng Can you please share the steps to reproduce the issue?

@codope codope added flink Issues related to flink priority:major degraded perf; unable to move forward; potential bugs writer-core Issues relating to core transactions/write actions labels Jun 17, 2022
@JerryYue-M
Copy link
Contributor

@danny0405 @codope
With Hudi release-0.11.0 version. this error appears frequently in the compact task so it can make compact fail.
I found that. firstly it may appear RemoteException: File does not exist error that can cause mergeHandle close,before close it flush some records.at finally.it occur follow error:

2022-06-24 21:22:55,019 ERROR org.apache.hudi.io.HoodieMergeHandle [] - Error writing record HoodieRecord{key=HoodieKey { recordKey=xxx ea125773f partitionPath=2022-06-21/18}, currentLocation='null', newLocation='null'}
java.io.IOException: The file being written is in an invalid state. Probably caused by an error thrown previously. Current state: COLUMN
at org.apache.hudi.org.apache.parquet.hadoop.ParquetFileWriter$STATE.error(ParquetFileWriter.java:217) ~[blob_p-295f7415f20d1fe87ffb9658937af184c87dc096-45deddc0573ab868da621d786b6f266a:0.11.0]
at org.apache.hudi.org.apache.parquet.hadoop.ParquetFileWriter$STATE.startBlock(ParquetFileWriter.java:209) ~[blob_p-295f7415f20d1fe87ffb9658937af184c87dc096-45deddc0573ab868da621d786b6f266a:0.11.0]
at org.apache.hudi.org.apache.parquet.hadoop.ParquetFileWriter.startBlock(ParquetFileWriter.java:407) ~[blob_p-295f7415f20d1fe87ffb9658937af184c87dc096-45deddc0573ab868da621d786b6f266a:0.11.0]
at org.apache.hudi.org.apache.parquet.hadoop.InternalParquetRecordWriter.flushRowGroupToStore(InternalParquetRecordWriter.java:184) ~[blob_p-295f7415f20d1fe87ffb9658937af184c87dc096-45deddc0573ab868da621d786b6f266a:0.11.0]
at org.apache.hudi.org.apache.parquet.hadoop.InternalParquetRecordWriter.checkBlockSizeReached(InternalParquetRecordWriter.java:158) ~[blob_p-295f7415f20d1fe87ffb9658937af184c87dc096-45deddc0573ab868da621d786b6f266a:0.11.0]
at org.apache.hudi.org.apache.parquet.hadoop.InternalParquetRecordWriter.write(InternalParquetRecordWriter.java:140) ~[blob_p-295f7415f20d1fe87ffb9658937af184c87dc096-45deddc0573ab868da621d786b6f266a:0.11.0]
at org.apache.hudi.org.apache.parquet.hadoop.ParquetWriter.write(ParquetWriter.java:310) ~[blob_p-295f7415f20d1fe87ffb9658937af184c87dc096-45deddc0573ab868da621d786b6f266a:0.11.0]
at org.apache.hudi.io.storage.HoodieParquetWriter.writeAvro(HoodieParquetWriter.java:104) ~[blob_p-295f7415f20d1fe87ffb9658937af184c87dc096-45deddc0573ab868da621d786b6f266a:0.11.0]
at org.apache.hudi.io.HoodieMergeHandle.writeToFile(HoodieMergeHandle.java:367) ~[blob_p-295f7415f20d1fe87ffb9658937af184c87dc096-45deddc0573ab868da621d786b6f266a:0.11.0]
at org.apache.hudi.io.HoodieMergeHandle.writeRecord(HoodieMergeHandle.java:296) ~[blob_p-295f7415f20d1fe87ffb9658937af184c87dc096-45deddc0573ab868da621d786b6f266a:0.11.0]
at org.apache.hudi.io.HoodieMergeHandle.writeInsertRecord(HoodieMergeHandle.java:277) ~[blob_p-295f7415f20d1fe87ffb9658937af184c87dc096-45deddc0573ab868da621d786b6f266a:0.11.0]
at org.apache.hudi.io.HoodieMergeHandle.writeIncomingRecords(HoodieMergeHandle.java:380) ~[blob_p-295f7415f20d1fe87ffb9658937af184c87dc096-45deddc0573ab868da621d786b6f266a:0.11.0]
at org.apache.hudi.io.HoodieMergeHandle.close(HoodieMergeHandle.java:388) ~[blob_p-295f7415f20d1fe87ffb9658937af184c87dc096-45deddc0573ab868da621d786b6f266a:0.11.0]
at org.apache.hudi.table.action.commit.FlinkMergeHelper.runMerge(FlinkMergeHelper.java:108) ~[blob_p-295f7415f20d1fe87ffb9658937af184c87dc096-45deddc0573ab868da621d786b6f266a:0.11.0]
at org.apache.hudi.table.HoodieFlinkCopyOnWriteTable.handleUpdateInternal(HoodieFlinkCopyOnWriteTable.java:379) ~[blob_p-295f7415f20d1fe87ffb9658937af184c87dc096-45deddc0573ab868da621d786b6f266a:0.11.0]
at org.apache.hudi.table.HoodieFlinkCopyOnWriteTable.handleUpdate(HoodieFlinkCopyOnWriteTable.java:370) ~[blob_p-295f7415f20d1fe87ffb9658937af184c87dc096-45deddc0573ab868da621d786b6f266a:0.11.0]
at org.apache.hudi.table.action.compact.HoodieCompactor.compact(HoodieCompactor.java:227) ~

@danny0405
Copy link
Contributor

Do you mean in HoodieCompactionHandler#handleInsert, we do not close the file handle correctly when exception occurs ? That's a valid point, maybe we can wrap the handles in try finally resource block.

@danny0405
Copy link
Contributor

The issue is expected to be resolved by this pr: #6106, feel free to re-open it if the problem still exists.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
flink Issues related to flink priority:major degraded perf; unable to move forward; potential bugs writer-core Issues relating to core transactions/write actions
Projects
None yet
Development

No branches or pull requests

5 participants