Thanks to visit codestin.com
Credit goes to github.com

Skip to content

[FLINK-32081][checkpoint] Compatibility between file-merging on and off across job runs #24873

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
wants to merge 2 commits into from
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
[FLINK-32081][test] Add Compatibility IT case for fileMerging
  • Loading branch information
ljz2051 committed Jun 7, 2024
commit 102191855b00444032ac758e316532593a56cfa5
Original file line number Diff line number Diff line change
Expand Up @@ -124,4 +124,16 @@ public void reusePreviousStateHandle(Collection<? extends StreamStateHandle> pre
public boolean couldReuseStateHandle(StreamStateHandle stateHandle) {
return fileMergingSnapshotManager.couldReusePreviousStateHandle(stateHandle);
}

@Override
public String toString() {
return "FsMergingCheckpointStorageLocation {"
+ "subtaskKey="
+ subtaskKey
+ ", FileMergingSnapshotManager="
+ fileMergingSnapshotManager
+ ", checkpointId="
+ checkpointId
+ "}";
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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();

Expand All @@ -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);
Expand Down
Original file line number Diff line number Diff line change
@@ -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<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 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<KeyedStateHandle> 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);
}
}