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

Skip to content

Commit 14113f5

Browse files
ljz2051snuyanzin
authored andcommitted
[FLINK-32081][test] Add Compatibility IT case for fileMerging
This fixes apache#24873
1 parent 23f15ee commit 14113f5

File tree

3 files changed

+218
-3
lines changed

3 files changed

+218
-3
lines changed

flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsMergingCheckpointStorageLocation.java

+12
Original file line numberDiff line numberDiff line change
@@ -124,4 +124,16 @@ public void reusePreviousStateHandle(Collection<? extends StreamStateHandle> pre
124124
public boolean couldReuseStateHandle(StreamStateHandle stateHandle) {
125125
return fileMergingSnapshotManager.couldReusePreviousStateHandle(stateHandle);
126126
}
127+
128+
@Override
129+
public String toString() {
130+
return "FsMergingCheckpointStorageLocation {"
131+
+ "subtaskKey="
132+
+ subtaskKey
133+
+ ", FileMergingSnapshotManager="
134+
+ fileMergingSnapshotManager
135+
+ ", checkpointId="
136+
+ checkpointId
137+
+ "}";
138+
}
127139
}

flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java

+18-3
Original file line numberDiff line numberDiff line change
@@ -402,7 +402,18 @@ private static String runJobAndGetExternalizedCheckpoint(
402402
MiniClusterWithClientResource cluster,
403403
RestoreMode restoreMode)
404404
throws Exception {
405-
JobGraph initialJobGraph = getJobGraph(backend, externalCheckpoint, restoreMode);
405+
return runJobAndGetExternalizedCheckpoint(
406+
backend, externalCheckpoint, cluster, restoreMode, new Configuration());
407+
}
408+
409+
static String runJobAndGetExternalizedCheckpoint(
410+
StateBackend backend,
411+
@Nullable String externalCheckpoint,
412+
MiniClusterWithClientResource cluster,
413+
RestoreMode restoreMode,
414+
Configuration jobConfig)
415+
throws Exception {
416+
JobGraph initialJobGraph = getJobGraph(backend, externalCheckpoint, restoreMode, jobConfig);
406417
NotifyingInfiniteTupleSource.countDownLatch = new CountDownLatch(PARALLELISM);
407418
cluster.getClusterClient().submitJob(initialJobGraph).get();
408419

@@ -423,8 +434,12 @@ private static String runJobAndGetExternalizedCheckpoint(
423434
}
424435

425436
private static JobGraph getJobGraph(
426-
StateBackend backend, @Nullable String externalCheckpoint, RestoreMode restoreMode) {
427-
final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
437+
StateBackend backend,
438+
@Nullable String externalCheckpoint,
439+
RestoreMode restoreMode,
440+
Configuration jobConfig) {
441+
final StreamExecutionEnvironment env =
442+
StreamExecutionEnvironment.getExecutionEnvironment(jobConfig);
428443

429444
env.enableCheckpointing(500);
430445
env.setStateBackend(backend);
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,188 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
19+
package org.apache.flink.test.checkpointing;
20+
21+
import org.apache.flink.configuration.CheckpointingOptions;
22+
import org.apache.flink.configuration.Configuration;
23+
import org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackend;
24+
import org.apache.flink.core.execution.RestoreMode;
25+
import org.apache.flink.runtime.checkpoint.OperatorState;
26+
import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
27+
import org.apache.flink.runtime.checkpoint.metadata.CheckpointMetadata;
28+
import org.apache.flink.runtime.state.IncrementalRemoteKeyedStateHandle;
29+
import org.apache.flink.runtime.state.KeyedStateHandle;
30+
import org.apache.flink.runtime.state.filemerging.SegmentFileStateHandle;
31+
import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
32+
import org.apache.flink.test.util.MiniClusterWithClientResource;
33+
import org.apache.flink.test.util.TestUtils;
34+
import org.apache.flink.util.TestLogger;
35+
36+
import org.junit.jupiter.api.Assertions;
37+
import org.junit.jupiter.api.io.TempDir;
38+
import org.junit.jupiter.params.ParameterizedTest;
39+
import org.junit.jupiter.params.provider.MethodSource;
40+
41+
import java.io.IOException;
42+
import java.nio.file.Path;
43+
import java.util.ArrayList;
44+
import java.util.Arrays;
45+
import java.util.Collection;
46+
import java.util.List;
47+
48+
import static org.apache.flink.test.checkpointing.ResumeCheckpointManuallyITCase.runJobAndGetExternalizedCheckpoint;
49+
import static org.assertj.core.api.Assertions.assertThat;
50+
51+
/**
52+
* FileMerging Compatibility IT case which tests recovery from a checkpoint created in different
53+
* fileMerging mode (i.e. fileMerging enabled/disabled).
54+
*/
55+
public class SnapshotFileMergingCompatibilityITCase extends TestLogger {
56+
57+
public static Collection<Object[]> parameters() {
58+
return Arrays.asList(
59+
new Object[][] {
60+
{RestoreMode.CLAIM, true},
61+
{RestoreMode.CLAIM, false},
62+
{RestoreMode.NO_CLAIM, true},
63+
{RestoreMode.NO_CLAIM, false}
64+
});
65+
}
66+
67+
@ParameterizedTest(name = "RestoreMode = {0}, fileMergingAcrossBoundary = {1}")
68+
@MethodSource("parameters")
69+
public void testSwitchFromDisablingToEnablingFileMerging(
70+
RestoreMode restoreMode, boolean fileMergingAcrossBoundary, @TempDir Path checkpointDir)
71+
throws Exception {
72+
testSwitchingFileMerging(
73+
checkpointDir, false, true, restoreMode, fileMergingAcrossBoundary);
74+
}
75+
76+
@ParameterizedTest(name = "RestoreMode = {0}, fileMergingAcrossBoundary = {1}")
77+
@MethodSource("parameters")
78+
public void testSwitchFromEnablingToDisablingFileMerging(
79+
RestoreMode restoreMode, boolean fileMergingAcrossBoundary, @TempDir Path checkpointDir)
80+
throws Exception {
81+
testSwitchingFileMerging(
82+
checkpointDir, true, false, restoreMode, fileMergingAcrossBoundary);
83+
}
84+
85+
private void testSwitchingFileMerging(
86+
Path checkpointDir,
87+
boolean firstFileMergingSwitch,
88+
boolean secondFileMergingSwitch,
89+
RestoreMode restoreMode,
90+
boolean fileMergingAcrossBoundary)
91+
throws Exception {
92+
final Configuration config = new Configuration();
93+
config.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir.toUri().toString());
94+
config.set(CheckpointingOptions.INCREMENTAL_CHECKPOINTS, true);
95+
config.set(CheckpointingOptions.FILE_MERGING_ACROSS_BOUNDARY, fileMergingAcrossBoundary);
96+
config.set(CheckpointingOptions.FILE_MERGING_ENABLED, firstFileMergingSwitch);
97+
MiniClusterWithClientResource firstCluster =
98+
new MiniClusterWithClientResource(
99+
new MiniClusterResourceConfiguration.Builder()
100+
.setConfiguration(config)
101+
.setNumberTaskManagers(2)
102+
.setNumberSlotsPerTaskManager(2)
103+
.build());
104+
EmbeddedRocksDBStateBackend stateBackend1 = new EmbeddedRocksDBStateBackend();
105+
stateBackend1.configure(config, Thread.currentThread().getContextClassLoader());
106+
firstCluster.before();
107+
String firstCheckpoint;
108+
try {
109+
firstCheckpoint =
110+
runJobAndGetExternalizedCheckpoint(
111+
stateBackend1, null, firstCluster, restoreMode, config);
112+
assertThat(firstCheckpoint).isNotNull();
113+
verifyStateHandleType(firstCheckpoint, firstFileMergingSwitch);
114+
} finally {
115+
firstCluster.after();
116+
}
117+
118+
config.set(CheckpointingOptions.FILE_MERGING_ENABLED, secondFileMergingSwitch);
119+
EmbeddedRocksDBStateBackend stateBackend2 = new EmbeddedRocksDBStateBackend();
120+
stateBackend2.configure(config, Thread.currentThread().getContextClassLoader());
121+
MiniClusterWithClientResource secondCluster =
122+
new MiniClusterWithClientResource(
123+
new MiniClusterResourceConfiguration.Builder()
124+
.setConfiguration(config)
125+
.setNumberTaskManagers(2)
126+
.setNumberSlotsPerTaskManager(2)
127+
.build());
128+
secondCluster.before();
129+
String secondCheckpoint;
130+
try {
131+
secondCheckpoint =
132+
runJobAndGetExternalizedCheckpoint(
133+
stateBackend2, firstCheckpoint, secondCluster, restoreMode, config);
134+
assertThat(secondCheckpoint).isNotNull();
135+
verifyStateHandleType(secondCheckpoint, secondFileMergingSwitch);
136+
} finally {
137+
secondCluster.after();
138+
}
139+
140+
EmbeddedRocksDBStateBackend stateBackend3 = new EmbeddedRocksDBStateBackend();
141+
stateBackend3.configure(config, Thread.currentThread().getContextClassLoader());
142+
MiniClusterWithClientResource thirdCluster =
143+
new MiniClusterWithClientResource(
144+
new MiniClusterResourceConfiguration.Builder()
145+
.setConfiguration(config)
146+
.setNumberTaskManagers(3)
147+
.setNumberSlotsPerTaskManager(2)
148+
.build());
149+
thirdCluster.before();
150+
try {
151+
String thirdCheckpoint =
152+
runJobAndGetExternalizedCheckpoint(
153+
stateBackend3, secondCheckpoint, thirdCluster, restoreMode, config);
154+
assertThat(thirdCheckpoint).isNotNull();
155+
verifyStateHandleType(thirdCheckpoint, secondFileMergingSwitch);
156+
} finally {
157+
thirdCluster.after();
158+
}
159+
}
160+
161+
private void verifyStateHandleType(String checkpointPath, boolean fileMergingEnabled)
162+
throws IOException {
163+
CheckpointMetadata metadata = TestUtils.loadCheckpointMetadata(checkpointPath);
164+
boolean hasKeyedState = false;
165+
for (OperatorState operatorState : metadata.getOperatorStates()) {
166+
for (OperatorSubtaskState subtaskState : operatorState.getStates()) {
167+
// Check keyed state handle
168+
List<KeyedStateHandle> keyedStateHandles =
169+
new ArrayList<>(subtaskState.getManagedKeyedState());
170+
keyedStateHandles.addAll(subtaskState.getRawKeyedState());
171+
for (KeyedStateHandle keyedStateHandle : keyedStateHandles) {
172+
Assertions.assertInstanceOf(
173+
IncrementalRemoteKeyedStateHandle.class, keyedStateHandle);
174+
((IncrementalRemoteKeyedStateHandle) keyedStateHandle)
175+
.streamSubHandles()
176+
.forEach(
177+
handle -> {
178+
Assertions.assertEquals(
179+
fileMergingEnabled,
180+
handle instanceof SegmentFileStateHandle);
181+
});
182+
hasKeyedState = true;
183+
}
184+
}
185+
}
186+
Assertions.assertTrue(hasKeyedState);
187+
}
188+
}

0 commit comments

Comments
 (0)