[
https://issues.apache.org/jira/browse/FLINK-38402?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=18032718#comment-18032718
]
Zakelly Lan commented on FLINK-38402:
-------------------------------------
This is rare and I only managed to reproduce twice out of 1000 round in my
local env.
[~roman] [~srichter] Is it possible that {*}from the TM's perspective{*}, the
checkpoint notification for last checkpoint is executed after the start of the
next checkpoint, assuming no concurrent checkpoints? If so, this might indicate
a bug. I'm saying that because this may make the
`RocksIncrementalSnapshotStrategy` behaves wrong, diverging from the state
tracking maintained by the `SharedStateRegistry` on the JM side. For example,
considering a buggy event timeline for `RocksIncrementalSnapshotStrategy`:
* Checkpoint 1 finished with state handle A for 1.sst.
* Checkpoint 2 start, based on no checkpoint, so re-upload the 1.sst with
handle B
* Received Checkpoint 1's notification of finish.
* Checkpoint 2 finished. JM thinks the handle A is subsumed.
* Checkpoint 3 start, based on checkpoint 1, for 1.sst we reuse the handle A.
* Received Checkpoint 1's subsume, Checkpoint 2's notification of finish.
* Checkpoint 3 finished. JM received the handle A's placeholder and tell
'Attempt to reference unknown state', since the handle A is removed when
checkpoint 2 finished.
The log from my reproduction matches the above guess. It occurs when a
checkpoint is based on the checkpoint before the last one, even though the JM
believes the most recent checkpoint has already finished.
> EventTimeWindowCheckpointingITCase failed in test_cron_azure tests
> ------------------------------------------------------------------
>
> Key: FLINK-38402
> URL: https://issues.apache.org/jira/browse/FLINK-38402
> Project: Flink
> Issue Type: Bug
> Components: Tests
> Affects Versions: 2.2.0
> Reporter: Ruan Hang
> Priority: Major
>
> Details:
> https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=69810&view=logs&j=39d5b1d5-3b41-54dc-6458-1e2ddd1cdcf3&t=115e5c38-6efb-5006-4921-5e2851da71ef
--
This message was sent by Atlassian Jira
(v8.20.10#820010)