XComp commented on a change in pull request #14847: URL: https://github.com/apache/flink/pull/14847#discussion_r581376284
########## 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: I went for the easy fix to add a graceful shutdown to the `JobManagerSharedServices` for now after discussing solutions with @zentol today. Here's what we came up with: The `StopWithSavepointTerminationHandler` interface would need to be adapted: Instead of providing a dedicated method for returning the result, `StopWithSavepointTerminationHandler.handleExecutionsTermination` has to return the `CompletableFuture` result. This enables us to add an optional execution for discarding the savepoint in case of global fail-over. The `StopWithSavepointTerminationManager` would forward this `CompletableFuture` as a result. In case of global fail-over, the result would only complete if the discard call terminated. The `CompletableFuture` returned by the `StopWithSavepointTerminationManager` could be collected by the `SchedulerBase` now. Multiple calls to `stopWithSavepoint` would result in combining the `CompletableFuture` results into a single `CompletableFuture`. We could then wait in `SchedulerBase.shutDownCheckpointServices` for this `CompletableFuture` to finish. It gets executed when the termination future of the underlying `ExecutionGraph` terminates. The drawback of this approach is that we would block in the main executor which might go against the rule of not having blocking operations on the main executor. The solution would be to introduce a terminationFuture that only completes if the executions are done and is then handled in `JobMaster.stopScheduling` which is called by `JobMaster.stopJobExecution`. It felt like it's getting a bit out of scope for the issue here - that's why we decided to go for the graceful shutdown approach for now and discuss the solution first. ---------------------------------------------------------------- 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