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


##########
flink-tests/src/test/java/org/apache/flink/test/checkpointing/SnapshotFileMergingCompatibilityITCase.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.configuration.CheckpointingOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackend;
+import org.apache.flink.core.execution.RestoreMode;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.test.util.MiniClusterWithClientResource;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.jupiter.api.io.TempDir;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
+
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Collection;
+
+import static 
org.apache.flink.test.checkpointing.ResumeCheckpointManuallyITCase.runJobAndGetExternalizedCheckpoint;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * FileMerging Compatibility IT case which tests recovery from a checkpoint 
created in different
+ * fileMerging mode (i.e. fileMerging enabled/disabled).
+ */
+public class SnapshotFileMergingCompatibilityITCase extends TestLogger {
+
+    public static Collection<Object[]> parameters() {
+        return Arrays.asList(
+                new Object[][] {
+                    {RestoreMode.CLAIM, true},
+                    {RestoreMode.CLAIM, false},
+                    {RestoreMode.NO_CLAIM, true},
+                    {RestoreMode.NO_CLAIM, false}
+                });
+    }
+
+    @ParameterizedTest(name = "RestoreMode = {0}, fileMergingAcrossBoundary = 
{1}")
+    @MethodSource("parameters")
+    public void testSwitchFromDisablingToEnablingFileMerging(
+            RestoreMode restoreMode, boolean fileMergingAcrossBoundary, 
@TempDir Path checkpointDir)
+            throws Exception {
+        testSwitchingFileMerging(
+                checkpointDir, false, true, restoreMode, 
fileMergingAcrossBoundary);
+    }
+
+    @ParameterizedTest(name = "RestoreMode = {0}, fileMergingAcrossBoundary = 
{1}")
+    @MethodSource("parameters")
+    public void testSwitchFromEnablingToDisablingFileMerging(
+            RestoreMode restoreMode, boolean fileMergingAcrossBoundary, 
@TempDir Path checkpointDir)
+            throws Exception {
+        testSwitchingFileMerging(
+                checkpointDir, true, false, restoreMode, 
fileMergingAcrossBoundary);
+    }
+
+    private void testSwitchingFileMerging(
+            Path checkpointDir,
+            boolean firstFileMergingSwitch,
+            boolean secondFileMergingSwitch,
+            RestoreMode restoreMode,
+            boolean fileMergingAcrossBoundary)
+            throws Exception {
+        final Configuration config = new Configuration();
+        config.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, 
checkpointDir.toUri().toString());
+        config.set(CheckpointingOptions.INCREMENTAL_CHECKPOINTS, true);
+        config.set(CheckpointingOptions.FILE_MERGING_ACROSS_BOUNDARY, 
fileMergingAcrossBoundary);
+        config.set(CheckpointingOptions.FILE_MERGING_ENABLED, 
firstFileMergingSwitch);
+        MiniClusterWithClientResource firstCluster =
+                new MiniClusterWithClientResource(
+                        new MiniClusterResourceConfiguration.Builder()
+                                .setConfiguration(config)
+                                .setNumberTaskManagers(2)
+                                .setNumberSlotsPerTaskManager(2)
+                                .build());
+        EmbeddedRocksDBStateBackend stateBackend1 = new 
EmbeddedRocksDBStateBackend();
+        stateBackend1.configure(config, 
Thread.currentThread().getContextClassLoader());
+        firstCluster.before();
+        String externalCheckpoint;
+        try {
+            externalCheckpoint =
+                    runJobAndGetExternalizedCheckpoint(
+                            stateBackend1, null, firstCluster, restoreMode, 
config);
+            assertThat(externalCheckpoint).isNotNull();
+        } finally {
+            firstCluster.after();
+        }
+
+        config.set(CheckpointingOptions.FILE_MERGING_ENABLED, 
secondFileMergingSwitch);
+        EmbeddedRocksDBStateBackend stateBackend2 = new 
EmbeddedRocksDBStateBackend();
+        stateBackend2.configure(config, 
Thread.currentThread().getContextClassLoader());
+        MiniClusterWithClientResource secondCluster =
+                new MiniClusterWithClientResource(
+                        new MiniClusterResourceConfiguration.Builder()
+                                .setConfiguration(config)
+                                .setNumberTaskManagers(2)
+                                .setNumberSlotsPerTaskManager(2)
+                                .build());
+        secondCluster.before();
+        try {
+            String secondCheckpoint =
+                    runJobAndGetExternalizedCheckpoint(
+                            stateBackend2, externalCheckpoint, secondCluster, 
restoreMode, config);
+            assertThat(secondCheckpoint).isNotNull();
+        } finally {
+            secondCluster.after();
+        }

Review Comment:
   How about using `CheckpointMetadata metadata = 
TestUtils.loadCheckpointMetadata(firstCheckpoint);` and visit all the state 
handle within it?
   
   And I found there is no managed keyed state in current tests, maybe you 
should fill some data in.



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