[HUDI-6366] Prevent flink offline table service rerun completed instant#8950
Conversation
3c3905b to
379ab53
Compare
How are these duplicates generated? |
For compaction, if the first run compact Similar for clustering but worse, because the second run will generate a parquet file with a new file ID, when you read from the table again the result will be wrong. |
We should skip the completed instant, is that the behavior of current code then? // fetch the instant based on the configured execution sequence
HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline(); |
yes, but that is the client code, task fail and restart won't rerun that code. |
| public void open(Configuration parameters) throws Exception { | ||
| // no operation | ||
| isPending = StreamerUtil.createMetaClient(conf).getActiveTimeline() | ||
| .getInstantsAsStream().anyMatch(i -> clusteringInstantTime.equals(i.getTimestamp()) && !i.isCompleted()); |
There was a problem hiding this comment.
There is a method named containsInstant, can we use that?
There was a problem hiding this comment.
Maybe we can also move the code into run so there is no need to keep a class member isPending.
|
Rebased this PR to fix build error. |
d9be799 to
79add0e
Compare
79add0e to
102ae95
Compare
102ae95 to
87974a6
Compare
If flink offline table service fail after commit the compaction/clustering instant, and a restart strategy is enable, then the completed instant will be rerun.
Consequence is, if the completed instant happen to be archived, there will be a file not found error; if not archived, then there will be duplicated base files.
Change Logs
check if the compaction/clustering instant is pending in active timeline before running flink offline table service
Impact
none
Risk level (write none, low medium or high below)
none
Documentation Update
Describe any necessary documentation update if there is any new feature, config, or user-facing change
ticket number here and follow the instruction to make
changes to the website.
Contributor's checklist