masteryhx commented on a change in pull request #18224: URL: https://github.com/apache/flink/pull/18224#discussion_r805252388
########## File path: flink-tests/src/test/java/org/apache/flink/test/checkpointing/ChangelogPeriodicMaterializationITCase.java ########## @@ -0,0 +1,734 @@ +/* + * 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.JobExecutionResult; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.api.common.functions.RichFlatMapFunction; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.state.CheckpointListener; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.state.State; +import org.apache.flink.api.common.state.StateDescriptor; +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.changelog.fs.FsStateChangelogStorageFactory; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.StateChangelogOptions; +import org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackend; +import org.apache.flink.core.fs.CloseableRegistry; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.runtime.checkpoint.CheckpointOptions; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; +import org.apache.flink.runtime.query.TaskKvStateRegistry; +import org.apache.flink.runtime.state.AbstractKeyedStateBackend; +import org.apache.flink.runtime.state.AbstractStateBackend; +import org.apache.flink.runtime.state.CheckpointStreamFactory; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.KeyGroupedInternalPriorityQueue; +import org.apache.flink.runtime.state.Keyed; +import org.apache.flink.runtime.state.KeyedStateHandle; +import org.apache.flink.runtime.state.OperatorStateBackend; +import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.PriorityComparable; +import org.apache.flink.runtime.state.SavepointResources; +import org.apache.flink.runtime.state.SnapshotResult; +import org.apache.flink.runtime.state.StateSnapshotTransformer; +import org.apache.flink.runtime.state.hashmap.HashMapStateBackend; +import org.apache.flink.runtime.state.heap.HeapPriorityQueueElement; +import org.apache.flink.runtime.state.ttl.TtlTimeProvider; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.streaming.api.environment.CheckpointConfig; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.streaming.api.functions.source.RichSourceFunction; +import org.apache.flink.streaming.api.graph.StreamGraph; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.util.Collector; +import org.apache.flink.util.TestLogger; +import org.apache.flink.util.function.FunctionWithException; + +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import javax.annotation.Nonnull; + +import java.io.File; +import java.io.IOException; +import java.io.Serializable; +import java.net.URI; +import java.nio.file.Files; +import java.nio.file.Path; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Queue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.RunnableFuture; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BooleanSupplier; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.shaded.guava30.com.google.common.collect.Iterables.get; +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** + * This verifies that checkpointing works correctly for Changelog state backend with materialized + * state / non-materialized state. + */ +@RunWith(Parameterized.class) +public class ChangelogPeriodicMaterializationITCase extends TestLogger { + + private static final int TOTAL_ELEMENTS = 20; + private static final int NUM_TASK_MANAGERS = 1; + private static final int NUM_TASK_SLOTS = 4; + private static final int NUM_SLOTS = NUM_TASK_MANAGERS * NUM_TASK_SLOTS; + + private final AbstractStateBackend delegatedStateBackend; + + private StreamExecutionEnvironment env; + + private URI checkpointURI; + + private MiniClusterWithClientResource cluster; + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Parameterized.Parameters(name = "delegated state backend type ={0}") + public static Collection<AbstractStateBackend> parameter() { + return Arrays.asList( + new HashMapStateBackend(), + new EmbeddedRocksDBStateBackend(), + new EmbeddedRocksDBStateBackend(true)); + } + + public ChangelogPeriodicMaterializationITCase(AbstractStateBackend delegatedStateBackend) { + this.delegatedStateBackend = delegatedStateBackend; + } + + @Before + public void setup() throws Exception { + cluster = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setConfiguration(configureDSTL()) + .setNumberTaskManagers(NUM_TASK_MANAGERS) + .setNumberSlotsPerTaskManager(NUM_TASK_SLOTS) + .build()); + cluster.before(); + env = StreamExecutionEnvironment.getExecutionEnvironment(); + checkpointURI = TEMPORARY_FOLDER.newFolder().toURI(); + env.enableCheckpointing(10) + .enableChangelogStateBackend(true) + .getCheckpointConfig() + .setCheckpointStorage(checkpointURI); + // init source data randomly + PreHandlingSource.initSourceData(TOTAL_ELEMENTS); + triggerDelegatedSnapshot = new AtomicBoolean(false); + } + + @After + public void teardown() throws IOException { + cluster.after(); + // clear result in sink + CollectionSink.clearExpectedResult(); + } + + /** Recovery from checkpoint only containing non-materialized state. */ + @Test + public void testNonMaterialization() { + env.setStateBackend(new DelegatedStateBackendWrapper(delegatedStateBackend, t -> t)) + .setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)); + env.configure( + new Configuration() + .set( + StateChangelogOptions.PERIODIC_MATERIALIZATION_INTERVAL, + Duration.ofMinutes(3))); + waitAndAssert( + buildStreamGraph( + new PreHandlingSource() { + @Override + protected void preHandle() throws Exception { + if (getRuntimeContext().getAttemptNumber() == 0 + && currentIndex == TOTAL_ELEMENTS / 2) { + waitUntilFalse(() -> completedCheckpointNum.get() <= 0); + throwArtificialFailure(); + } + } + })); + } + + /** Recovery from checkpoint containing non-materialized state and materialized state. */ + @Test + public void testMaterialization() { + // Maintain the RunnableFuture of SnapshotResult to make sure the materialization phase has + // been done + Queue<RunnableFuture<SnapshotResult<KeyedStateHandle>>> pendingMaterialization = + new ConcurrentLinkedQueue<>(); + SerializableFunctionWithException<RunnableFuture<SnapshotResult<KeyedStateHandle>>> + snapshotResultConsumer = + snapshotResultFuture -> { + pendingMaterialization.add(snapshotResultFuture); + return snapshotResultFuture; + }; + env.setStateBackend( + new DelegatedStateBackendWrapper( + delegatedStateBackend, snapshotResultConsumer)) + .setRestartStrategy(RestartStrategies.fixedDelayRestart(2, 0)); + env.configure( + new Configuration() + .set( + StateChangelogOptions.PERIODIC_MATERIALIZATION_INTERVAL, + Duration.ofMillis(10))); + waitAndAssert( + buildStreamGraph( + new PreHandlingSource() { + @Override + protected void preHandle() throws Exception { + if (getRuntimeContext().getAttemptNumber() == 0 + && currentIndex == TOTAL_ELEMENTS / 4) { + waitUntilFalse( + () -> + completedCheckpointNum.get() <= 0 + && pendingMaterialization.stream() + .noneMatch(Future::isDone)); + Thread.sleep(10); + pendingMaterialization.clear(); + throwArtificialFailure(); + } else if (getRuntimeContext().getAttemptNumber() == 1 + && currentIndex == TOTAL_ELEMENTS / 2) { + waitUntilFalse( + () -> + completedCheckpointNum.get() <= 1 + && pendingMaterialization.stream() + .noneMatch(Future::isDone)); + Thread.sleep(10); + throwArtificialFailure(); + } + } + })); + } + + private static AtomicBoolean triggerDelegatedSnapshot = new AtomicBoolean(); + + @Test + public void testFailedMaterialization() { + env.setStateBackend( + new DelegatedStateBackendWrapper( + delegatedStateBackend, + snapshotResultFuture -> + triggerDelegatedSnapshot.compareAndSet(false, true) + ? new RunnableFutureWithException<>( + snapshotResultFuture) + : snapshotResultFuture)) + .setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)); + env.configure( + new Configuration() + .set( + StateChangelogOptions.PERIODIC_MATERIALIZATION_INTERVAL, + Duration.ofMillis(20)) + .set(StateChangelogOptions.MATERIALIZATION_MAX_FAILURES_ALLOWED, 1)); + waitAndAssert( + buildStreamGraph( + new PreHandlingSource() { + @Override + protected void preHandle() throws Exception { + waitUntilFalse( + () -> + currentIndex >= 15 + && !triggerDelegatedSnapshot.get()); + } + })); + } + + @Test + public void testRescaleOut() throws IOException { + testRescale(NUM_SLOTS / 2, NUM_SLOTS); + } + + @Test + public void testRescaleIn() throws IOException { + testRescale(NUM_SLOTS, NUM_SLOTS / 2); + } + + private void testRescale(int firstParallelism, int secondParallelism) throws IOException { + Queue<RunnableFuture<SnapshotResult<KeyedStateHandle>>> pendingMaterialization = + new ConcurrentLinkedQueue<>(); + SerializableFunctionWithException<RunnableFuture<SnapshotResult<KeyedStateHandle>>> + snapshotResultConsumer = + snapshotResultFuture -> { + pendingMaterialization.add(snapshotResultFuture); + return snapshotResultFuture; + }; + env.setStateBackend( + new DelegatedStateBackendWrapper( + delegatedStateBackend, snapshotResultConsumer)) + .setRestartStrategy(RestartStrategies.noRestart()); + env.getCheckpointConfig() + .setExternalizedCheckpointCleanup( + CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION); + env.configure( + new Configuration() + .set( + StateChangelogOptions.PERIODIC_MATERIALIZATION_INTERVAL, + Duration.ofMillis(10))); + env.setParallelism(firstParallelism); + + StreamGraph firstStreamGraph = + buildStreamGraph( + new PreHandlingSource() { + @Override + protected void preHandle() throws Exception { + if (currentIndex == TOTAL_ELEMENTS / 2) { + waitUntilFalse( + () -> + completedCheckpointNum.get() <= 0 + && pendingMaterialization.stream() + .noneMatch(Future::isDone)); + } else if (currentIndex > TOTAL_ELEMENTS / 2) { + throwArtificialFailure(); + } + } + }); + + JobGraph firstJobGraph = firstStreamGraph.getJobGraph(); + try { + cluster.getMiniCluster().submitJob(firstJobGraph).get(); + cluster.getMiniCluster().requestJobResult(firstJobGraph.getJobID()).get(); + } catch (Exception ex) { + assertThat(ex.getCause().getCause().getMessage(), containsString("Artificial Failure")); + } + + env.setParallelism(secondParallelism); + StreamGraph streamGraph = buildStreamGraph(new PreHandlingSource()); + Optional<Path> checkpoint = + findExternalizedCheckpoint(checkpointURI.getPath(), firstJobGraph.getJobID()); + assertTrue(checkpoint.isPresent()); + streamGraph.setSavepointRestoreSettings( + SavepointRestoreSettings.forPath(checkpoint.get().toString())); + waitAndAssert(streamGraph); + } + + private static Optional<Path> findExternalizedCheckpoint(String checkpointDir, JobID jobId) + throws IOException { + try (Stream<Path> checkpoints = + Files.list(new File(checkpointDir).toPath().resolve(jobId.toString()))) { + return checkpoints + .filter(path -> path.getFileName().toString().startsWith("chk-")) + .filter( + path -> { + try (Stream<Path> checkpointFiles = Files.list(path)) { + return checkpointFiles.anyMatch( + child -> + child.getFileName() + .toString() + .contains("meta")); + } catch (IOException ignored) { + return false; + } + }) + .findAny(); + } + } + + private Configuration configureDSTL() throws IOException { + Configuration configuration = new Configuration(); + FsStateChangelogStorageFactory.configure(configuration, TEMPORARY_FOLDER.newFolder()); + return configuration; + } + + private StreamGraph buildStreamGraph(PreHandlingSource preHandlingSource) { + env.addSource(preHandlingSource) + .keyBy(element -> element) + .flatMap(new CountMapper()) + .addSink(new CollectionSink()) + .setParallelism(1); + return env.getStreamGraph(); + } + + private void waitUntilJobFinished(StreamGraph streamGraph) { + try { + JobExecutionResult jobExecutionResult = env.execute(streamGraph); + JobStatus jobStatus = + cluster.getClusterClient().getJobStatus(jobExecutionResult.getJobID()).get(); + assertSame(jobStatus, JobStatus.FINISHED); + } catch (Exception e) { + e.printStackTrace(); Review comment: You are right. Just removed try-catch -- 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]
