xccui opened a new issue, #8120:
URL: https://github.com/apache/hudi/issues/8120
Occasionally, our Flink Job was stuck on
`AbstractStreamWriteFunction.instantToWrite()` when recovering from a
checkpoint. All the `stream_write` functions fell into the dead loop below.
```
while (confirming) {
// wait condition:
// 1. there is no inflight instant
// 2. the inflight instant does not change and the checkpoint has
buffering data
if (instant == null || invalidInstant(instant, hasData)) {
// sleep for a while
timeWait.waitFor();
// refresh the inflight instant
instant = lastPendingInstant();
} else {
// the pending instant changed, that means the last instant was
committed
// successfully.
confirming = false;
}
}
```
I checked the `ckp_meta/` folder of each table, and all of them were empty.
It seems that the `stream_write` functions tried to fetch the instant files (in
a snapshot) before they were written by the `StreamWriteOperatorCoordinator`.
Not sure if it's related, but the checkpoint interval was set to only 30s since
it's a dev env.
A workaround to solve the problem is by force restarting the JobManager.
**Environment Description**
* Hudi version : 0.12.2
* Flink version : 1.14.4
* Hive version : 3.1.3
* Hadoop version : 3.3.4
* Storage (HDFS/S3/GCS..) : s3a
**Additional context**
Metadata table was not enabled.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]