tillrohrmann commented on a change in pull request #14948: URL: https://github.com/apache/flink/pull/14948#discussion_r593135816
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/StopWithSavepoint.java ########## @@ -0,0 +1,317 @@ +/* + * 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.CheckpointCoordinator; +import org.apache.flink.runtime.checkpoint.CheckpointException; +import org.apache.flink.runtime.checkpoint.CheckpointFailureReason; +import org.apache.flink.runtime.checkpoint.CompletedCheckpoint; +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.SchedulerBase; +import org.apache.flink.runtime.scheduler.SchedulerUtils; +import org.apache.flink.runtime.scheduler.stopwithsavepoint.StopWithSavepointOperationHandlerImpl; +import org.apache.flink.runtime.scheduler.stopwithsavepoint.StopWithSavepointOperationManager; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; + +import javax.annotation.Nullable; + +import java.time.Duration; +import java.util.Collection; +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 + * StopWithSavepointOperationManager}, which is shared with {@link SchedulerBase}. + */ +class StopWithSavepoint extends StateWithExecutionGraph implements StopWithSavepointOperations { + + private final CompletableFuture<String> operationCompletionFuture; + private final Context context; + private final ClassLoader userCodeClassLoader; + private final StopWithSavepointOperationManager stopWithSavepointOperationManager; + + StopWithSavepoint( + Context context, + ExecutionGraph executionGraph, + ExecutionGraphHandler executionGraphHandler, + OperatorCoordinatorHandler operatorCoordinatorHandler, + Logger logger, + ClassLoader userCodeClassLoader, + @Nullable String targetDirectory, + boolean terminate) { + super(context, executionGraph, executionGraphHandler, operatorCoordinatorHandler, logger); + this.context = context; + this.userCodeClassLoader = userCodeClassLoader; + + // to ensure that all disjoint subgraphs of a job finish successfully on savepoint creation, + // we track the job termination via all execution termination futures (FLINK-21030). + final CompletableFuture<Collection<ExecutionState>> executionTerminationsFuture = + SchedulerUtils.getCombinedExecutionTerminationFuture(executionGraph); + + stopWithSavepointOperationManager = + new StopWithSavepointOperationManager( + this, + new StopWithSavepointOperationHandlerImpl( + executionGraph.getJobID(), context, this, logger)); + + this.operationCompletionFuture = + stopWithSavepointOperationManager.trackStopWithSavepoint( + terminate, + targetDirectory, + executionTerminationsFuture, + context.getMainThreadExecutor()); + } + + @Override + public void onLeave(Class<? extends State> newState) { + // we need to delay the execution because this may trigger a state transition of the + // AdaptiveScheduler. + context.getMainThreadExecutor() + .execute( + () -> + stopWithSavepointOperationManager.abortOperation( + new CheckpointException( + "Aborting stop with savepoint operation for transition to state " + + newState.getSimpleName() + + ".", + CheckpointFailureReason.CHECKPOINT_ABORTED))); + + 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) { + 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 operationCompletionFuture; + } + + @Override + public void startCheckpointScheduler() { + final CheckpointCoordinator coordinator = getExecutionGraph().getCheckpointCoordinator(); + if (coordinator == null) { + if (!getExecutionGraph().getState().isTerminalState()) { + // for a streaming job, the checkpoint coordinator is always set (even if periodic + // checkpoints are disabled). The only situation where it can be null is when the + // job reached a terminal state. + throw new IllegalStateException( + "Coordinator is only allowed to be null if we are in a terminal state."); + } + return; + } + if (coordinator.isPeriodicCheckpointingConfigured()) { + coordinator.startCheckpointScheduler(); + } + } + + @Override + public void stopCheckpointScheduler() { + final CheckpointCoordinator coordinator = getExecutionGraph().getCheckpointCoordinator(); + if (coordinator != null) { + coordinator.stopCheckpointScheduler(); + } + } + + @Override + public CompletableFuture<CompletedCheckpoint> triggerSynchronousSavepoint( + boolean terminate, @Nullable String targetLocation) { + Preconditions.checkNotNull( + getExecutionGraph().getCheckpointCoordinator(), + "Checkpoint coordinator must be set for this operation. Is this a streaming job?"); + return getExecutionGraph() + .getCheckpointCoordinator() + .triggerSynchronousSavepoint(terminate, targetLocation); Review comment: I can't remember either. ---------------------------------------------------------------- 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