-
Notifications
You must be signed in to change notification settings - Fork 13.8k
[FLINK-38408][checkpoint] Complete the checkpoint CompletableFuture after updating statistics to ensures semantic correctness and prevent test failure #27050
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: master
Are you sure you want to change the base?
Conversation
dc2613d
to
9afe37c
Compare
9afe37c
to
736fe94
Compare
…fter updating statistics to ensures semantic correctness and prevent test failure
736fe94
to
b0e8240
Compare
There was a problem hiding this 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); |
There was a problem hiding this comment.
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:
- 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) - Checkpoint completion will be slightly delayed, but reporting is a quick operation, so doesn't seem to be critical
- 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)) |
There was a problem hiding this comment.
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++) { |
There was a problem hiding this comment.
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); |
There was a problem hiding this comment.
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); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
ditto
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:
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:
flink/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
Line 1389 in 39a4628
Checkpoint Coordinator mechanism:
Test code timeline:
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:
Benefits:
Verifying this change
testCompletionFutureCompletesAfterReporting
Does this pull request potentially affect one of the following parts:
@Public(Evolving)
: noDocumentation