Open hbgstc123 opened 1 year ago
trigger checkpoint 3574 -> Completed checkpoint 3574 -> trigger checkpoint 3575
-> TM AppendWriteFunction fail due to Timeout(601000ms) while waiting for instant initialize
-> checkpoint 3575 failed due to TM fail
-> TM restart, send bootstrap event to StreamWriteOperatorCoordinator
-> StreamWriteOperatorCoordinator
generate new commit 20230504102505338 (trigger by checkpoint 3574 complete)
-> StreamWriteOperatorCoordinator
receive bootstrap event from TM restart, start to rollback commit 20230504102505338 and start a new instant
-> AppendWriteFunction
start to write with instant 20230504102505338(which is wrong, should write with the new instant after 20230504102505338 is rollbacked), and these new written data files of 20230504102505338 escape from rollback
-> instant 20230504102505338 rollback complete, start a new instant 20230504102509969
-> trigger checkpoint 3576 -> Completed checkpoint 3576
-> start to commit 20230504102509969
-> AppendWriteFunction
start to write with instant 20230504102509969(which is wrong)
-> delete invalid data files of 20230504102509969, part of data files wrong writing is deleted, cause write conflict error File does not exist: ...... [Lease. Holder: DFSClient_NONMAPREDUCE_-536912522_86, pending creates: 5]
-> marker dir .temp/20230504102509969 removed leaving some 20230504102509969 data file of length 0 on hdfs
-> schedule clustering c1, include those escaped data files of 20230504102509969.
-> next time clustering run instant c1, will report this error.
Seems the cause is from when TM get the wrong instant 20230504102505338 to write, so I submit a pr to fix this: https://github.com/apache/hudi/pull/8673
From my understanding, if checkpoint 3574 had been successful(but the checkpoint success event missed for the coordinator), then we should still recommit the instant. Each write task should hold a write metadata and re-send the events, yeah, but it seems you are right, the coordinator does not know that. The question is, if it is a global failover for all the write tasks, the initialization of all the tasks bootstrap event would re-trigger the re-commit of the instant.
BTW, can you help to validate whether this patch can solve the problem: https://github.com/apache/hudi/pull/8594
From my understanding, if checkpoint 3574 had been successful(but the checkpoint success event missed for the coordinator), then we should still recommit the instant. Each write task should hold a write metadata and re-send the events, yeah, but it seems you are right, the coordinator does not know that. The question is, if it is a global failover for all the write tasks, the initialization of all the tasks bootstrap event would re-trigger the re-commit of the instant.
The instant commit successfully after checkpoint 3574 complete, 20230504102505338 is a new instant generate right after
AbstractStreamWriteFunction
send the bootstrap event but before StreamWriteOperatorCoordinator
receive the bootstrap events, then 20230504102505338 rollbacked and start a new instant 20230504102509969, but AbstractStreamWriteFunction
wouldn't know it, because it think 20230504102505338 is the new instant.
BTW, can you help to validate whether this patch can solve the problem: #8594
This error is not easy to reproduce.
But I think if the coordinator restart with the operators like this PR can avoid this error too. Because new instant only start after StreamWriteOperatorCoordinator
receive all bootstrap events so AbstractStreamWriteFunction
will not get the wrong instant.
BTW, can you help to validate whether this patch can solve the problem: #8594
Sorry I have some update about this, I run a test and read about the behavior of RecreateOnResetOperatorCoordinator , it only restart the coordinator in global failure, which is not the case of my issue, my case is all subtask restart.
So https://github.com/apache/hudi/pull/8594 can not solve this issue. @danny0405
Is the same problom? https://github.com/apache/hudi/issues/8686
Maybe, it seems spark data source would include some invalid files in the reader view.
@hbgstc123 https://github.com/apache/hudi/pull/9867 here is the fix, maybe it's helpful for you.
Tested https://github.com/apache/hudi/pull/9867 locally, it can not fix this issue. Writer still can get the wrong instant.
And I draw a graph to describe how the writer get wrong instant to write
And I update my PR, it could solve this issue https://github.com/apache/hudi/pull/8673
Also find this issue in 0.13.1 branch.
@hbgstc123 still confusing about
-> delete invalid data files of 20230504102509969, part of data files wrong writing is deleted, cause write conflict error File does not exist: ...... [Lease. Holder: DFSClient_NONMAPREDUCE_-536912522_86, pending creates: 5]
-> marker dir .temp/20230504102509969 removed leaving some 20230504102509969 data file of length 0 on hdfs
would u mind leave your weChat number maybe? thanks!
hey folks. is it that, some spurious files were added after marker reconciliation (may be stray executor) and one of the spurious file size is very less and we run into the aforementioned exception?
can someone summarize the issue to me.
We are working to fix this using completion markers https://github.com/apache/hudi/pull/11593 fyi
Describe the problem you faced
A flink write hudi job, we have hdfs jitter, cause flink task to fail over, and see this error
To Reproduce
Steps to reproduce the behavior:
*have chance to reproduce
1.flink write with online clustering 2.task fail over when StreamWriteOperatorCoordinator start new instant 3.task fail over again
Expected behavior
no error
Environment Description
Hudi version : 0.12
Storage (HDFS/S3/GCS..) : HDFS
Running on Docker? (yes/no) : no
Additional context
Stacktrace