tillrohrmann commented on a change in pull request #14948:
URL: https://github.com/apache/flink/pull/14948#discussion_r597674958
##########
File path:
flink-tests/src/test/java/org/apache/flink/test/scheduling/AdaptiveSchedulerITCase.java
##########
@@ -88,6 +123,189 @@ public void testGlobalFailoverCanRecoverState() throws
Exception {
env.execute();
}
+ private enum StopWithSavepointTestBehavior {
+ NO_FAILURE,
+ FAIL_ON_CHECKPOINT,
+ FAIL_ON_STOP,
+ FAIL_ON_FIRST_CHECKPOINT_ONLY
+ }
+
+ @Test
+ public void testStopWithSavepointNoError() throws Exception {
+ StreamExecutionEnvironment env =
getEnvWithSource(StopWithSavepointTestBehavior.NO_FAILURE);
+
+ DummySource.resetForParallelism(PARALLELISM);
+
+ JobClient client = env.executeAsync();
+
+ DummySource.awaitRunning();
+
+ final File savepointDirectory = tempFolder.newFolder("savepoint");
+ final String savepoint =
+ client.stopWithSavepoint(false,
savepointDirectory.getAbsolutePath()).get();
+ assertThat(savepoint,
containsString(savepointDirectory.getAbsolutePath()));
+ assertThat(client.getJobStatus().get(), is(JobStatus.FINISHED));
+ }
+
+ @Test
+ public void testStopWithSavepointFailOnCheckpoint() throws Exception {
Review comment:
Yes, unit test is good enough.
##########
File path:
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/StopWithSavepointTest.java
##########
@@ -0,0 +1,535 @@
+/*
+ * 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.runtime.scheduler.adaptive;
+
+import org.apache.flink.api.common.JobStatus;
+import org.apache.flink.runtime.checkpoint.CheckpointScheduling;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.scheduler.ExecutionGraphHandler;
+import org.apache.flink.runtime.scheduler.OperatorCoordinatorHandler;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import java.time.Duration;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+import static org.apache.flink.core.testutils.FlinkMatchers.containsCause;
+import static
org.apache.flink.runtime.scheduler.adaptive.ExecutingTest.createFailingStateTransition;
+import static
org.apache.flink.runtime.scheduler.adaptive.WaitingForResourcesTest.assertNonNull;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+
+/** Tests for the {@link StopWithSavepoint} state. */
+public class StopWithSavepointTest extends TestLogger {
+ private static final String SAVEPOINT_PATH = "test://savepoint/path";
+
+ @Test
+ public void testFinishedOnSuccessfulStopWithSavepoint() throws Exception {
+ try (MockStopWithSavepointContext ctx = new
MockStopWithSavepointContext()) {
+ StateTrackingMockExecutionGraph mockExecutionGraph =
+ new StateTrackingMockExecutionGraph();
+ CompletableFuture<String> savepointFuture = new
CompletableFuture<>();
+
+ StopWithSavepoint sws =
+ createStopWithSavepoint(ctx, mockExecutionGraph,
savepointFuture);
+ ctx.setStopWithSavepoint(sws);
+ ctx.setExpectFinished(assertNonNull());
+
+ savepointFuture.complete(SAVEPOINT_PATH);
+ mockExecutionGraph.completeTerminationFuture(JobStatus.FINISHED);
+ ctx.triggerExecutors();
+
+ assertThat(sws.getOperationFuture().get(), is(SAVEPOINT_PATH));
+ }
+ }
Review comment:
I think the execution order is that `StopWithSavepoint` first runs
`onGloballyTerminalState` and then `handleSavepointCompletion`. This is the
similar order as in `testJobFinishedBeforeSavepointFuture`.
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/Executing.java
##########
@@ -158,6 +159,10 @@ public void notifyNewResourcesAvailable() {
getLogger().info("Triggering stop-with-savepoint for job {}.",
executionGraph.getJobID());
+ CheckpointScheduling schedulingProvider = new
CheckpointSchedulingProvider(executionGraph);
Review comment:
Maybe: If we move this into the `AdaptiveScheduler`, then the
`Executing` state might become a bit simpler. Not very important though.
--
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:
[email protected]