rkhachatryan commented on a change in pull request #16773: URL: https://github.com/apache/flink/pull/16773#discussion_r688413795
########## File path: flink-tests/src/test/java/org/apache/flink/test/savepoint/StopWithSavepointITCase.java ########## @@ -0,0 +1,448 @@ +/* + * 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.savepoint; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.state.CheckpointListener; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.runtime.OperatorIDPair; +import org.apache.flink.runtime.jobgraph.IntermediateDataSet; +import org.apache.flink.runtime.jobgraph.JobEdge; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.runtime.testutils.CommonTestUtils; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.functions.sink.DiscardingSink; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.BoundedMultiInput; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.test.util.AbstractTestBase; +import org.apache.flink.testutils.junit.SharedObjects; +import org.apache.flink.testutils.junit.SharedReference; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.ListIterator; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.stream.Collectors; + +import static java.lang.String.format; +import static java.util.Arrays.asList; +import static java.util.Collections.emptyMap; +import static org.apache.flink.api.common.restartstrategy.RestartStrategies.noRestart; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +/** + * A test suite to check that the operator methods are called according to contract when the job is + * stopped with savepoint. The contract was refined in FLIP-147. + * + * <p>Checked assumptions: + * + * <ol> + * <li>Downstream should only be "finished" after all of its the upstreams are + * <li>Order of events when finishing an operator: + * <ol> + * <li>(last data element) + * <li>{@link Watermark#MAX_WATERMARK MAX_WATERMARK} (if with drain) + * <li>{@link BoundedMultiInput#endInput endInput} (if with drain) + * <li>timer service quiesced + * <li>{@link StreamOperator#finish() finish} (if with drain; support is planned for + * no-drain) + * <li>{@link AbstractStreamOperator#snapshotState(StateSnapshotContext) snapshotState} (for + * the respective checkpoint) + * <li>{@link CheckpointListener#notifyCheckpointComplete notifyCheckpointComplete} (for the + * respective checkpoint) + * <li>(task termination) + * </ol> + * <li>Timers can be registered until the operator is finished (though may not fire) (simply + * register every 1ms and don't expect any exception) + * <li>The same watermark is received + * </ol> + * + * <p>Variants: + * + * <ul> + * <li>command - with or without drain (MAX_WATERMARK and endInput should be iff drain) + * <li>graph - different exchanges (keyBy, forward) + * <li>graph - multi-inputs (NOT IMPLEMENTED), unions + * <li>graph - FLIP-27 and regular sources (should work for both) - NOT IMPLEMENTED + * </ul> + * + * <p>Not checked: + * + * <ul> + * <li>state distribution on recovery (when a new job started from the taken savepoint) (a + * separate IT case for partial finishing and state distribution) + * <li>re-taking a savepoint after one fails (and job fails over) (as it should not affect + * savepoints) + * <li>taking a savepoint after recovery (as it should not affect savepoints) + * <li>taking a savepoint on a partially completed graph (a separate IT case) + * </ul> + */ +@RunWith(Parameterized.class) +public class StopWithSavepointITCase extends AbstractTestBase { + private static final Logger LOG = LoggerFactory.getLogger(StopWithSavepointITCase.class); + + @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); + @Rule public final SharedObjects sharedObjects = SharedObjects.create(); + + @Parameter public boolean withDrain; + + @Test + public void test() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(4); + env.setRestartStrategy(noRestart()); + env.enableCheckpointing(200); // shouldn't matter + env.getConfig().setAutoWatermarkInterval(50); + + SharedReference<List<TestEvent>> eventsRef = + sharedObjects.add(new CopyOnWriteArrayList<>()); + TestSetup testSetup = buildGraph(env, eventsRef); + submitAndStopWithSavepoint(testSetup.jobGraph, withDrain, eventsRef); + + List<TestEvent> events = eventsRef.get(); + + checkOperatorsLifecycle(events, testSetup); + if (withDrain) { + // currently (1.14), sources do not stop before taking a savepoint and continue emission + // todo: enable after updating production code + checkDataFlow(events, testSetup); + } + } + + private static class TestSetup { + private final JobGraph jobGraph; + private final Set<String> operatorsWithLifecycleTracking; + private final Set<String> operatorsWithDataFlowTracking; + + private TestSetup( + JobGraph jobGraph, + Set<String> operatorsWithLifecycleTracking, + Set<String> operatorsWithDataFlowTracking) { + this.jobGraph = jobGraph; + this.operatorsWithLifecycleTracking = operatorsWithLifecycleTracking; + this.operatorsWithDataFlowTracking = operatorsWithDataFlowTracking; + } + } + + private TestSetup buildGraph( + StreamExecutionEnvironment env, SharedReference<List<TestEvent>> eventsRef) { + + // using hashes so that operators emit identifiable events + String srcLeft = OP_ID_HASH_PREFIX + "1"; + String srcRight = OP_ID_HASH_PREFIX + "2"; + String mapForward = OP_ID_HASH_PREFIX + "3"; + String mapKeyed = OP_ID_HASH_PREFIX + "4"; + + // todo: add multi-inputs and FLIP-27 sources + DataStream<TestEvent> unitedSources = + env.addSource(new TestEventSource(srcLeft, eventsRef)) + .setUidHash(srcLeft) + .assignTimestampsAndWatermarks(createWmAssigner()) + .union( + env.addSource(new TestEventSource(srcRight, eventsRef)) + .setUidHash(srcRight) + .assignTimestampsAndWatermarks(createWmAssigner())); Review comment: As discussed offline, I'll address it with a lower priority (and leave a `todo` for now); I've also added `MultipleInputTransformation` which is similar (though not completely the same). -- 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: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org