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



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/StopWithSavepointTerminationHandlerImpl.java
##########
@@ -0,0 +1,258 @@
+/*
+ * 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;
+
+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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.util.FlinkException;
+
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+
+import javax.annotation.Nonnull;
+
+import java.util.Collection;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+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 CompletableFuture<String> result = new CompletableFuture<>();
+
+    private State state = new WaitingForSavepoint();
+
+    public <S extends SchedulerNG & CheckpointScheduling> 
StopWithSavepointTerminationHandlerImpl(
+            @Nonnull JobID jobId, @Nonnull S schedulerWithCheckpointing, 
@Nonnull Logger log) {
+        this(jobId, schedulerWithCheckpointing, schedulerWithCheckpointing, 
log);
+    }
+
+    @VisibleForTesting
+    StopWithSavepointTerminationHandlerImpl(
+            @Nonnull JobID jobId,
+            @Nonnull SchedulerNG scheduler,
+            @Nonnull CheckpointScheduling checkpointScheduling,
+            @Nonnull Logger log) {
+        this.jobId = checkNotNull(jobId);
+        this.scheduler = checkNotNull(scheduler);
+        this.checkpointScheduling = checkNotNull(checkpointScheduling);
+        this.log = checkNotNull(log);
+    }
+
+    @Override
+    public CompletableFuture<String> handlesStopWithSavepointTermination(
+            CompletableFuture<CompletedCheckpoint> completedSavepointFuture,
+            CompletableFuture<Collection<ExecutionState>> 
terminatedExecutionsFuture,
+            ComponentMainThreadExecutor mainThreadExecutor) {
+        completedSavepointFuture
+                .whenCompleteAsync(
+                        (completedSavepoint, throwable) -> {
+                            if (throwable != null) {
+                                handleSavepointCreationFailure(throwable);
+                            } else {
+                                handleSavepointCreation(completedSavepoint);
+                            }
+                        },
+                        mainThreadExecutor)
+                .thenCompose(
+                        aVoid ->
+                                terminatedExecutionsFuture.thenAcceptAsync(
+                                        this::handleExecutionsTermination, 
mainThreadExecutor));
+
+        return result;
+    }
+
+    private synchronized void handleSavepointCreation(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 synchronized 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 synchronized void handleExecutionsTermination(
+            Collection<ExecutionState> executionStates) {
+        final State oldState = state;
+        state = state.onExecutionsTermination(executionStates);
+
+        log.debug(
+                "Stop-with-savepoint transitioned from {} to {} on execution 
termination handling for job {}.",
+                oldState,
+                state,
+                jobId);
+    }
+
+    /**
+     * 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);
+
+        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 left it in `StpWithSavepointTerminationHandlerImpl` because it's 
part of the actual handle logic and therefore is easier to be tested there.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to