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



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/Executing.java
##########
@@ -143,6 +145,26 @@ public void notifyNewResourcesAvailable() {
         }
     }
 
+    CompletableFuture<String> stopWithSavepoint(
+            @Nullable final String targetDirectory, boolean terminate) {
+        final ExecutionGraph executionGraph = getExecutionGraph();
+
+        StopWithSavepointOperationManager.checkStopWithSavepointPreconditions(
+                executionGraph.getCheckpointCoordinator(),
+                targetDirectory,
+                executionGraph.getJobID(),
+                getLogger());
+
+        getLogger().info("Triggering stop-with-savepoint for job {}.", 
executionGraph.getJobID());

Review comment:
       nit: This sanity check could also go into the 
`AdaptiveScheduler.goToStopWithSavepoint` implementation if one argues that it 
is not the responsibility of the `Executing` state to decide whether this 
operation is possible or not.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/StopWithSavepointOperationManagerForAdaptiveScheduler.java
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.runtime.checkpoint.CompletedCheckpoint;
+import org.apache.flink.runtime.checkpoint.StopWithSavepointOperations;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import 
org.apache.flink.runtime.scheduler.stopwithsavepoint.StopWithSavepointOperationHandler;
+import 
org.apache.flink.runtime.scheduler.stopwithsavepoint.StopWithSavepointOperationHandlerImpl;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+
+import javax.annotation.Nullable;
+
+import java.util.Collection;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.stream.Collectors;
+
+/**
+ * A thin wrapper around the StopWithSavepointOperationHandler to handle 
global failures according
+ * to the needs of AdaptiveScheduler. AdaptiveScheduler currently doesn't 
support local failover,
+ * hence, any reported failure will lead to a transition to Restarting or 
Failing state.
+ */
+class StopWithSavepointOperationManagerForAdaptiveScheduler implements 
GlobalFailureHandler {
+    private final StopWithSavepointOperationHandler handler;
+    private final ExecutionGraph executionGraph;
+    private final StopWithSavepointOperations stopWithSavepointOperations;
+    private boolean operationFinished = false;
+
+    StopWithSavepointOperationManagerForAdaptiveScheduler(
+            ExecutionGraph executionGraph,
+            StopWithSavepointOperations stopWithSavepointOperations,
+            boolean terminate,
+            @Nullable String targetDirectory,
+            Executor mainThreadExecutor,
+            Logger logger) {
+
+        this.stopWithSavepointOperations = stopWithSavepointOperations;
+
+        // do not trigger checkpoints while creating the final savepoint. We 
will start the
+        // scheduler onLeave() again.
+        stopWithSavepointOperations.stopCheckpointScheduler();
+
+        final CompletableFuture<CompletedCheckpoint> savepointFuture =
+                
stopWithSavepointOperations.triggerSynchronousSavepoint(terminate, 
targetDirectory);
+
+        this.handler =
+                new StopWithSavepointOperationHandlerImpl(
+                        executionGraph.getJobID(), this, 
stopWithSavepointOperations, logger);
+
+        FutureUtils.assertNoException(
+                savepointFuture
+                        // the completedSavepointFuture could also be 
completed by
+                        // CheckpointCanceller which doesn't run in the 
mainThreadExecutor

Review comment:
       This is a very good finding.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/StopWithSavepointOperationManagerForAdaptiveScheduler.java
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.runtime.checkpoint.CompletedCheckpoint;
+import org.apache.flink.runtime.checkpoint.StopWithSavepointOperations;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import 
org.apache.flink.runtime.scheduler.stopwithsavepoint.StopWithSavepointOperationHandler;
+import 
org.apache.flink.runtime.scheduler.stopwithsavepoint.StopWithSavepointOperationHandlerImpl;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+
+import javax.annotation.Nullable;
+
+import java.util.Collection;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.stream.Collectors;
+
+/**
+ * A thin wrapper around the StopWithSavepointOperationHandler to handle 
global failures according
+ * to the needs of AdaptiveScheduler. AdaptiveScheduler currently doesn't 
support local failover,
+ * hence, any reported failure will lead to a transition to Restarting or 
Failing state.
+ */
+class StopWithSavepointOperationManagerForAdaptiveScheduler implements 
GlobalFailureHandler {
+    private final StopWithSavepointOperationHandler handler;
+    private final ExecutionGraph executionGraph;
+    private final StopWithSavepointOperations stopWithSavepointOperations;
+    private boolean operationFinished = false;
+
+    StopWithSavepointOperationManagerForAdaptiveScheduler(
+            ExecutionGraph executionGraph,
+            StopWithSavepointOperations stopWithSavepointOperations,
+            boolean terminate,
+            @Nullable String targetDirectory,
+            Executor mainThreadExecutor,
+            Logger logger) {
+
+        this.stopWithSavepointOperations = stopWithSavepointOperations;
+
+        // do not trigger checkpoints while creating the final savepoint. We 
will start the
+        // scheduler onLeave() again.
+        stopWithSavepointOperations.stopCheckpointScheduler();
+
+        final CompletableFuture<CompletedCheckpoint> savepointFuture =
+                
stopWithSavepointOperations.triggerSynchronousSavepoint(terminate, 
targetDirectory);
+
+        this.handler =
+                new StopWithSavepointOperationHandlerImpl(
+                        executionGraph.getJobID(), this, 
stopWithSavepointOperations, logger);
+
+        FutureUtils.assertNoException(
+                savepointFuture
+                        // the completedSavepointFuture could also be 
completed by
+                        // CheckpointCanceller which doesn't run in the 
mainThreadExecutor
+                        .handleAsync(
+                        (completedSavepoint, throwable) -> {
+                            
handler.handleSavepointCreation(completedSavepoint, throwable);

Review comment:
       I think we will stay forever in the state `StopWithSavepoint` if the 
savepoint future fails.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/StopWithSavepointOperationManagerForAdaptiveScheduler.java
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.runtime.checkpoint.CompletedCheckpoint;
+import org.apache.flink.runtime.checkpoint.StopWithSavepointOperations;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import 
org.apache.flink.runtime.scheduler.stopwithsavepoint.StopWithSavepointOperationHandler;
+import 
org.apache.flink.runtime.scheduler.stopwithsavepoint.StopWithSavepointOperationHandlerImpl;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+
+import javax.annotation.Nullable;
+
+import java.util.Collection;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.stream.Collectors;
+
+/**
+ * A thin wrapper around the StopWithSavepointOperationHandler to handle 
global failures according
+ * to the needs of AdaptiveScheduler. AdaptiveScheduler currently doesn't 
support local failover,
+ * hence, any reported failure will lead to a transition to Restarting or 
Failing state.
+ */
+class StopWithSavepointOperationManagerForAdaptiveScheduler implements 
GlobalFailureHandler {
+    private final StopWithSavepointOperationHandler handler;
+    private final ExecutionGraph executionGraph;
+    private final StopWithSavepointOperations stopWithSavepointOperations;
+    private boolean operationFinished = false;
+
+    StopWithSavepointOperationManagerForAdaptiveScheduler(
+            ExecutionGraph executionGraph,
+            StopWithSavepointOperations stopWithSavepointOperations,
+            boolean terminate,
+            @Nullable String targetDirectory,
+            Executor mainThreadExecutor,
+            Logger logger) {
+
+        this.stopWithSavepointOperations = stopWithSavepointOperations;
+
+        // do not trigger checkpoints while creating the final savepoint. We 
will start the
+        // scheduler onLeave() again.
+        stopWithSavepointOperations.stopCheckpointScheduler();
+
+        final CompletableFuture<CompletedCheckpoint> savepointFuture =
+                
stopWithSavepointOperations.triggerSynchronousSavepoint(terminate, 
targetDirectory);
+
+        this.handler =
+                new StopWithSavepointOperationHandlerImpl(
+                        executionGraph.getJobID(), this, 
stopWithSavepointOperations, logger);
+
+        FutureUtils.assertNoException(
+                savepointFuture
+                        // the completedSavepointFuture could also be 
completed by
+                        // CheckpointCanceller which doesn't run in the 
mainThreadExecutor
+                        .handleAsync(
+                        (completedSavepoint, throwable) -> {
+                            
handler.handleSavepointCreation(completedSavepoint, throwable);
+                            return null;
+                        },
+                        mainThreadExecutor));
+
+        this.executionGraph = executionGraph;
+    }
+
+    public CompletableFuture<String> getSavepointPathFuture() {
+        return handler.getSavepointPathFuture();
+    }
+
+    public void onError(Throwable cause) {
+        operationFinished = true;
+        handler.abortOperation(cause);
+    }
+
+    public void onLeave() {
+        // if we are leaving to Finished state (on success), all executions 
will be in final
+        // state, the handler will complete the stop with savepoint future
+        // if we are in any error state, the StopWithSavepoint state will 
decide what to do.
+        operationFinished = true;
+        Collection<ExecutionState> executionsState =
+                executionGraph.getRegisteredExecutions().values().stream()
+                        .map(Execution::getState)
+                        .collect(Collectors.toList());
+        handler.handleExecutionsTermination(executionsState);
+
+        // make sure we are reactivating the scheduler in all cases.
+        stopWithSavepointOperations.startCheckpointScheduler();
+    }
+
+    @Override
+    public void handleGlobalFailure(Throwable cause) {
+        // ignore failures reported from the 
StopWithSavepointOperationHandlerImpl.
+        Preconditions.checkState(
+                operationFinished,
+                "Unexpected failure. Assuming failures only after operation 
has finished.");

Review comment:
       It is a bit confusing that we only expect global failures to happen 
after the operation is finished. I took me a while to understand the reason tbh.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/StopWithSavepointOperationManagerForAdaptiveScheduler.java
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.runtime.checkpoint.CompletedCheckpoint;
+import org.apache.flink.runtime.checkpoint.StopWithSavepointOperations;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import 
org.apache.flink.runtime.scheduler.stopwithsavepoint.StopWithSavepointOperationHandler;
+import 
org.apache.flink.runtime.scheduler.stopwithsavepoint.StopWithSavepointOperationHandlerImpl;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+
+import javax.annotation.Nullable;
+
+import java.util.Collection;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.stream.Collectors;
+
+/**
+ * A thin wrapper around the StopWithSavepointOperationHandler to handle 
global failures according
+ * to the needs of AdaptiveScheduler. AdaptiveScheduler currently doesn't 
support local failover,
+ * hence, any reported failure will lead to a transition to Restarting or 
Failing state.
+ */
+class StopWithSavepointOperationManagerForAdaptiveScheduler implements 
GlobalFailureHandler {
+    private final StopWithSavepointOperationHandler handler;
+    private final ExecutionGraph executionGraph;
+    private final StopWithSavepointOperations stopWithSavepointOperations;
+    private boolean operationFinished = false;
+
+    StopWithSavepointOperationManagerForAdaptiveScheduler(
+            ExecutionGraph executionGraph,
+            StopWithSavepointOperations stopWithSavepointOperations,
+            boolean terminate,
+            @Nullable String targetDirectory,
+            Executor mainThreadExecutor,
+            Logger logger) {
+
+        this.stopWithSavepointOperations = stopWithSavepointOperations;
+
+        // do not trigger checkpoints while creating the final savepoint. We 
will start the
+        // scheduler onLeave() again.
+        stopWithSavepointOperations.stopCheckpointScheduler();
+
+        final CompletableFuture<CompletedCheckpoint> savepointFuture =
+                
stopWithSavepointOperations.triggerSynchronousSavepoint(terminate, 
targetDirectory);
+
+        this.handler =
+                new StopWithSavepointOperationHandlerImpl(
+                        executionGraph.getJobID(), this, 
stopWithSavepointOperations, logger);
+
+        FutureUtils.assertNoException(
+                savepointFuture
+                        // the completedSavepointFuture could also be 
completed by
+                        // CheckpointCanceller which doesn't run in the 
mainThreadExecutor
+                        .handleAsync(
+                        (completedSavepoint, throwable) -> {
+                            
handler.handleSavepointCreation(completedSavepoint, throwable);
+                            return null;
+                        },
+                        mainThreadExecutor));
+
+        this.executionGraph = executionGraph;
+    }
+
+    public CompletableFuture<String> getSavepointPathFuture() {
+        return handler.getSavepointPathFuture();
+    }
+
+    public void onError(Throwable cause) {
+        operationFinished = true;
+        handler.abortOperation(cause);
+    }
+
+    public void onLeave() {
+        // if we are leaving to Finished state (on success), all executions 
will be in final
+        // state, the handler will complete the stop with savepoint future
+        // if we are in any error state, the StopWithSavepoint state will 
decide what to do.
+        operationFinished = true;
+        Collection<ExecutionState> executionsState =
+                executionGraph.getRegisteredExecutions().values().stream()
+                        .map(Execution::getState)
+                        .collect(Collectors.toList());
+        handler.handleExecutionsTermination(executionsState);

Review comment:
       Intuitively, I would have expected an `handler.abort` call here instead.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/StopWithSavepointOperationManagerForAdaptiveScheduler.java
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.runtime.checkpoint.CompletedCheckpoint;
+import org.apache.flink.runtime.checkpoint.StopWithSavepointOperations;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import 
org.apache.flink.runtime.scheduler.stopwithsavepoint.StopWithSavepointOperationHandler;
+import 
org.apache.flink.runtime.scheduler.stopwithsavepoint.StopWithSavepointOperationHandlerImpl;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+
+import javax.annotation.Nullable;
+
+import java.util.Collection;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.stream.Collectors;
+
+/**
+ * A thin wrapper around the StopWithSavepointOperationHandler to handle 
global failures according
+ * to the needs of AdaptiveScheduler. AdaptiveScheduler currently doesn't 
support local failover,
+ * hence, any reported failure will lead to a transition to Restarting or 
Failing state.
+ */
+class StopWithSavepointOperationManagerForAdaptiveScheduler implements 
GlobalFailureHandler {
+    private final StopWithSavepointOperationHandler handler;
+    private final ExecutionGraph executionGraph;
+    private final StopWithSavepointOperations stopWithSavepointOperations;
+    private boolean operationFinished = false;
+
+    StopWithSavepointOperationManagerForAdaptiveScheduler(
+            ExecutionGraph executionGraph,
+            StopWithSavepointOperations stopWithSavepointOperations,
+            boolean terminate,
+            @Nullable String targetDirectory,
+            Executor mainThreadExecutor,
+            Logger logger) {
+
+        this.stopWithSavepointOperations = stopWithSavepointOperations;
+
+        // do not trigger checkpoints while creating the final savepoint. We 
will start the
+        // scheduler onLeave() again.
+        stopWithSavepointOperations.stopCheckpointScheduler();
+
+        final CompletableFuture<CompletedCheckpoint> savepointFuture =
+                
stopWithSavepointOperations.triggerSynchronousSavepoint(terminate, 
targetDirectory);
+
+        this.handler =
+                new StopWithSavepointOperationHandlerImpl(
+                        executionGraph.getJobID(), this, 
stopWithSavepointOperations, logger);
+
+        FutureUtils.assertNoException(
+                savepointFuture
+                        // the completedSavepointFuture could also be 
completed by
+                        // CheckpointCanceller which doesn't run in the 
mainThreadExecutor
+                        .handleAsync(
+                        (completedSavepoint, throwable) -> {
+                            
handler.handleSavepointCreation(completedSavepoint, throwable);
+                            return null;
+                        },
+                        mainThreadExecutor));

Review comment:
       We could also remove this special handling here by saying that 
`StopWithSavepointOperations.triggerSynchronousSavepoint` must return a future 
which is completed in the main thread. That way we move the problem out of this 
class. This reduces a bit of complexity here.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointOperationHandlerImpl.java
##########
@@ -236,10 +260,17 @@ public State onExecutionsFinished() {
         @Override
         public State onAnyExecutionNotFinished(
                 Iterable<ExecutionState> notFinishedExecutionStates) {
-            terminateExceptionallyWithGlobalFailover(
+            terminateExceptionallyWithGlobalFailoverOnUnfinishedExecution(
                     notFinishedExecutionStates, 
completedSavepoint.getExternalPointer());
             return new FinalState();
         }
+
+        @Override
+        public State onAbort(Throwable cause) {
+            terminateExceptionallyWithGlobalFailover(
+                    cause, completedSavepoint.getExternalPointer());

Review comment:
       Why do we fail globally `onAbort`? Wouldn't it make more sense to say 
that `onAbort` stops the stop with savepoint operation but the caller has to 
make sure that the system continues correctly?

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointOperationHandlerImpl.java
##########
@@ -217,6 +230,17 @@ public State onSavepointCreationFailure(Throwable 
throwable) {
             terminateExceptionally(throwable);
             return new FinalState();
         }
+
+        @Override
+        public State onAbort(Throwable cause) {
+            return onSavepointCreationFailure(cause);

Review comment:
       Why does `onAbort` trigger a savepoint creation failure? Couldn't it 
transition to `Finished` and fail the operation future?

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/StopWithSavepoint.java
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.StopWithSavepointOperations;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.TaskExecutionStateTransition;
+import org.apache.flink.runtime.scheduler.ExecutionGraphHandler;
+import org.apache.flink.runtime.scheduler.OperatorCoordinatorHandler;
+import 
org.apache.flink.runtime.scheduler.stopwithsavepoint.StopWithSavepointOperationHandler;
+import 
org.apache.flink.runtime.scheduler.stopwithsavepoint.StopWithSavepointOperationManager;
+
+import org.slf4j.Logger;
+
+import javax.annotation.Nullable;
+
+import java.time.Duration;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * When a "stop with savepoint" operation (wait until savepoint has been 
created, then cancel job)
+ * is triggered on the {@link Executing} state, we transition into this state. 
This state is
+ * delegating the tracking of the stop with savepoint operation to the {@link
+ * StopWithSavepointOperationManagerForAdaptiveScheduler} which tracks the 
operation through the
+ * {@link StopWithSavepointOperationHandler}. This allows us to share the 
operation tracking logic
+ * across all scheduler implementations.
+ */
+class StopWithSavepoint extends StateWithExecutionGraph {
+
+    private final Context context;
+    private final ClassLoader userCodeClassLoader;
+
+    private final StopWithSavepointOperationManagerForAdaptiveScheduler
+            stopWithSavepointOperationManager;
+
+    StopWithSavepoint(
+            Context context,
+            ExecutionGraph executionGraph,
+            ExecutionGraphHandler executionGraphHandler,
+            OperatorCoordinatorHandler operatorCoordinatorHandler,
+            StopWithSavepointOperations stopWithSavepointOperations,
+            Logger logger,
+            ClassLoader userCodeClassLoader,
+            @Nullable String targetDirectory,
+            boolean terminate) {
+        super(context, executionGraph, executionGraphHandler, 
operatorCoordinatorHandler, logger);
+        this.context = context;
+        this.userCodeClassLoader = userCodeClassLoader;
+
+        stopWithSavepointOperationManager =
+                new StopWithSavepointOperationManagerForAdaptiveScheduler(
+                        executionGraph,
+                        stopWithSavepointOperations,
+                        terminate,
+                        targetDirectory,
+                        context.getMainThreadExecutor(),
+                        logger);
+    }
+
+    @Override
+    public void onLeave(Class<? extends State> newState) {
+        stopWithSavepointOperationManager.onLeave();
+
+        super.onLeave(newState);
+    }
+
+    @Override
+    public void cancel() {
+        context.goToCanceling(
+                getExecutionGraph(), getExecutionGraphHandler(), 
getOperatorCoordinatorHandler());
+    }
+
+    @Override
+    public JobStatus getJobStatus() {
+        return JobStatus.RUNNING;
+    }
+
+    @Override
+    public void handleGlobalFailure(Throwable cause) {
+        handleAnyFailure(cause);
+    }
+
+    /**
+     * The {@code executionTerminationsFuture} will complete if a task reached 
a terminal state, and
+     * {@link StopWithSavepointOperationManager} will act accordingly.
+     */
+    @Override
+    boolean updateTaskExecutionState(TaskExecutionStateTransition 
taskExecutionStateTransition) {
+        final boolean successfulUpdate =
+                getExecutionGraph().updateState(taskExecutionStateTransition);
+
+        if (successfulUpdate) {
+            if (taskExecutionStateTransition.getExecutionState() == 
ExecutionState.FAILED) {
+                Throwable cause = 
taskExecutionStateTransition.getError(userCodeClassLoader);
+                handleAnyFailure(cause);
+            }
+        }
+
+        return successfulUpdate;
+    }
+
+    @Override
+    void onGloballyTerminalState(JobStatus globallyTerminalState) {
+        
context.goToFinished(ArchivedExecutionGraph.createFrom(getExecutionGraph()));
+    }
+
+    private void handleAnyFailure(Throwable cause) {
+        stopWithSavepointOperationManager.onError(cause);
+
+        final Executing.FailureResult failureResult = 
context.howToHandleFailure(cause);
+
+        if (failureResult.canRestart()) {
+            context.goToRestarting(
+                    getExecutionGraph(),
+                    getExecutionGraphHandler(),
+                    getOperatorCoordinatorHandler(),
+                    failureResult.getBackoffTime());
+        } else {
+            context.goToFailing(
+                    getExecutionGraph(),
+                    getExecutionGraphHandler(),
+                    getOperatorCoordinatorHandler(),
+                    failureResult.getFailureCause());
+        }
+    }
+
+    CompletableFuture<String> getOperationCompletionFuture() {
+        return stopWithSavepointOperationManager.getSavepointPathFuture();
+    }
+
+    interface Context extends StateWithExecutionGraph.Context {
+        /**
+         * Asks how to handle the failure.
+         *
+         * @param failure failure describing the failure cause
+         * @return {@link Executing.FailureResult} which describes how to 
handle the failure
+         */
+        Executing.FailureResult howToHandleFailure(Throwable failure);
+
+        /**
+         * Transitions into the {@link Canceling} state.
+         *
+         * @param executionGraph executionGraph to pass to the {@link 
Canceling} state
+         * @param executionGraphHandler executionGraphHandler to pass to the 
{@link Canceling} state
+         * @param operatorCoordinatorHandler operatorCoordinatorHandler to 
pass to the {@link
+         *     Canceling} state
+         */
+        void goToCanceling(
+                ExecutionGraph executionGraph,
+                ExecutionGraphHandler executionGraphHandler,
+                OperatorCoordinatorHandler operatorCoordinatorHandler);
+
+        /**
+         * Transitions into the {@link Restarting} state.
+         *
+         * @param executionGraph executionGraph to pass to the {@link 
Restarting} state
+         * @param executionGraphHandler executionGraphHandler to pass to the 
{@link Restarting}
+         *     state
+         * @param operatorCoordinatorHandler operatorCoordinatorHandler to pas 
to the {@link
+         *     Restarting} state
+         * @param backoffTime backoffTime to wait before transitioning to the 
{@link Restarting}
+         *     state
+         */
+        void goToRestarting(
+                ExecutionGraph executionGraph,
+                ExecutionGraphHandler executionGraphHandler,
+                OperatorCoordinatorHandler operatorCoordinatorHandler,
+                Duration backoffTime);
+
+        /**
+         * Transitions into the {@link Failing} state.
+         *
+         * @param executionGraph executionGraph to pass to the {@link Failing} 
state
+         * @param executionGraphHandler executionGraphHandler to pass to the 
{@link Failing} state
+         * @param operatorCoordinatorHandler operatorCoordinatorHandler to 
pass to the {@link
+         *     Failing} state
+         * @param failureCause failureCause describing why the job execution 
failed
+         */
+        void goToFailing(
+                ExecutionGraph executionGraph,
+                ExecutionGraphHandler executionGraphHandler,
+                OperatorCoordinatorHandler operatorCoordinatorHandler,
+                Throwable failureCause);

Review comment:
       How are we going back to `Executing` if the savepoint future failed? Or 
can this not happen?




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