Skip to content

Conversation

1996fanrui
Copy link
Member

@1996fanrui 1996fanrui commented Sep 26, 2025

What is the purpose of the change

MapStateNullValueCheckpointingITCase failed with No checkpoint was created yet

Root Cause Analysis

Problem Location

Log analysis revealed that the checkpoint had actually completed successfully:

07:19:37,522 [jobmanager-io-thread-1] INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator [] - Completed checkpoint 1 for job b809cf46d67c23697786fd514565c737 (4464 bytes, checkpointDuration=45 ms, finalizationTime=4 ms)

However, the test code could not find the completed checkpoint when calling CommonTestUtils.getLatestCompletedCheckpointPath().

Root Cause

The problem occurs in the execution order of the CheckpointCoordinator.completePendingCheckpoint() method:

pendingCheckpoint.getCompletionFuture().complete(completedCheckpoint);
reportCompletedCheckpoint(completedCheckpoint);

Checkpoint Coordinator mechanism:

  1. A​: pendingCheckpoint.getCompletionFuture().complete(completedCheckpoint) completes the completion future first{}
  2. B​: reportCompletedCheckpoint(completedCheckpoint) updates checkpoint statistics.

Test code timeline:

  1. C: Detect future completion
  2. D: Call getLatestCompletedCheckpointPath() immediately

Usually, the execution sequence is A -> B -> C -> D, it works well.

The bug happens if execution sequence is A > C -> D -> B.

Reproduction Method

In the completePendingCheckpoint() method, inserting Thread.sleep(100) between complete() and reportCompletedCheckpoint() can reproduce this issue 100%.

Brief change log: Adjust the execution order in CheckpointCoordinator

[FLINK-38408][checkpoint] Complete the checkpoint CompletableFuture after updating statistics to ensures semantic correctness and prevent test failure

Changes:

// Update statistics first 
reportCompletedCheckpoint(completedCheckpoint);
// Complete the future later
pendingCheckpoint.getCompletionFuture().complete(completedCheckpoint);

Benefits:

  • Fundamentally eliminates race conditions
  • Ensures semantic correctness: Waiting parties are notified only when the checkpoint is fully processed

Verifying this change

  • Added testCompletionFutureCompletesAfterReporting

Does this pull request potentially affect one of the following parts:

  • Dependencies (does it add or upgrade a dependency): no
  • The public API, i.e., is any changed class annotated with @Public(Evolving): no
  • The serializers: no
  • The runtime per-record code paths (performance sensitive): no
  • Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: no
  • The S3 file system connector: no

Documentation

  • Does this pull request introduce a new feature? no

@flinkbot
Copy link
Collaborator

flinkbot commented Sep 26, 2025

CI report:

Bot commands The @flinkbot bot supports the following commands:
  • @flinkbot run azure re-run the last Azure build

@1996fanrui 1996fanrui marked this pull request as ready for review September 29, 2025 18:54
@1996fanrui 1996fanrui marked this pull request as draft September 30, 2025 09:07
@1996fanrui 1996fanrui force-pushed the 38408/no-checkpoint branch from 9afe37c to 736fe94 Compare October 2, 2025 09:00
…fter updating statistics to ensures semantic correctness and prevent test failure
@1996fanrui 1996fanrui force-pushed the 38408/no-checkpoint branch from 736fe94 to b0e8240 Compare October 2, 2025 09:06
@1996fanrui 1996fanrui marked this pull request as ready for review October 2, 2025 09:13
Copy link
Contributor

@Izeren Izeren left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you for the change @1996fanrui. Overall, LGTM, my main concern is about potential test flakiness, PTAL

lastSubsumed = null;
}

pendingCheckpoint.getCompletionFuture().complete(completedCheckpoint);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I have concerns that change like this can have potential impacts like:

  1. Deadlock / race condition if reportCompletedCheckpoint would trigger any handler that also waits on the checkpoint future before its completion (in general, unlikely situation, and should be caught by existing test)
  2. Checkpoint completion will be slightly delayed, but reporting is a quick operation, so doesn't seem to be critical
  3. If reporting throws exception it will result in checkpoint being completed exceptionally. Could we confirm that this behaviour matches the previous one?

}
});

assertThat(tracker.getReportStartedFuture().get(20, TimeUnit.SECONDS))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That is likely to end up being flaky test. Test in CI could freeze for 15min and more, so 20 seconds timeout may not be sufficient in general.
I suggest to use indefinite timeout of at least a few hours

.as("reportCompletedCheckpoint should be started soon when checkpoint is acked.")
.isNull();

for (int i = 0; i < 30; i++) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Similarly to above, I am not sure you can confirm whether expected change did not occur because of being blocked vs corresponding thread being inactive. Will be better to wait indefinitely here


tracker.getReportBlockingFuture().complete(null);

CompletedCheckpoint result = checkpointFuture.get(5, TimeUnit.SECONDS);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ditto

.as("Checkpoint future should complete after reportCompletedCheckpoint finishes")
.isNotNull();

ackTask.get(5, TimeUnit.SECONDS);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ditto

@github-actions github-actions bot added the community-reviewed PR has been reviewed by the community. label Oct 7, 2025
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

community-reviewed PR has been reviewed by the community.

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants