pnowojski commented on code in PR #22392:
URL: https://github.com/apache/flink/pull/22392#discussion_r1168635822
##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SubtaskCheckpointCoordinatorImpl.java:
##########
@@ -381,7 +383,7 @@ private void registerAlignmentTimer(
registerTimer.registerTask(
() -> {
try {
-
operatorChain.alignedBarrierTimeout(checkpointId);
+
operatorChain.alignedBarrierTimeout(checkpointId, metrics);
Review Comment:
Ehhh. [4] is indeed a bit complicated. I think I misspoke, I actually meant
something a bit different ([5] below).
[3] Technically it works, but I don't like how fragile the contract there
actually is, where value of the `volatile boolean unalignedCheckpoint` is only
valid if other methods/things are happening in the correct order. To fix it, we
would need something like:
[5] Change `CheckpointMetricsBuilder#unalignedCheckpoint` into some kind of
`CompletableFuture<Boolean>`. Using that, `AsyncCheckpointRunnable` could just
call `CheckpointMetricsBuilder#unalignedCheckpoint.get()`, without taking into
account if that's safe or not. However I hoped that it can be set exactly as
[3], but there is a problem. We know when to set it to `true`, but when to set
it to `false` would require quite a bit of logic :/
[6] Another potential solution would be to move out the completion of the
`AsyncCheckpointRunnable` from that the async thread, into the mailbox thread,
which would also remove some race conditions and simplify the logic. But that's
probably not worth doing for the sake of this single flag...
All in all, I'm started to think that maybe your original idea, to
approximate the `true/false` flag based on the `bytesPersistedDuringAlignment >
0` might be the lesser evil. The case when `bytesPersistedDuringAlignment == 0`
but the checkpoint barrier actually timed out in the output buffers is quite
extreme/rare, and shouldn't be that significant to the end user.
--
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]