masteryhx commented on code in PR #19907:
URL: https://github.com/apache/flink/pull/19907#discussion_r895363701


##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskLocalStateStoreImplTest.java:
##########
@@ -180,6 +216,19 @@ public void abortCheckpoint() throws Exception {
         checkStoredAsExpected(taskStateSnapshots, aborted + 1, chkCount);
     }
 
+    public void abortCheckpointForChangelog() throws Exception {

Review Comment:
   Maybe lost the annotation of "@Test".



##########
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogKeyedStateBackend.java:
##########
@@ -407,8 +407,9 @@ public RunnableFuture<SnapshotResult<KeyedStateHandle>> 
snapshot(
                                         
metrics.reportSnapshotResult(snapshotResult))
                         .thenApply(
                                 snapshotResult ->
-                                        SnapshotResult.of(
-                                                
snapshotResult.getJobManagerOwnedSnapshot())));
+                                        SnapshotResult.withLocalState(

Review Comment:
   There are "@Nonnull" annotations in SnapshotResult#withLocalState.
   So I think it's better to adding some check before calling it.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskLocalStateStoreImpl.java:
##########
@@ -98,6 +104,15 @@ public class TaskLocalStateStoreImpl implements 
OwnedTaskLocalStateStore {
     @GuardedBy("lock")
     private final SortedMap<Long, TaskStateSnapshot> 
storedTaskStateByCheckpointID;
 
+    /** The relationship between checkpoints. (cp1,cp3) means cp3 refer to 
cp1's state. */
+    private final Map<Long, Long> referredByCheckpointID;
+
+    /**
+     * (MaterializationID, created by checkpointID). Because local store only 
keeps one checkpoint,
+     * at most one materialization can exist at the same time.
+     */
+    private Tuple2<Long, Long> currentMaterializationID;
+
     public TaskLocalStateStoreImpl(

Review Comment:
   Maybe considering to extract some codes related to changelog to a new 
subclass?



##########
flink-tests/src/test/java/org/apache/flink/test/checkpointing/ChangelogLocalRecoveryITCase.java:
##########
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.test.checkpointing;
+
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.changelog.fs.FsStateChangelogStorageFactory;
+import org.apache.flink.configuration.CheckpointingOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.StateChangelogOptions;
+import org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackend;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.minicluster.MiniCluster;
+import org.apache.flink.runtime.state.AbstractStateBackend;
+import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.runtime.state.hashmap.HashMapStateBackend;
+import org.apache.flink.runtime.testutils.CommonTestUtils;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.environment.CheckpointConfig;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import 
org.apache.flink.test.checkpointing.ChangelogPeriodicMaterializationTestBase.CollectionSink;
+import 
org.apache.flink.test.checkpointing.ChangelogPeriodicMaterializationTestBase.CountFunction;
+import org.apache.flink.test.util.InfiniteIntegerSource;
+import org.apache.flink.test.util.MiniClusterWithClientResource;
+
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.File;
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+
+import static 
org.apache.flink.configuration.CheckpointingOptions.LOCAL_RECOVERY;
+import static 
org.apache.flink.configuration.ClusterOptions.JOB_MANAGER_PROCESS_WORKING_DIR_BASE;
+import static 
org.apache.flink.configuration.ClusterOptions.PROCESS_WORKING_DIR_BASE;
+import static 
org.apache.flink.configuration.ClusterOptions.TASK_MANAGER_PROCESS_WORKING_DIR_BASE;
+import static 
org.apache.flink.runtime.testutils.CommonTestUtils.waitForAllTaskRunning;
+
+/**
+ * Local recovery IT case for changelog. It never fails because local recovery 
is nice but not
+ * necessary.
+ */
+@RunWith(Parameterized.class)
+public class ChangelogLocalRecoveryITCase {

Review Comment:
   If the procedure of local recovery has some issues, the ITCase may still 
retain success ?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskLocalStateStoreImpl.java:
##########
@@ -213,24 +253,14 @@ public TaskStateSnapshot retrieveLocalState(long 
checkpointID) {
         }
 
         if (snapshot != null) {
-            if (LOG.isTraceEnabled()) {
-                LOG.trace(
-                        "Found registered local state for checkpoint {} in 
subtask ({} - {} - {}) : {}",
-                        checkpointID,
-                        jobID,
-                        jobVertexID,
-                        subtaskIndex,
-                        snapshot);
-            } else if (LOG.isDebugEnabled()) {
-                LOG.debug(
-                        "Found registered local state for checkpoint {} in 
subtask ({} - {} - {})",
-                        checkpointID,
-                        jobID,
-                        jobVertexID,
-                        subtaskIndex);
-            }
+            LOG.info(
+                    "Found registered local state for checkpoint {} in subtask 
({} - {} - {})",

Review Comment:
   Why removing logging "snapshot" ?



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

Reply via email to