You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/05/16 08:14:54 UTC

[GitHub] [flink] fredia commented on a diff in pull request #19448: [FLINK-25872][state] Support restore from non-changelog checkpoint with changelog enabled in CLAIM mode

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


##########
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:
   There are a few details to ask for your opinion:
   ● should we refactor `SharedStateRegistry` to not limit `StreamStateHandle` to register(https://issues.apache.org/jira/browse/FLINK-25862) here?
   ● if yes, how should we deal with the multi-register case? For changelog states, we would like to avoid `SharedStateRegistry`  to discard multi-registered identical changelog states( https://issues.apache.org/jira/browse/FLINK-26101), which is different from other state handles.  In other words, we also need a wrapper with a different #equals method for changelog.



-- 
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-unsubscribe@flink.apache.org

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