pnowojski commented on a change in pull request #13091:
URL: https://github.com/apache/flink/pull/13091#discussion_r467832735
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointRequestDecider.java
##########
@@ -55,20 +55,23 @@
@GuardedBy("lock")
private final NavigableSet<CheckpointTriggerRequest> queuedRequests =
new TreeSet<>(checkpointTriggerRequestsComparator());
private final int maxQueuedRequests;
+ private final Supplier<Long> completionTimeSupplier;
CheckpointRequestDecider(
int maxConcurrentCheckpointAttempts,
Consumer<Long> rescheduleTrigger,
Clock clock,
long minPauseBetweenCheckpoints,
Supplier<Integer> pendingCheckpointsSizeSupplier,
+ Supplier<Long> completionTimeSupplier,
Review comment:
Those suppliers/consumers are making this class harder to follow and I
think they are showing some deeper design problem - too tight coupling between
`CheckpointCoordinator`/`CheckpointRequestDecider`.
In particular, it looks like adding `Supplier<Long> completionTimeSupplier`
reveals quite weird concurrency contract between those two. Both are using the
same lock object, and they are cross accessing the
`CheckpointCoordinator#lastCheckpointCompletionRelativeTime` field from within
the lock `CheckpointRequestDecider#lock` for read access and
`CheckpointCoordinator#lock` for the write access.
What about making `CheckpointRequestDecider` a non thread safe class, remove
the `CheckpointRequestDecider#lock` and just relay on the caller
(`CheckpointCoordinator`) to either access `CheckpointRequestDecider` from a
single thread or provide thread safety. For now `CheckpointCoordinator` would
be acquiring `CheckpointCoordinator#lock` before calling
`CheckpointRequestDecider`, while after completing threading model refactor,
the `CheckpointCoordinator#lock` would be simply removed, without a need to
modify `CheckpointRequestDecider`?
----------------------------------------------------------------
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.
For queries about this service, please contact Infrastructure at:
[email protected]