pnowojski commented on a change in pull request #15631: URL: https://github.com/apache/flink/pull/15631#discussion_r615540856
########## File path: flink-tests/src/test/java/org/apache/flink/test/checkpointing/UnalignedCheckpointStressITCase.java ########## @@ -0,0 +1,553 @@ +/* + * 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.functions.AbstractRichFunction; +import org.apache.flink.api.common.functions.MapFunction; +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.configuration.Configuration; +import org.apache.flink.configuration.RestOptions; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; +import org.apache.flink.runtime.operators.testutils.ExpectedTestException; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.datastream.DataStreamUtils; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.environment.CheckpointConfig.ExternalizedCheckpointCleanup; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.co.KeyedCoProcessFunction; +import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; +import org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction; +import org.apache.flink.streaming.api.graph.StreamGraph; +import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.util.Collector; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FileUtils; +import org.apache.flink.util.TestLogger; + +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import javax.annotation.Nullable; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Comparator; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.ThreadLocalRandom; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.configuration.CheckpointingOptions.CHECKPOINTS_DIRECTORY; +import static org.apache.flink.configuration.CheckpointingOptions.MAX_RETAINED_CHECKPOINTS; +import static org.apache.flink.shaded.guava18.com.google.common.collect.Iterables.getOnlyElement; +import static org.apache.flink.test.util.TestUtils.submitJobAndWaitForResult; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * A stress test that runs for a pre-defined amount of time, verifying data correctness and every + * couple of checkpoints is triggering fail over to stress test unaligned checkpoints. + */ +@SuppressWarnings("serial") +public class UnalignedCheckpointStressITCase extends TestLogger { + + protected static final int CHECKPOINT_INTERVAL = 20; + protected static final int MINIMUM_COMPLETED_CHECKPOINTS_BETWEEN_FAILURES = 2; + protected static final int MAXIMUM_COMPLETED_CHECKPOINTS_BETWEEN_FAILURES = 10; + protected static final long TEST_DURATION = Time.minutes(1).toMilliseconds(); + protected static final int NUM_TASK_MANAGERS = 3; + protected static final int NUM_TASK_SLOTS = 2; + protected static final int PARALLELISM = NUM_TASK_MANAGERS * NUM_TASK_SLOTS; + protected static final int BUFFER_SIZE = 1024 * 4; + protected static final int BUFFER_TIME = 4; + protected static final int NORMAL_RECORD_SLEEP = 1; + protected static final int SMALL_RECORD_SIZE = + (BUFFER_SIZE / BUFFER_TIME) * NORMAL_RECORD_SLEEP; + + @ClassRule public static TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private static MiniClusterWithClientResource cluster; + + @Before + public void setup() throws Exception { + Configuration configuration = new Configuration(); + File folder = temporaryFolder.getRoot(); + configuration.set(CHECKPOINTS_DIRECTORY, folder.toURI().toString()); + configuration.set(MAX_RETAINED_CHECKPOINTS, 1); + + cluster = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setConfiguration(configuration) + .setNumberTaskManagers(NUM_TASK_MANAGERS) + .setNumberSlotsPerTaskManager(NUM_TASK_SLOTS) + .build()); + cluster.before(); + + FileUtils.cleanDirectory(temporaryFolder.getRoot()); + } + + @After + public void shutDownExistingCluster() { + if (cluster != null) { + cluster.after(); + cluster = null; + } + } + + public static int getSourceId(Record record) { + return record.getSourceId(); + } + + public void testProgram(StreamExecutionEnvironment env) { + int numberOfSources1 = PARALLELISM; + int numberOfSources2 = PARALLELISM / 2; + int numberOfSources3 = PARALLELISM / 3; + int numberOfSources4 = PARALLELISM / 4; + int totalNumberOfSources = + numberOfSources1 + numberOfSources2 + numberOfSources3 + numberOfSources4; + DataStreamSource<Record> source1 = + env.addSource(new LegacySourceFunction(0)).setParallelism(numberOfSources1); + + DataStreamSource<Record> source2 = + env.addSource(new LegacySourceFunction(numberOfSources2)) + .setParallelism(numberOfSources2); + DataStreamSource<Record> source3 = + env.addSource(new LegacySourceFunction(numberOfSources1 + numberOfSources2)) + .setParallelism(numberOfSources3); + DataStreamSource<Record> source4 = + env.addSource( + new LegacySourceFunction( + numberOfSources1 + numberOfSources2 + numberOfSources3)) + .setParallelism(numberOfSources4); + + DataStream<Record> source12 = source1.union(source2); + DataStream<Record> source34 = source3.union(source4); + + SingleOutputStreamOperator<Record> sources = + source12.keyBy(Record::getSourceId) + .connect(source34.keyBy(Record::getSourceId)) + .process( + new KeyedCoProcessFunction<Integer, Record, Record, Record>() { + @Override + public void processElement1( + Record value, Context ctx, Collector<Record> out) { + out.collect(value.validate()); + } + + @Override + public void processElement2( + Record value, Context ctx, Collector<Record> out) { + out.collect(value.validate()); + } + }); + + DataStream<Record> stream = + sources.rebalance() + .map((MapFunction<Record, Record>) value -> value.validate()) + .keyBy(Record::getSourceId) + // add small throttling to prevent WindowOperator from blowing up + .map(new ThrottlingMap(100)); + DataStreamUtils.reinterpretAsKeyedStream(stream, Record::getSourceId) + .window( + TumblingProcessingTimeWindows.of( + Time.milliseconds(NORMAL_RECORD_SLEEP * 5))) + .process(new ReEmitAll()) + // main throttling + .map(new ThrottlingMap(Math.max(1, totalNumberOfSources - 2))) + .setParallelism(1) + .map(new FailingMapper()) + .setParallelism(1); + } + + @Test + public void runStressTest() throws Exception { + long startTs = System.currentTimeMillis(); + Optional<File> externalizedCheckpoint = Optional.empty(); + while (System.currentTimeMillis() < startTs + TEST_DURATION) { + externalizedCheckpoint = + Optional.of(runAndTakeExternalCheckpoint(externalizedCheckpoint)); + cleanDirectoryExcept(externalizedCheckpoint.get()); + } + } + + private void cleanDirectoryExcept(File externalizedCheckpoint) throws IOException { + File directoryToKeep = externalizedCheckpoint.getParentFile(); + for (File directory : temporaryFolder.getRoot().listFiles()) { + if (!directory.equals(directoryToKeep)) { + FileUtils.deleteDirectory(directory); + } + } + } + + private File runAndTakeExternalCheckpoint(Optional<File> startingCheckpoint) throws Exception { + + StreamExecutionEnvironment env = defineEnvironment(); + testProgram(env); + + StreamGraph streamGraph = env.getStreamGraph(); + startingCheckpoint + .map(File::toString) + .map(SavepointRestoreSettings::forPath) + .ifPresent(streamGraph::setSavepointRestoreSettings); + JobGraph jobGraph = streamGraph.getJobGraph(); + + try { + submitJobAndWaitForResult( + cluster.getClusterClient(), jobGraph, getClass().getClassLoader()); + } catch (Exception e) { + if (!ExceptionUtils.findThrowable(e, ExpectedTestException.class).isPresent()) { + throw e; + } + } + + return discoverRetainedCheckpoint(); + } + + private static final Pattern LAST_INT_PATTERN = Pattern.compile("[^0-9]+([0-9]+)$"); + + private static int getCheckpointNumberFromPath(Path checkpointDir) { + Matcher matcher = LAST_INT_PATTERN.matcher(checkpointDir.toString()); + checkState(matcher.find()); + return Integer.parseInt(matcher.group(1)); + } + + private File discoverRetainedCheckpoint() throws Exception { + // structure: root/attempt/checkpoint/_metadata + File attemptDir = temporaryFolder.getRoot(); + Path checkpointDir = null; + + for (int i = 0; i <= 1000 && checkpointDir == null; i++) { Review comment: I'm not sure if @rkhachatryan's version would be working: 1. `Comparator.comparing(Path::toString)` I think doesn't work when comparing checkpoints with different number of digits in the checkpoint number, for example using path comparison `chk-99` > `chk-100` 2. UCRescaleITCase is relaying on cluster shutdown `miniCluster.after();` to make sure the checkpoints are flushed. I'm resubmitting new job for each iteration to the same cluster. I don't know exactly why, but I'm assuming that's why without looping and sleeps my code is failing. (on the other hand, without re-using cluster test would be slower). -- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org