Zakelly commented on code in PR #24402:
URL: https://github.com/apache/flink/pull/24402#discussion_r1517422921


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskLocalStateStoreImpl.java:
##########
@@ -219,6 +219,18 @@ public TaskStateSnapshot retrieveLocalState(long 
checkpointID) {
             snapshot = loadTaskStateSnapshot(checkpointID);
         }
 
+        // Even if local recovery is disabled, it is still necessary to load 
the TaskStateSnapshot
+        // so that it can be managed by the TaskLocalStateStore.
+        if (!localRecoveryConfig.isLocalRecoveryEnabled()) {

Review Comment:
   I thought the behavior here should align with the 
`NoOpTaskLocalStateStoreImpl`, no output print. WDYT?



##########
flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java:
##########
@@ -158,8 +158,12 @@ public class CheckpointingOptions {
      *
      * <p>Local recovery currently only covers keyed state backends (including 
both the
      * EmbeddedRocksDBStateBackend and the HashMapStateBackend).
+     *
+     * @deprecated use {@link StateRecoveryOptions#LOCAL_RECOVERY} instead.

Review Comment:
   How about add `CheckpointingOptions#LOCAL_BACKUP_ENABLED` as well?



##########
flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java:
##########
@@ -35,7 +35,7 @@
 import java.util.Collection;
 import java.util.Collections;
 
-import static 
org.apache.flink.configuration.CheckpointingOptions.LOCAL_RECOVERY;
+import static 
org.apache.flink.configuration.StateRecoveryOptions.LOCAL_RECOVERY;

Review Comment:
   IIUC, this rather not be replaced without other modification. The 
`isConfigurationSupportedByChangelog` in this file should also be changed 
accordingly, considering both two new options.



##########
flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java:
##########
@@ -35,7 +35,7 @@
 import java.util.Collection;
 import java.util.Collections;
 
-import static 
org.apache.flink.configuration.CheckpointingOptions.LOCAL_RECOVERY;
+import static 
org.apache.flink.configuration.StateRecoveryOptions.LOCAL_RECOVERY;

Review Comment:
   Actually, Changelog supports local recovery, so this should be fix by 
another PR. We focus on equivalent conversion first.



##########
flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java:
##########
@@ -169,6 +173,27 @@ public class CheckpointingOptions {
                                     + "deactivated. Local recovery currently 
only covers keyed state backends "
                                     + "(including both the 
EmbeddedRocksDBStateBackend and the HashMapStateBackend).");
 
+    /**
+     * This option configures local backup for the state backend, which 
indicates whether to make
+     * backup checkpoint on local disk. If not configured, fallback to {@link
+     * StateRecoveryOptions#LOCAL_RECOVERY}. By default, local backup is 
deactivated. Local backup
+     * currently only covers keyed state backends (including both the 
EmbeddedRocksDBStateBackend
+     * and the HashMapStateBackend).
+     */
+    public static final ConfigOption<Boolean> LOCAL_BACKUP_ENABLED =

Review Comment:
   I suggest put this option at last of this class. Almost all of the options 
here will be deprecated and we can gather them on top of this class.



-- 
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