tillrohrmann commented on a change in pull request #14847:
URL: https://github.com/apache/flink/pull/14847#discussion_r580465887



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java
##########
@@ -835,8 +839,15 @@ public void updateAccumulators(final AccumulatorSnapshot 
accumulatorSnapshot) {
                         mainThreadExecutor);
     }
 
-    private void startCheckpointScheduler(final CheckpointCoordinator 
checkpointCoordinator) {
+    @Override
+    public void stopCheckpointScheduler() {
+        getCheckpointCoordinator().stopCheckpointScheduler();

Review comment:
       `getCheckpointCoordinator` can return `null` if checkpointing is not 
enabled. Maybe add a precondition that checkpointing is enabled.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointTerminationHandler.java
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.stopwithsavepoint;
+
+import org.apache.flink.runtime.checkpoint.CompletedCheckpoint;
+import org.apache.flink.runtime.execution.ExecutionState;
+
+import javax.annotation.Nullable;
+
+import java.util.Collection;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * {@code StopWithSavepointTerminationHandler} handles the termination both 
steps necessary for the
+ * stop-with-savepoint operation to finish. It includes:
+ *
+ * <ol>
+ *   <li>Creating a savepoint needs to be completed
+ *   <li>Waiting for the executions of the underlying job to finish
+ * </ol>
+ */
+public interface StopWithSavepointTerminationHandler {
+
+    /**
+     * Returns the a {@code CompletableFuture} referring to the result of the 
stop-with-savepoint

Review comment:
       ```suggestion
        * Returns a {@code CompletableFuture} referring to the result of the 
stop-with-savepoint
   ```

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointTerminationHandler.java
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.stopwithsavepoint;
+
+import org.apache.flink.runtime.checkpoint.CompletedCheckpoint;
+import org.apache.flink.runtime.execution.ExecutionState;
+
+import javax.annotation.Nullable;
+
+import java.util.Collection;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * {@code StopWithSavepointTerminationHandler} handles the termination both 
steps necessary for the

Review comment:
       ```suggestion
    * {@code StopWithSavepointTerminationHandler} handles the termination steps 
necessary for the
   ```

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointTerminationHandlerImpl.java
##########
@@ -0,0 +1,297 @@
+/*
+ * 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.stopwithsavepoint;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.checkpoint.CheckpointScheduling;
+import org.apache.flink.runtime.checkpoint.CompletedCheckpoint;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.scheduler.SchedulerNG;
+import org.apache.flink.util.FlinkException;
+
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+
+import java.util.Collection;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@code StopWithSavepointTerminationHandlerImpl} implements {@link
+ * StopWithSavepointTerminationHandler}.
+ *
+ * <p>The operation only succeeds if both steps, the savepoint creation and 
the successful
+ * termination of the job, succeed. If the former step fails, the operation 
fails exceptionally
+ * without any further actions. If the latter one fails, a global fail-over is 
triggered before
+ * failing the operation.
+ */
+public class StopWithSavepointTerminationHandlerImpl
+        implements StopWithSavepointTerminationHandler {
+
+    private final Logger log;
+
+    private final SchedulerNG scheduler;
+    private final CheckpointScheduling checkpointScheduling;
+    private final JobID jobId;
+    private final Executor ioExecutor;
+
+    private final CompletableFuture<String> result = new CompletableFuture<>();
+
+    private State state = new WaitingForSavepoint();
+
+    public <S extends SchedulerNG & CheckpointScheduling> 
StopWithSavepointTerminationHandlerImpl(
+            JobID jobId, S schedulerWithCheckpointing, Executor ioExecutor, 
Logger log) {
+        this(jobId, schedulerWithCheckpointing, schedulerWithCheckpointing, 
ioExecutor, log);
+    }
+
+    @VisibleForTesting
+    StopWithSavepointTerminationHandlerImpl(
+            JobID jobId,
+            SchedulerNG scheduler,
+            CheckpointScheduling checkpointScheduling,
+            Executor ioExecutor,
+            Logger log) {
+        this.jobId = checkNotNull(jobId);
+        this.scheduler = checkNotNull(scheduler);
+        this.checkpointScheduling = checkNotNull(checkpointScheduling);
+        this.ioExecutor = checkNotNull(ioExecutor);
+        this.log = checkNotNull(log);
+    }
+
+    @Override
+    public CompletableFuture<String> getSavepointPath() {
+        return result;
+    }
+
+    @Override
+    public void handleSavepointCreation(

Review comment:
       Just out of curiosity why did you choose this API call as the entry into 
the `StopWithSavepointTerminationHandler` compared to exposing 
`handleSavepointCreationFailure` and `handleSavepointCreationSuccess`?

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointTerminationHandlerImpl.java
##########
@@ -0,0 +1,297 @@
+/*
+ * 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.stopwithsavepoint;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.checkpoint.CheckpointScheduling;
+import org.apache.flink.runtime.checkpoint.CompletedCheckpoint;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.scheduler.SchedulerNG;
+import org.apache.flink.util.FlinkException;
+
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+
+import java.util.Collection;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@code StopWithSavepointTerminationHandlerImpl} implements {@link
+ * StopWithSavepointTerminationHandler}.
+ *
+ * <p>The operation only succeeds if both steps, the savepoint creation and 
the successful
+ * termination of the job, succeed. If the former step fails, the operation 
fails exceptionally
+ * without any further actions. If the latter one fails, a global fail-over is 
triggered before
+ * failing the operation.
+ */
+public class StopWithSavepointTerminationHandlerImpl
+        implements StopWithSavepointTerminationHandler {
+
+    private final Logger log;
+
+    private final SchedulerNG scheduler;
+    private final CheckpointScheduling checkpointScheduling;
+    private final JobID jobId;
+    private final Executor ioExecutor;
+
+    private final CompletableFuture<String> result = new CompletableFuture<>();
+
+    private State state = new WaitingForSavepoint();
+
+    public <S extends SchedulerNG & CheckpointScheduling> 
StopWithSavepointTerminationHandlerImpl(
+            JobID jobId, S schedulerWithCheckpointing, Executor ioExecutor, 
Logger log) {
+        this(jobId, schedulerWithCheckpointing, schedulerWithCheckpointing, 
ioExecutor, log);
+    }
+
+    @VisibleForTesting
+    StopWithSavepointTerminationHandlerImpl(
+            JobID jobId,
+            SchedulerNG scheduler,
+            CheckpointScheduling checkpointScheduling,
+            Executor ioExecutor,
+            Logger log) {
+        this.jobId = checkNotNull(jobId);
+        this.scheduler = checkNotNull(scheduler);
+        this.checkpointScheduling = checkNotNull(checkpointScheduling);
+        this.ioExecutor = checkNotNull(ioExecutor);
+        this.log = checkNotNull(log);
+    }
+
+    @Override
+    public CompletableFuture<String> getSavepointPath() {
+        return result;
+    }
+
+    @Override
+    public void handleSavepointCreation(
+            CompletedCheckpoint completedSavepoint, Throwable throwable) {
+        if (throwable != null) {
+            checkArgument(
+                    completedSavepoint == null,
+                    "No savepoint should be provided if a throwable is 
passed.");
+            handleSavepointCreationFailure(throwable);
+        } else {
+            handleSavepointCreationSuccess(checkNotNull(completedSavepoint));
+        }
+    }
+
+    @Override
+    public void handleExecutionsTermination(Collection<ExecutionState> 
terminatedExecutionStates) {
+        final Set<ExecutionState> notFinishedExecutionStates =
+                checkNotNull(terminatedExecutionStates).stream()
+                        .filter(state -> state != ExecutionState.FINISHED)
+                        .collect(Collectors.toSet());
+
+        if (notFinishedExecutionStates.isEmpty()) {
+            handleExecutionsFinished();
+        } else {
+            handleAnyExecutionNotFinished(notFinishedExecutionStates);
+        }
+    }
+
+    private void handleSavepointCreationSuccess(CompletedCheckpoint 
completedCheckpoint) {
+        final State oldState = state;
+        state = state.onSavepointCreation(completedCheckpoint);
+
+        log.debug(
+                "Stop-with-savepoint transitioned from {} to {} on savepoint 
creation handling for job {}.",
+                oldState,
+                state,
+                jobId);
+    }
+
+    private void handleSavepointCreationFailure(Throwable throwable) {
+        final State oldState = state;
+        state = state.onSavepointCreationFailure(throwable);
+
+        log.debug(
+                "Stop-with-savepoint transitioned from {} to {} on savepoint 
creation failure handling for job {}.",
+                oldState,
+                state,
+                jobId);
+    }
+
+    private void handleExecutionsFinished() {
+        final State oldState = state;
+        state = state.onExecutionsFinished();
+
+        log.debug(
+                "Stop-with-savepoint transitioned from {} to {} on execution 
termination handling with all executions being finished for job {}.",
+                oldState,
+                state,
+                jobId);
+    }
+
+    private void handleAnyExecutionNotFinished(Set<ExecutionState> 
notFinishedExecutionStates) {
+        final State oldState = state;
+        state = state.onAnyExecutionNotFinished(notFinishedExecutionStates);
+
+        log.warn(
+                "Stop-with-savepoint transitioned from {} to {} on execution 
termination handling for job {} with some executions being in an not-finished 
state: {}",
+                oldState,
+                state,
+                jobId,
+                notFinishedExecutionStates);
+    }
+
+    /**
+     * Handles the termination of the {@code 
StopWithSavepointTerminationHandler} exceptionally
+     * after triggering a global job fail-over.
+     *
+     * @param completedSavepoint the completed savepoint that needs to be 
discarded.
+     * @param unfinishedExecutionStates the unfinished states that caused the 
failure.
+     */
+    private void terminateExceptionallyWithGlobalFailover(
+            CompletedCheckpoint completedSavepoint,
+            Iterable<ExecutionState> unfinishedExecutionStates) {
+        String errorMessage =
+                String.format(
+                        "Inconsistent execution state after stopping with 
savepoint. At least one execution is still in one of the following states: %s. 
A global fail-over is triggered to recover the job %s.",
+                        StringUtils.join(unfinishedExecutionStates, ", "), 
jobId);
+        FlinkException inconsistentFinalStateException = new 
FlinkException(errorMessage);
+
+        scheduler.handleGlobalFailure(inconsistentFinalStateException);
+        discardSavepoint(completedSavepoint, inconsistentFinalStateException);
+
+        terminateExceptionally(inconsistentFinalStateException);
+    }
+
+    /**
+     * Handles the termination of the {@code 
StopWithSavepointTerminationHandler} exceptionally
+     * without triggering a global job fail-over. It does restart the 
checkpoint scheduling.
+     *
+     * @param throwable the error that caused the exceptional termination.
+     */
+    private void terminateExceptionally(Throwable throwable) {
+        checkpointScheduling.startCheckpointScheduler();
+        result.completeExceptionally(throwable);
+    }
+
+    /**
+     * Handles the successful termination of the {@code 
StopWithSavepointTerminationHandler}.
+     *
+     * @param completedSavepoint the completed savepoint
+     */
+    private void terminateSuccessfully(CompletedCheckpoint completedSavepoint) 
{
+        result.complete(completedSavepoint.getExternalPointer());
+    }
+
+    private void discardSavepoint(
+            CompletedCheckpoint completedSavepoint,
+            FlinkException inconsistentFinalStateException) {
+        CompletableFuture.runAsync(
+                () -> {
+                    try {
+                        completedSavepoint.discard();
+                    } catch (Exception e) {
+                        log.warn(
+                                "Error occurred while cleaning up completed 
savepoint due to stop-with-savepoint failure.",
+                                e);
+                        inconsistentFinalStateException.addSuppressed(e);

Review comment:
       I understand your motives here. I am just wondering whether the 
non-determinism due to the concurrency is a problem.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointTerminationManager.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.stopwithsavepoint;
+
+import org.apache.flink.runtime.checkpoint.CompletedCheckpoint;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Collection;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * {@code StopWithSavepointTerminationManager} fulfills the contract given by 
{@link
+ * StopWithSavepointTerminationHandler} to run the stop-with-savepoint steps 
in a specific order.
+ */
+public class StopWithSavepointTerminationManager {
+
+    private final StopWithSavepointTerminationHandler 
stopWithSavepointTerminationHandler;
+
+    public StopWithSavepointTerminationManager(
+            StopWithSavepointTerminationHandler 
stopWithSavepointTerminationHandler) {
+        this.stopWithSavepointTerminationHandler =
+                
Preconditions.checkNotNull(stopWithSavepointTerminationHandler);
+    }
+
+    /**
+     * Enforces the correct completion order of the passed {@code 
CompletableFuture} instances in
+     * accordance to the contract of {@link 
StopWithSavepointTerminationHandler}.
+     *
+     * @param completedSavepointFuture The {@code CompletableFuture} of the 
savepoint creation step.
+     * @param terminatedExecutionStatesFuture The {@code CompletableFuture} of 
the termination step.
+     * @param mainThreadExecutor The executor the {@code 
StopWithSavepointTerminationHandler}
+     *     operations run on.
+     * @return A {@code CompletableFuture} containing the path to the created 
savepoint.
+     */
+    public CompletableFuture<String> stopWithSavepoint(
+            CompletableFuture<CompletedCheckpoint> completedSavepointFuture,
+            CompletableFuture<Collection<ExecutionState>> 
terminatedExecutionStatesFuture,
+            ComponentMainThreadExecutor mainThreadExecutor) {
+        FutureUtils.assertNoException(
+                completedSavepointFuture
+                        // the completedSavepointFuture could also be 
completed by
+                        // CheckpointCanceller which doesn't run in the 
mainThreadExecutor
+                        .handleAsync(
+                                (completedSavepoint, throwable) -> {
+                                    
stopWithSavepointTerminationHandler.handleSavepointCreation(
+                                            completedSavepoint, throwable);
+                                    return null;
+                                },
+                                mainThreadExecutor)
+                        .thenRun(
+                                () ->
+                                        FutureUtils.assertNoException(
+                                                
terminatedExecutionStatesFuture.thenAccept(
+                                                        
stopWithSavepointTerminationHandler
+                                                                
::handleExecutionsTermination))));
+
+        return stopWithSavepointTerminationHandler.getSavepointPath();
+    }

Review comment:
       Nice :-)

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointTerminationHandlerImpl.java
##########
@@ -0,0 +1,297 @@
+/*
+ * 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.stopwithsavepoint;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.checkpoint.CheckpointScheduling;
+import org.apache.flink.runtime.checkpoint.CompletedCheckpoint;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.scheduler.SchedulerNG;
+import org.apache.flink.util.FlinkException;
+
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+
+import java.util.Collection;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@code StopWithSavepointTerminationHandlerImpl} implements {@link
+ * StopWithSavepointTerminationHandler}.
+ *
+ * <p>The operation only succeeds if both steps, the savepoint creation and 
the successful
+ * termination of the job, succeed. If the former step fails, the operation 
fails exceptionally
+ * without any further actions. If the latter one fails, a global fail-over is 
triggered before
+ * failing the operation.
+ */
+public class StopWithSavepointTerminationHandlerImpl
+        implements StopWithSavepointTerminationHandler {
+
+    private final Logger log;
+
+    private final SchedulerNG scheduler;
+    private final CheckpointScheduling checkpointScheduling;
+    private final JobID jobId;
+    private final Executor ioExecutor;
+
+    private final CompletableFuture<String> result = new CompletableFuture<>();
+
+    private State state = new WaitingForSavepoint();
+
+    public <S extends SchedulerNG & CheckpointScheduling> 
StopWithSavepointTerminationHandlerImpl(
+            JobID jobId, S schedulerWithCheckpointing, Executor ioExecutor, 
Logger log) {
+        this(jobId, schedulerWithCheckpointing, schedulerWithCheckpointing, 
ioExecutor, log);
+    }
+
+    @VisibleForTesting
+    StopWithSavepointTerminationHandlerImpl(
+            JobID jobId,
+            SchedulerNG scheduler,
+            CheckpointScheduling checkpointScheduling,
+            Executor ioExecutor,
+            Logger log) {
+        this.jobId = checkNotNull(jobId);
+        this.scheduler = checkNotNull(scheduler);
+        this.checkpointScheduling = checkNotNull(checkpointScheduling);
+        this.ioExecutor = checkNotNull(ioExecutor);
+        this.log = checkNotNull(log);
+    }
+
+    @Override
+    public CompletableFuture<String> getSavepointPath() {
+        return result;
+    }
+
+    @Override
+    public void handleSavepointCreation(
+            CompletedCheckpoint completedSavepoint, Throwable throwable) {
+        if (throwable != null) {
+            checkArgument(
+                    completedSavepoint == null,
+                    "No savepoint should be provided if a throwable is 
passed.");
+            handleSavepointCreationFailure(throwable);
+        } else {
+            handleSavepointCreationSuccess(checkNotNull(completedSavepoint));
+        }
+    }
+
+    @Override
+    public void handleExecutionsTermination(Collection<ExecutionState> 
terminatedExecutionStates) {
+        final Set<ExecutionState> notFinishedExecutionStates =
+                checkNotNull(terminatedExecutionStates).stream()
+                        .filter(state -> state != ExecutionState.FINISHED)
+                        .collect(Collectors.toSet());
+
+        if (notFinishedExecutionStates.isEmpty()) {
+            handleExecutionsFinished();
+        } else {
+            handleAnyExecutionNotFinished(notFinishedExecutionStates);
+        }
+    }
+
+    private void handleSavepointCreationSuccess(CompletedCheckpoint 
completedCheckpoint) {
+        final State oldState = state;
+        state = state.onSavepointCreation(completedCheckpoint);
+
+        log.debug(
+                "Stop-with-savepoint transitioned from {} to {} on savepoint 
creation handling for job {}.",
+                oldState,
+                state,
+                jobId);
+    }
+
+    private void handleSavepointCreationFailure(Throwable throwable) {
+        final State oldState = state;
+        state = state.onSavepointCreationFailure(throwable);
+
+        log.debug(
+                "Stop-with-savepoint transitioned from {} to {} on savepoint 
creation failure handling for job {}.",
+                oldState,
+                state,
+                jobId);
+    }
+
+    private void handleExecutionsFinished() {
+        final State oldState = state;
+        state = state.onExecutionsFinished();
+
+        log.debug(
+                "Stop-with-savepoint transitioned from {} to {} on execution 
termination handling with all executions being finished for job {}.",
+                oldState,
+                state,
+                jobId);
+    }
+
+    private void handleAnyExecutionNotFinished(Set<ExecutionState> 
notFinishedExecutionStates) {
+        final State oldState = state;
+        state = state.onAnyExecutionNotFinished(notFinishedExecutionStates);
+
+        log.warn(
+                "Stop-with-savepoint transitioned from {} to {} on execution 
termination handling for job {} with some executions being in an not-finished 
state: {}",
+                oldState,
+                state,
+                jobId,
+                notFinishedExecutionStates);
+    }
+
+    /**
+     * Handles the termination of the {@code 
StopWithSavepointTerminationHandler} exceptionally
+     * after triggering a global job fail-over.
+     *
+     * @param completedSavepoint the completed savepoint that needs to be 
discarded.
+     * @param unfinishedExecutionStates the unfinished states that caused the 
failure.
+     */
+    private void terminateExceptionallyWithGlobalFailover(
+            CompletedCheckpoint completedSavepoint,
+            Iterable<ExecutionState> unfinishedExecutionStates) {
+        String errorMessage =
+                String.format(
+                        "Inconsistent execution state after stopping with 
savepoint. At least one execution is still in one of the following states: %s. 
A global fail-over is triggered to recover the job %s.",
+                        StringUtils.join(unfinishedExecutionStates, ", "), 
jobId);
+        FlinkException inconsistentFinalStateException = new 
FlinkException(errorMessage);
+
+        scheduler.handleGlobalFailure(inconsistentFinalStateException);
+        discardSavepoint(completedSavepoint, inconsistentFinalStateException);
+
+        terminateExceptionally(inconsistentFinalStateException);
+    }
+
+    /**
+     * Handles the termination of the {@code 
StopWithSavepointTerminationHandler} exceptionally
+     * without triggering a global job fail-over. It does restart the 
checkpoint scheduling.
+     *
+     * @param throwable the error that caused the exceptional termination.
+     */
+    private void terminateExceptionally(Throwable throwable) {
+        checkpointScheduling.startCheckpointScheduler();
+        result.completeExceptionally(throwable);
+    }
+
+    /**
+     * Handles the successful termination of the {@code 
StopWithSavepointTerminationHandler}.
+     *
+     * @param completedSavepoint the completed savepoint
+     */
+    private void terminateSuccessfully(CompletedCheckpoint completedSavepoint) 
{
+        result.complete(completedSavepoint.getExternalPointer());
+    }
+
+    private void discardSavepoint(
+            CompletedCheckpoint completedSavepoint,
+            FlinkException inconsistentFinalStateException) {
+        CompletableFuture.runAsync(
+                () -> {
+                    try {
+                        completedSavepoint.discard();
+                    } catch (Exception e) {
+                        log.warn(
+                                "Error occurred while cleaning up completed 
savepoint due to stop-with-savepoint failure.",
+                                e);
+                        inconsistentFinalStateException.addSuppressed(e);
+                    }
+                },
+                ioExecutor);

Review comment:
       How are we ensuring that this action is executed before we shut down the 
scheduler?

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointTerminationHandlerImplTest.java
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.stopwithsavepoint;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.core.testutils.FlinkMatchers;
+import org.apache.flink.runtime.checkpoint.CheckpointProperties;
+import org.apache.flink.runtime.checkpoint.CompletedCheckpoint;
+import org.apache.flink.runtime.checkpoint.TestingCheckpointScheduling;
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.scheduler.SchedulerBase;
+import org.apache.flink.runtime.scheduler.SchedulerNG;
+import org.apache.flink.runtime.scheduler.TestingSchedulerNG;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.testutils.EmptyStreamStateHandle;
+import 
org.apache.flink.runtime.state.testutils.TestCompletedCheckpointStorageLocation;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.function.Consumer;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * {@code StopWithSavepointTerminationHandlerImplTest} tests the 
stop-with-savepoint functionality
+ * of {@link SchedulerBase#stopWithSavepoint(String, boolean)}.
+ */
+public class StopWithSavepointTerminationHandlerImplTest extends TestLogger {
+
+    private static final JobID JOB_ID = new JobID();
+
+    private final TestingCheckpointScheduling checkpointScheduling =
+            new TestingCheckpointScheduling(false);
+
+    private StopWithSavepointTerminationHandlerImpl 
createTestInstanceFailingOnGlobalFailOver() {
+        return createTestInstance(
+                throwableCausingGlobalFailOver -> fail("No global failover 
should be triggered."));
+    }
+
+    private StopWithSavepointTerminationHandlerImpl createTestInstance(
+            Consumer<Throwable> handleGlobalFailureConsumer) {
+        // checkpointing should be always stopped before initiating 
stop-with-savepoint
+        checkpointScheduling.stopCheckpointScheduler();
+
+        final SchedulerNG scheduler =
+                TestingSchedulerNG.newBuilder()
+                        
.setHandleGlobalFailureConsumer(handleGlobalFailureConsumer)
+                        .build();
+        return new StopWithSavepointTerminationHandlerImpl(
+                JOB_ID, scheduler, checkpointScheduling, 
Executors.directExecutor(), log);
+    }
+
+    @Test
+    public void testHappyPath() throws ExecutionException, 
InterruptedException {
+        final StopWithSavepointTerminationHandlerImpl testInstance =
+                createTestInstanceFailingOnGlobalFailOver();
+
+        final EmptyStreamStateHandle streamStateHandle = new 
EmptyStreamStateHandle();
+        final CompletedCheckpoint completedSavepoint = 
createCompletedSavepoint(streamStateHandle);
+        testInstance.handleSavepointCreation(completedSavepoint, null);
+        
testInstance.handleExecutionsTermination(Collections.singleton(ExecutionState.FINISHED));
+
+        assertThat(
+                testInstance.getSavepointPath().get(), 
is(completedSavepoint.getExternalPointer()));
+
+        assertFalse(
+                "The savepoint should not have been discarded.", 
streamStateHandle.isDisposed());
+        assertFalse("Checkpoint scheduling should be disabled.", 
checkpointScheduling.isEnabled());
+    }
+
+    @Test
+    public void testSavepointCreationFailure() {
+        final StopWithSavepointTerminationHandlerImpl testInstance =
+                createTestInstanceFailingOnGlobalFailOver();
+
+        final String expectedErrorMessage = "Expected exception during 
savepoint creation.";
+        testInstance.handleSavepointCreation(null, new 
Exception(expectedErrorMessage));
+
+        try {
+            testInstance.getSavepointPath().get();
+            fail("An ExecutionException is expected.");
+        } catch (Throwable e) {
+            final Optional<Throwable> actualException =
+                    ExceptionUtils.findThrowableWithMessage(e, 
expectedErrorMessage);
+            assertTrue(
+                    "An exception with the expected error message should have 
been thrown.",
+                    actualException.isPresent());
+        }
+
+        // the checkpoint scheduling should be enabled in case of failure
+        assertTrue("Checkpoint scheduling should be enabled.", 
checkpointScheduling.isEnabled());
+    }

Review comment:
       Maybe add one more tests which ensures that we can call 
`handleExecutionsTermination` after a failed savepoint and that this is ignored.

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointTerminationManagerTest.java
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.stopwithsavepoint;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.checkpoint.CheckpointProperties;
+import org.apache.flink.runtime.checkpoint.CompletedCheckpoint;
+import 
org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.state.testutils.EmptyStreamStateHandle;
+import 
org.apache.flink.runtime.state.testutils.TestCompletedCheckpointStorageLocation;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.BiConsumer;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * {@code StopWithSavepointTerminationManagerTest} tests that {@link
+ * StopWithSavepointTerminationManager} applies the correct order expected by 
{@link
+ * StopWithSavepointTerminationHandler} regardless of the completion of the 
provided {@code
+ * CompletableFutures}.
+ */
+public class StopWithSavepointTerminationManagerTest extends TestLogger {

Review comment:
       Nice, I like the tests. They are very clean now :-)

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointTerminationHandlerImpl.java
##########
@@ -0,0 +1,297 @@
+/*
+ * 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.stopwithsavepoint;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.checkpoint.CheckpointScheduling;
+import org.apache.flink.runtime.checkpoint.CompletedCheckpoint;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.scheduler.SchedulerNG;
+import org.apache.flink.util.FlinkException;
+
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+
+import java.util.Collection;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@code StopWithSavepointTerminationHandlerImpl} implements {@link
+ * StopWithSavepointTerminationHandler}.
+ *
+ * <p>The operation only succeeds if both steps, the savepoint creation and 
the successful
+ * termination of the job, succeed. If the former step fails, the operation 
fails exceptionally
+ * without any further actions. If the latter one fails, a global fail-over is 
triggered before
+ * failing the operation.
+ */
+public class StopWithSavepointTerminationHandlerImpl
+        implements StopWithSavepointTerminationHandler {
+
+    private final Logger log;
+
+    private final SchedulerNG scheduler;
+    private final CheckpointScheduling checkpointScheduling;
+    private final JobID jobId;
+    private final Executor ioExecutor;
+
+    private final CompletableFuture<String> result = new CompletableFuture<>();
+
+    private State state = new WaitingForSavepoint();
+
+    public <S extends SchedulerNG & CheckpointScheduling> 
StopWithSavepointTerminationHandlerImpl(
+            JobID jobId, S schedulerWithCheckpointing, Executor ioExecutor, 
Logger log) {
+        this(jobId, schedulerWithCheckpointing, schedulerWithCheckpointing, 
ioExecutor, log);
+    }
+
+    @VisibleForTesting
+    StopWithSavepointTerminationHandlerImpl(
+            JobID jobId,
+            SchedulerNG scheduler,
+            CheckpointScheduling checkpointScheduling,
+            Executor ioExecutor,
+            Logger log) {
+        this.jobId = checkNotNull(jobId);
+        this.scheduler = checkNotNull(scheduler);
+        this.checkpointScheduling = checkNotNull(checkpointScheduling);
+        this.ioExecutor = checkNotNull(ioExecutor);
+        this.log = checkNotNull(log);
+    }
+
+    @Override
+    public CompletableFuture<String> getSavepointPath() {
+        return result;
+    }
+
+    @Override
+    public void handleSavepointCreation(
+            CompletedCheckpoint completedSavepoint, Throwable throwable) {
+        if (throwable != null) {
+            checkArgument(
+                    completedSavepoint == null,
+                    "No savepoint should be provided if a throwable is 
passed.");
+            handleSavepointCreationFailure(throwable);
+        } else {
+            handleSavepointCreationSuccess(checkNotNull(completedSavepoint));
+        }
+    }
+
+    @Override
+    public void handleExecutionsTermination(Collection<ExecutionState> 
terminatedExecutionStates) {
+        final Set<ExecutionState> notFinishedExecutionStates =
+                checkNotNull(terminatedExecutionStates).stream()
+                        .filter(state -> state != ExecutionState.FINISHED)
+                        .collect(Collectors.toSet());
+
+        if (notFinishedExecutionStates.isEmpty()) {
+            handleExecutionsFinished();
+        } else {
+            handleAnyExecutionNotFinished(notFinishedExecutionStates);
+        }
+    }
+
+    private void handleSavepointCreationSuccess(CompletedCheckpoint 
completedCheckpoint) {
+        final State oldState = state;
+        state = state.onSavepointCreation(completedCheckpoint);
+
+        log.debug(
+                "Stop-with-savepoint transitioned from {} to {} on savepoint 
creation handling for job {}.",
+                oldState,
+                state,
+                jobId);
+    }
+
+    private void handleSavepointCreationFailure(Throwable throwable) {
+        final State oldState = state;
+        state = state.onSavepointCreationFailure(throwable);
+
+        log.debug(
+                "Stop-with-savepoint transitioned from {} to {} on savepoint 
creation failure handling for job {}.",
+                oldState,
+                state,
+                jobId);
+    }
+
+    private void handleExecutionsFinished() {
+        final State oldState = state;
+        state = state.onExecutionsFinished();
+
+        log.debug(
+                "Stop-with-savepoint transitioned from {} to {} on execution 
termination handling with all executions being finished for job {}.",
+                oldState,
+                state,
+                jobId);
+    }
+
+    private void handleAnyExecutionNotFinished(Set<ExecutionState> 
notFinishedExecutionStates) {
+        final State oldState = state;
+        state = state.onAnyExecutionNotFinished(notFinishedExecutionStates);
+
+        log.warn(
+                "Stop-with-savepoint transitioned from {} to {} on execution 
termination handling for job {} with some executions being in an not-finished 
state: {}",
+                oldState,
+                state,
+                jobId,
+                notFinishedExecutionStates);
+    }
+
+    /**
+     * Handles the termination of the {@code 
StopWithSavepointTerminationHandler} exceptionally
+     * after triggering a global job fail-over.
+     *
+     * @param completedSavepoint the completed savepoint that needs to be 
discarded.
+     * @param unfinishedExecutionStates the unfinished states that caused the 
failure.
+     */
+    private void terminateExceptionallyWithGlobalFailover(
+            CompletedCheckpoint completedSavepoint,
+            Iterable<ExecutionState> unfinishedExecutionStates) {
+        String errorMessage =
+                String.format(
+                        "Inconsistent execution state after stopping with 
savepoint. At least one execution is still in one of the following states: %s. 
A global fail-over is triggered to recover the job %s.",
+                        StringUtils.join(unfinishedExecutionStates, ", "), 
jobId);
+        FlinkException inconsistentFinalStateException = new 
FlinkException(errorMessage);
+
+        scheduler.handleGlobalFailure(inconsistentFinalStateException);
+        discardSavepoint(completedSavepoint, inconsistentFinalStateException);
+
+        terminateExceptionally(inconsistentFinalStateException);
+    }
+
+    /**
+     * Handles the termination of the {@code 
StopWithSavepointTerminationHandler} exceptionally
+     * without triggering a global job fail-over. It does restart the 
checkpoint scheduling.
+     *
+     * @param throwable the error that caused the exceptional termination.
+     */
+    private void terminateExceptionally(Throwable throwable) {
+        checkpointScheduling.startCheckpointScheduler();
+        result.completeExceptionally(throwable);
+    }
+
+    /**
+     * Handles the successful termination of the {@code 
StopWithSavepointTerminationHandler}.
+     *
+     * @param completedSavepoint the completed savepoint
+     */
+    private void terminateSuccessfully(CompletedCheckpoint completedSavepoint) 
{
+        result.complete(completedSavepoint.getExternalPointer());
+    }
+
+    private void discardSavepoint(
+            CompletedCheckpoint completedSavepoint,
+            FlinkException inconsistentFinalStateException) {
+        CompletableFuture.runAsync(

Review comment:
       Why did you choose to create a `CompletableFuture` here?




----------------------------------------------------------------
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]


Reply via email to