rkhachatryan commented on code in PR #19448:
URL: https://github.com/apache/flink/pull/19448#discussion_r873863463


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryImpl.java:
##########
@@ -47,23 +50,33 @@ public class SharedStateRegistryImpl implements 
SharedStateRegistry {
     /** All registered state objects by an artificial key */
     private final Map<SharedStateRegistryKey, SharedStateEntry> 
registeredStates;
 
+    /** All registered checkpoints */
+    private final Set<CompletedCheckpoint> registeredCheckpoints;

Review Comment:
   I realized that this field is now (partially) duplicating the 
`DefaultCompletedCheckpointStore.completedCheckpoints`.
   
   Do you think it could/should be refactored so that 
`CompletedCheckpointStore` "transfers the ownership" of subsumed checkpoints to 
`SharedStateRegistry`? E.g. by passing them to `unregisterUnusedState`.
   
   Another issue: `CompletedCheckpoint` doesn't have `equals/hashCode` 
overriden, so it's better to use `Map<Long, CompletedCheckpoint>`.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java:
##########
@@ -227,6 +229,14 @@ private static void registerSharedState(
         for (KeyedStateHandle stateHandle : stateHandles) {
             if (stateHandle != null) {
                 stateHandle.registerSharedStates(sharedStateRegistry, 
checkpointID);
+                // Registering state handle to the given sharedStateRegistry 
serves two purposes:
+                // 1. let sharedStateRegistry be responsible for cleaning the 
state handle,

Review Comment:
   I think this can be avoided because checkpoint will be discarded once no 
state is in use, which will also discard the private state.
   And it's better to keep discarding there (in `CheckpointCleaner`), rather 
than in `SharedStateRegistry` because the back-pressure mechanism is better 
there.
   
   So I propose to have an empty `discardState` method in wrapper.
   (That would also allow to replace `keyedStateHandle` in it with only an ID)



##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryImpl.java:
##########
@@ -137,12 +150,26 @@ public StreamStateHandle registerReference(
         return entry.stateHandle;
     }
 
+    @Override
+    public void registerCompletedCheckpoint(CompletedCheckpoint checkpoint) {
+        synchronized (registeredCheckpoints) {
+            LOG.trace("Register checkpoint {}.", checkpoint.getCheckpointID());
+            registeredCheckpoints.add(checkpoint);
+        }
+    }
+
+    @Override
+    public void setPostCleanAction(Runnable postCleanAction) {
+        this.postCleanAction = postCleanAction;

Review Comment:
   I'd prefer this action to be passed as an argument to 
`unregisterUnusedState` rather than having a mutable field, because
   1. on shutdown, it should not be called, right? with an argument, it's 
easier to pass a no-op callback
   2. it's easy to forget to call this setter and more difficult to understand 
**when** is it called



##########
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/DefaultCompletedCheckpointStore.java:
##########
@@ -137,23 +134,19 @@ public CompletedCheckpoint 
addCheckpointAndSubsumeOldestOne(
 
         completedCheckpoints.addLast(checkpoint);
 
+        // Register checkpoint to SharedStateRegistry, and checkpoint would be 
discarded during
+        // unregistering.
+        getSharedStateRegistry().registerCompletedCheckpoint(checkpoint);
+
+        // Remove completed checkpoint from queue and 
checkpointStateHandleStore, not discard.
         Optional<CompletedCheckpoint> subsume =
                 CheckpointSubsumeHelper.subsume(
                         completedCheckpoints,
                         maxNumberOfCheckpointsToRetain,
-                        completedCheckpoint ->
-                                tryRemoveCompletedCheckpoint(
-                                        completedCheckpoint,
-                                        
completedCheckpoint.shouldBeDiscardedOnSubsume(),
-                                        checkpointsCleaner,
-                                        postCleanup));
+                        completedCheckpoint -> 
tryRemove(completedCheckpoint.getCheckpointID()));

Review Comment:
   In `shutdown`, the store actually calls `checkpointCleaner.cleanCheckpoint`, 
as opposed to adding a checkpoint (where it only removes it from ZK).
   
   After that `shutdown` calls `unregisterUnusedState` (line 200).
   I wonder if there will be a duplicate call to cleanup a checkpoint.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java:
##########
@@ -227,6 +229,14 @@ private static void registerSharedState(
         for (KeyedStateHandle stateHandle : stateHandles) {
             if (stateHandle != null) {
                 stateHandle.registerSharedStates(sharedStateRegistry, 
checkpointID);
+                // Registering state handle to the given sharedStateRegistry 
serves two purposes:
+                // 1. let sharedStateRegistry be responsible for cleaning the 
state handle,
+                // 2. update the status of the checkpoint in 
sharedStateRegistry to which the state
+                // handle belongs
+                sharedStateRegistry.registerReference(
+                        new 
SharedStateRegistryKey(stateHandle.getStateHandleId().getKeyString()),
+                        new StreamStateHandleWrapper(stateHandle),

Review Comment:
   As per the comment above, I think there should be a wrapper to have an empty 
`discardState`. So the interface may stay the same (FLINK-25862 not needed here 
IMO).



##########
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java:
##########
@@ -227,6 +229,14 @@ private static void registerSharedState(
         for (KeyedStateHandle stateHandle : stateHandles) {
             if (stateHandle != null) {
                 stateHandle.registerSharedStates(sharedStateRegistry, 
checkpointID);
+                // Registering state handle to the given sharedStateRegistry 
serves two purposes:
+                // 1. let sharedStateRegistry be responsible for cleaning the 
state handle,
+                // 2. update the status of the checkpoint in 
sharedStateRegistry to which the state
+                // handle belongs
+                sharedStateRegistry.registerReference(
+                        new 
SharedStateRegistryKey(stateHandle.getStateHandleId().getKeyString()),
+                        new StreamStateHandleWrapper(stateHandle),

Review Comment:
   Another question: can we get rid of the same call in 
`ChangelogStateBackendHandle`? 
   In fact, they will try to use the same key, won't they?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryImpl.java:
##########
@@ -156,14 +183,38 @@ public void unregisterUnusedState(long 
lowestCheckpointID) {
                         subsumed.add(entry.stateHandle);
                     }
                     it.remove();
+                } else {
+                    checkpointInUse.add(entry.createdByCheckpointID);
                 }
             }
         }
-
         LOG.trace("Discard {} state asynchronously", subsumed.size());
         for (StreamStateHandle handle : subsumed) {
             scheduleAsyncDelete(handle);
         }
+
+        // One checkpoint should be discarded after all its private states 
have been discarded.
+        synchronized (registeredCheckpoints) {
+            Iterator<CompletedCheckpoint> iterator = 
registeredCheckpoints.iterator();
+            while (iterator.hasNext()) {
+                CompletedCheckpoint checkpoint = iterator.next();
+                if (checkpoint.getCheckpointID() < lowestCheckpointID
+                        && 
!checkpointInUse.contains(checkpoint.getCheckpointID())) {
+                    iterator.remove();
+                    LOG.trace("Discard checkpoint {}.", 
checkpoint.getCheckpointID());
+
+                    try {
+                        checkpointsCleaner.cleanCheckpoint(
+                                checkpoint,
+                                checkpoint.shouldBeDiscardedOnSubsume(),
+                                postCleanAction,
+                                asyncDisposalExecutor);
+                    } catch (Exception e) {
+                        LOG.warn("Fail to discard the old checkpoint.", 
checkpoint);

Review Comment:
   `{}` is missing in the message.



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to