masteryhx commented on a change in pull request #18224: URL: https://github.com/apache/flink/pull/18224#discussion_r818474384
########## File path: flink-tests/src/test/java/org/apache/flink/test/checkpointing/ChangelogPeriodicMaterializationTestBase.java ########## @@ -0,0 +1,569 @@ +/* + * 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.JobID; +import org.apache.flink.api.common.JobSubmissionResult; +import org.apache.flink.api.common.functions.RichMapFunction; +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.CheckpointingOptions; +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.checkpoint.metadata.CheckpointMetadata; +import org.apache.flink.runtime.clusterframework.ApplicationStatus; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmaster.JobResult; +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.StateBackend; +import org.apache.flink.runtime.state.StateSnapshotTransformer; +import org.apache.flink.runtime.state.changelog.ChangelogStateBackendHandle; +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.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.streaming.api.functions.source.RichSourceFunction; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.testutils.junit.SharedObjects; +import org.apache.flink.util.AbstractID; +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.Rule; +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.nio.file.Path; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.RunnableFuture; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicInteger; +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.apache.flink.test.util.TestUtils.findAllSortedExternalizedCheckpoint; +import static org.apache.flink.test.util.TestUtils.loadCheckpointMetadata; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertSame; + +/** Base class for tests related to period materialization of ChangelogStateBackend. */ +@RunWith(Parameterized.class) +public abstract class ChangelogPeriodicMaterializationTestBase extends TestLogger { + + private static final int NUM_TASK_MANAGERS = 1; + private static final int NUM_TASK_SLOTS = 4; + protected static final int NUM_SLOTS = NUM_TASK_MANAGERS * NUM_TASK_SLOTS; + protected static final int TOTAL_ELEMENTS = 10_000; + + protected final AbstractStateBackend delegatedStateBackend; + + protected MiniClusterWithClientResource cluster; + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule public final SharedObjects sharedObjects = SharedObjects.create(); + + @Parameterized.Parameters(name = "delegated state backend type ={0}") + public static Collection<AbstractStateBackend> parameter() { + return Arrays.asList( + new HashMapStateBackend(), + new EmbeddedRocksDBStateBackend(true), + new EmbeddedRocksDBStateBackend(false)); + } + + public ChangelogPeriodicMaterializationTestBase(AbstractStateBackend delegatedStateBackend) { + this.delegatedStateBackend = delegatedStateBackend; + } + + @Before + public void setup() throws Exception { + cluster = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setConfiguration(configure()) + .setNumberTaskManagers(NUM_TASK_MANAGERS) + .setNumberSlotsPerTaskManager(NUM_TASK_SLOTS) + .build()); + cluster.before(); + cluster.getMiniCluster().overrideRestoreModeForChangelogStateBackend(); + } + + @After + public void tearDown() throws IOException { + cluster.after(); + // clear result in sink + CollectionSink.clearExpectedResult(); + } + + protected StreamExecutionEnvironment getEnv( + StateBackend stateBackend, + File checkpointFile, + long checkpointInterval, + int restartAttempts, + int materializationInterval, + int materializationMaxFailure) { + StreamExecutionEnvironment env = + getEnv(stateBackend, checkpointFile, checkpointInterval, restartAttempts); + env.configure( + new Configuration() + .set( + StateChangelogOptions.PERIODIC_MATERIALIZATION_INTERVAL, + Duration.ofMillis(materializationInterval)) + .set( + StateChangelogOptions.MATERIALIZATION_MAX_FAILURES_ALLOWED, + materializationMaxFailure)); + return env; + } + + protected StreamExecutionEnvironment getEnv( + StateBackend stateBackend, + File checkpointFile, + long checkpointInterval, + int restartAttempts) { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.enableCheckpointing(checkpointInterval) + .enableChangelogStateBackend(true) + .getCheckpointConfig() + .setCheckpointStorage(checkpointFile.toURI()); + env.setStateBackend(stateBackend) + .setRestartStrategy(RestartStrategies.fixedDelayRestart(restartAttempts, 0)); + return env; + } + + protected StreamExecutionEnvironment getEnv(StateBackend stateBackend) throws IOException { + return getEnv(stateBackend, TEMPORARY_FOLDER.newFolder(), 10, 1); + } + + protected JobGraph buildJobGraph( + StreamExecutionEnvironment env, ControlledSource controlledSource, JobID jobId) { + env.addSource(controlledSource) + .keyBy(element -> element) + .map(new CountMapper()) + .addSink(new CollectionSink()) + .setParallelism(1); + return env.getStreamGraph().getJobGraph(jobId); + } + + protected void waitAndAssert(JobGraph jobGraph) throws Exception { + waitUntilJobFinished(jobGraph); + assertEquals(CollectionSink.getActualResult(), ControlledSource.getExpectedResult()); + } + + protected JobID generateJobID() { + byte[] randomBytes = new byte[AbstractID.SIZE]; + ThreadLocalRandom.current().nextBytes(randomBytes); + return JobID.fromByteArray(randomBytes); + } + + protected static List<Long> getAllMaterializationId(File checkpointFile, JobID jobID) + throws IOException { + List<Path> checkpointPaths = findAllSortedExternalizedCheckpoint(checkpointFile, jobID); + for (Path checkpointPath : checkpointPaths) { + CheckpointMetadata checkpointMetadata = + loadCheckpointMetadata(checkpointPath.toString()); + List<Long> materializationIds = + checkpointMetadata.getOperatorStates().stream() + .flatMap(operatorState -> operatorState.getStates().stream()) + .flatMap( + operatorSubtaskState -> + operatorSubtaskState.getManagedKeyedState().stream()) + .map(keyedStateHandle -> (ChangelogStateBackendHandle) keyedStateHandle) + .filter( + changelogStateBackendHandle -> + !changelogStateBackendHandle + .getMaterializedStateHandles() + .isEmpty()) + .map(ChangelogStateBackendHandle::getMaterializationID) + .collect(Collectors.toList()); + if (!materializationIds.isEmpty()) { + return materializationIds; + } Review comment: Replace `findExternalizedCheckpoint` with `getMostRecentCompletedCheckpointMaybe` -- 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]
