rkhachatryan commented on a change in pull request #18224: URL: https://github.com/apache/flink/pull/18224#discussion_r805769118
########## File path: flink-tests/src/test/java/org/apache/flink/test/checkpointing/ChangelogPeriodicMaterializationTestBase.java ########## @@ -0,0 +1,485 @@ +/* + * 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.RichMapFunction; +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.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.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.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.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.rules.Timeout; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import javax.annotation.Nonnull; + +import java.io.IOException; +import java.io.Serializable; +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.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.RunnableFuture; +import java.util.concurrent.ThreadLocalRandom; +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.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; + + // Maintain the RunnableFuture of SnapshotResult to make sure the materialization phase has + // been done + protected static final Queue<RunnableFuture<SnapshotResult<KeyedStateHandle>>> + PENDING_MATERIALIZATION = new ConcurrentLinkedQueue<>(); + + @Rule public Timeout globalTimeout = Timeout.seconds(60); + + @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(true), + new EmbeddedRocksDBStateBackend()); + } + + public ChangelogPeriodicMaterializationTestBase(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(); + cluster.getMiniCluster().overrideRestoreModeForChangelogStateBackend(); + + // init source data randomly + ControlledSource.initSourceData(TOTAL_ELEMENTS); + } + + @After + public void tearDown() throws IOException { + cluster.after(); + // clear result in sink + CollectionSink.clearExpectedResult(); + PENDING_MATERIALIZATION.clear(); + } + + protected StreamGraph buildStreamGraph( + StreamExecutionEnvironment env, ControlledSource controlledSource) { + env.addSource(controlledSource) + .keyBy(element -> element) + .map(new CountMapper()) + .addSink(new CollectionSink()) + .setParallelism(1); + return env.getStreamGraph(); + } + + protected void waitAndAssert(StreamExecutionEnvironment env, StreamGraph streamGraph) + throws Exception { + waitUntilJobFinished(env, streamGraph); + assertEquals(CollectionSink.getExpectedResult(), ControlledSource.getActualResult()); + } + + private void waitUntilJobFinished(StreamExecutionEnvironment env, StreamGraph streamGraph) + throws Exception { + JobExecutionResult jobExecutionResult = env.execute(streamGraph); + JobStatus jobStatus = + cluster.getClusterClient().getJobStatus(jobExecutionResult.getJobID()).get(); + assertSame(jobStatus, JobStatus.FINISHED); + } + + private Configuration configureDSTL() throws IOException { + Configuration configuration = new Configuration(); + FsStateChangelogStorageFactory.configure(configuration, TEMPORARY_FOLDER.newFolder()); + return configuration; + } + + /** A source consumes data which is generated randomly and supports pre-handling record. */ + protected static class ControlledSource extends RichSourceFunction<Integer> + implements CheckpointedFunction, CheckpointListener { + + private static final long serialVersionUID = 1L; + + protected volatile int currentIndex; + + protected final AtomicInteger completedCheckpointNum; + + protected volatile boolean isCanceled; + + private static List<Integer> sourceList; Review comment: Can we avoid at least mutable static fields? Maybe make the list (the reference and contents) immutable and initialize on class loading? ``` private static final List<Integer> SOURCE_LIST = Collections.unmodifiableList(initSourceData(TOTAL_ELEMENTS)); ``` Or pass the elements to the constructor - so there is no static field? -- 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]
