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

[SUPPORT]FLINK CDC WRITE HUDI, restart job get exception:org.apache.hudi.org.apache.avro.InvalidAvroMagicException: Not an Avro data file #4249

Closed
wjcwin opened this issue Dec 8, 2021 · 13 comments

Comments

@wjcwin
Copy link

wjcwin commented Dec 8, 2021

FLINK CDC WRITE HUDI, restart job get exception:org.apache.hudi.org.apache.avro.InvalidAvroMagicException: Not an Avro data file

logs:

org.apache.flink.util.FlinkException: Global failure triggered by OperatorCoordinator for 'hoodie_stream_write' (operator f1d7c56f4bf5fc204e4401416e5b3884).
	at org.apache.flink.runtime.operators.coordination.OperatorCoordinatorHolder$LazyInitializedCoordinatorContext.failJob(OperatorCoordinatorHolder.java:557)
	at org.apache.hudi.sink.StreamWriteOperatorCoordinator.lambda$start$0(StreamWriteOperatorCoordinator.java:170)
	at org.apache.hudi.sink.utils.NonThrownExecutor.lambda$execute$0(NonThrownExecutor.java:103)
	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.hudi.exception.HoodieException: Executor executes action [initialize instant ] error
	... 5 more
Caused by: org.apache.hudi.exception.HoodieIOException: Fetching rollback plan failed for [==>20211208144119644__rollback__REQUESTED]
	at org.apache.hudi.client.AbstractHoodieWriteClient.lambda$getPendingRollbackInfos$8(AbstractHoodieWriteClient.java:883)
	at java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
	at java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1382)
	at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:481)
	at java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:471)
	at java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:708)
	at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
	at java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:499)
	at org.apache.hudi.client.AbstractHoodieWriteClient.getPendingRollbackInfos(AbstractHoodieWriteClient.java:886)
	at org.apache.hudi.client.AbstractHoodieWriteClient.rollbackFailedWrites(AbstractHoodieWriteClient.java:909)
	at org.apache.hudi.client.AbstractHoodieWriteClient.rollbackFailedWrites(AbstractHoodieWriteClient.java:899)
	at org.apache.hudi.client.AbstractHoodieWriteClient.lambda$startCommitWithTime$97cdbdca$1(AbstractHoodieWriteClient.java:785)
	at org.apache.hudi.common.util.CleanerUtils.rollbackFailedWrites(CleanerUtils.java:143)
	at org.apache.hudi.client.AbstractHoodieWriteClient.startCommitWithTime(AbstractHoodieWriteClient.java:784)
	at org.apache.hudi.client.AbstractHoodieWriteClient.startCommitWithTime(AbstractHoodieWriteClient.java:777)
	at org.apache.hudi.sink.StreamWriteOperatorCoordinator.startInstant(StreamWriteOperatorCoordinator.java:334)
	at org.apache.hudi.sink.StreamWriteOperatorCoordinator.lambda$initInstant$5(StreamWriteOperatorCoordinator.java:361)
	at org.apache.hudi.sink.utils.NonThrownExecutor.lambda$execute$0(NonThrownExecutor.java:93)
	... 3 more
Caused by: org.apache.hudi.org.apache.avro.InvalidAvroMagicException: Not an Avro data file
	at org.apache.hudi.org.apache.avro.file.DataFileReader.openReader(DataFileReader.java:56)
	at org.apache.hudi.common.table.timeline.TimelineMetadataUtils.deserializeAvroMetadata(TimelineMetadataUtils.java:183)
	at org.apache.hudi.table.action.rollback.RollbackUtils.getRollbackPlan(RollbackUtils.java:68)
	at org.apache.hudi.client.AbstractHoodieWriteClient.lambda$getPendingRollbackInfos$8(AbstractHoodieWriteClient.java:879)
	... 20 more

it seems like some 0 byte files caused , i try delete this files, become normal , but next time i cancel , submit job , this problem appear again
can you help me . thks

@waywtdcc
Copy link
Contributor

waywtdcc commented Dec 13, 2021

I also encountered this problem. Is this a bug. What version of Hudi is this? I encountered this problem in 0.9

@danny0405
Copy link
Contributor

Can you try 0.10.0 please ? Seems has been fixed in the latest version.

@waywtdcc
Copy link
Contributor

Not an Avro data file

Is the release version 0.10 okay? I see there is also an exception of the 0.10 rc version here #4204

@danny0405
Copy link
Contributor

Then it is not fixed yet. Would fire a fix then ~

@danny0405
Copy link
Contributor

Fired a fix in #4296

@waywtdcc
Copy link
Contributor

Then it is not fixed yet. Would fire a fix then ~

'hoodie.fail.on.timeline.archiving': 'false'
Can this setting avoid this error

@danny0405
Copy link
Contributor

No, archiving and rollback is different action.

@vinothchandar
Copy link
Member

cc @nsivabalan can you please look into this?

@nsivabalan
Copy link
Contributor

@wjcwin : Can you post the contents of .hoodie/. Is the rollback.requested meta file of interest is an empty file. I am trying to get a sense of why would an empty rollback.requested file will be created. Will look into the code and update here. Or if you know why this could occur, let me know.

@danny0405
Copy link
Contributor

image

Maybe here is the reason, the timeline only request the non-complete instant but the instant may be in inflight state, which is empty, we must ensure that the file we request is in requested state.

@wjcwin
Copy link
Author

wjcwin commented Dec 14, 2021

@nsivabalan ;
@danny0405 ;
my application base on flink version:1.13.1 hudi-flink-bundle version:0.10.0 .
sorry, i have cleared this files, and i remove hive-exec-2.1.1.jar , flink-connector-hive_2.11-1.13.1.jar ,hive-metastore-2.1.1.jar from FLINK_HOME/lib , this exception not happen again , maybe some Jar package conflict caused this ,but i do not know why

@danny0405
Copy link
Contributor

No, it is not jar conflict, i have fixed it in #4296, you can use the patch branch code here:
#4287, the branch has several critical fix that may be useful to you.

@wjcwin
Copy link
Author

wjcwin commented Dec 14, 2021

@danny0405 ; nice

@wjcwin wjcwin closed this as completed Dec 14, 2021
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

No branches or pull requests

5 participants