diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/filemerging/FileMergingSnapshotManagerBase.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/filemerging/FileMergingSnapshotManagerBase.java index 30e9af996738a..097e3ee7b0975 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/filemerging/FileMergingSnapshotManagerBase.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/filemerging/FileMergingSnapshotManagerBase.java @@ -644,7 +644,9 @@ public boolean couldReusePreviousStateHandle(StreamStateHandle stateHandle) { return file.getPhysicalFile().isCouldReuse(); } } - return true; + // If a stateHandle is not of the type SegmentFileStateHandle or if its corresponding file + // is not recognized by the fileMergingManager, it needs to be re-uploaded. + return false; } public void discardSingleLogicalFile(LogicalFile logicalFile, long checkpointId) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointStreamFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointStreamFactory.java index 8fa1adb2eeb95..59129758bcfd4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointStreamFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointStreamFactory.java @@ -18,6 +18,8 @@ package org.apache.flink.runtime.state; +import org.apache.flink.runtime.state.filemerging.SegmentFileStateHandle; + import java.io.IOException; import java.util.Collection; import java.util.List; @@ -89,6 +91,13 @@ default void reusePreviousStateHandle(Collection pr * @return true if it can be reused. */ default boolean couldReuseStateHandle(StreamStateHandle stateHandle) { + + // By default, the CheckpointStreamFactory doesn't support snapshot-file-merging, so the + // SegmentFileStateHandle type of stateHandle can not be reused. + if (stateHandle instanceof SegmentFileStateHandle) { + return false; + } + return true; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsMergingCheckpointStorageLocation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsMergingCheckpointStorageLocation.java index d0ae8d8645855..632f810e61e5a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsMergingCheckpointStorageLocation.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsMergingCheckpointStorageLocation.java @@ -124,4 +124,16 @@ public void reusePreviousStateHandle(Collection pre public boolean couldReuseStateHandle(StreamStateHandle stateHandle) { return fileMergingSnapshotManager.couldReusePreviousStateHandle(stateHandle); } + + @Override + public String toString() { + return "FsMergingCheckpointStorageLocation {" + + "subtaskKey=" + + subtaskKey + + ", FileMergingSnapshotManager=" + + fileMergingSnapshotManager + + ", checkpointId=" + + checkpointId + + "}"; + } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java index a7c8d4f3f899a..32c1b41b3a5f4 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java @@ -402,7 +402,18 @@ private static String runJobAndGetExternalizedCheckpoint( MiniClusterWithClientResource cluster, RestoreMode restoreMode) throws Exception { - JobGraph initialJobGraph = getJobGraph(backend, externalCheckpoint, restoreMode); + return runJobAndGetExternalizedCheckpoint( + backend, externalCheckpoint, cluster, restoreMode, new Configuration()); + } + + static String runJobAndGetExternalizedCheckpoint( + StateBackend backend, + @Nullable String externalCheckpoint, + MiniClusterWithClientResource cluster, + RestoreMode restoreMode, + Configuration jobConfig) + throws Exception { + JobGraph initialJobGraph = getJobGraph(backend, externalCheckpoint, restoreMode, jobConfig); NotifyingInfiniteTupleSource.countDownLatch = new CountDownLatch(PARALLELISM); cluster.getClusterClient().submitJob(initialJobGraph).get(); @@ -423,8 +434,12 @@ private static String runJobAndGetExternalizedCheckpoint( } private static JobGraph getJobGraph( - StateBackend backend, @Nullable String externalCheckpoint, RestoreMode restoreMode) { - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StateBackend backend, + @Nullable String externalCheckpoint, + RestoreMode restoreMode, + Configuration jobConfig) { + final StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(jobConfig); env.enableCheckpointing(500); env.setStateBackend(backend); diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SnapshotFileMergingCompatibilityITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SnapshotFileMergingCompatibilityITCase.java new file mode 100644 index 0000000000000..a6e9783236ff9 --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SnapshotFileMergingCompatibilityITCase.java @@ -0,0 +1,188 @@ +/* + * 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.checkpoint.OperatorState; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.checkpoint.metadata.CheckpointMetadata; +import org.apache.flink.runtime.state.IncrementalRemoteKeyedStateHandle; +import org.apache.flink.runtime.state.KeyedStateHandle; +import org.apache.flink.runtime.state.filemerging.SegmentFileStateHandle; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.test.util.TestUtils; +import org.apache.flink.util.TestLogger; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; + +import java.io.IOException; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; + +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 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 firstCheckpoint; + try { + firstCheckpoint = + runJobAndGetExternalizedCheckpoint( + stateBackend1, null, firstCluster, restoreMode, config); + assertThat(firstCheckpoint).isNotNull(); + verifyStateHandleType(firstCheckpoint, firstFileMergingSwitch); + } 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(); + String secondCheckpoint; + try { + secondCheckpoint = + runJobAndGetExternalizedCheckpoint( + stateBackend2, firstCheckpoint, secondCluster, restoreMode, config); + assertThat(secondCheckpoint).isNotNull(); + verifyStateHandleType(secondCheckpoint, secondFileMergingSwitch); + } finally { + secondCluster.after(); + } + + EmbeddedRocksDBStateBackend stateBackend3 = new EmbeddedRocksDBStateBackend(); + stateBackend3.configure(config, Thread.currentThread().getContextClassLoader()); + MiniClusterWithClientResource thirdCluster = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setConfiguration(config) + .setNumberTaskManagers(3) + .setNumberSlotsPerTaskManager(2) + .build()); + thirdCluster.before(); + try { + String thirdCheckpoint = + runJobAndGetExternalizedCheckpoint( + stateBackend3, secondCheckpoint, thirdCluster, restoreMode, config); + assertThat(thirdCheckpoint).isNotNull(); + verifyStateHandleType(thirdCheckpoint, secondFileMergingSwitch); + } finally { + thirdCluster.after(); + } + } + + private void verifyStateHandleType(String checkpointPath, boolean fileMergingEnabled) + throws IOException { + CheckpointMetadata metadata = TestUtils.loadCheckpointMetadata(checkpointPath); + boolean hasKeyedState = false; + for (OperatorState operatorState : metadata.getOperatorStates()) { + for (OperatorSubtaskState subtaskState : operatorState.getStates()) { + // Check keyed state handle + List keyedStateHandles = + new ArrayList<>(subtaskState.getManagedKeyedState()); + keyedStateHandles.addAll(subtaskState.getRawKeyedState()); + for (KeyedStateHandle keyedStateHandle : keyedStateHandles) { + Assertions.assertInstanceOf( + IncrementalRemoteKeyedStateHandle.class, keyedStateHandle); + ((IncrementalRemoteKeyedStateHandle) keyedStateHandle) + .streamSubHandles() + .forEach( + handle -> { + Assertions.assertEquals( + fileMergingEnabled, + handle instanceof SegmentFileStateHandle); + }); + hasKeyedState = true; + } + } + } + Assertions.assertTrue(hasKeyedState); + } +}