tillrohrmann commented on a change in pull request #14948: URL: https://github.com/apache/flink/pull/14948#discussion_r591459690
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java ########## @@ -930,7 +952,7 @@ public void runIfState(State expectedState, Runnable action, Duration delay) { /** Note: Do not call this method from a State constructor or State#onLeave. */ @VisibleForTesting - void transitionToState(StateFactory<?> targetState) { + <T extends State> T transitionToState(StateFactory<T> targetState) { Review comment: JavaDocs could be helpful explaining how this methods behaves. ########## 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; Review comment: `checkState`? ########## 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))); Review comment: I would suggest to not let the `stopWithSavepointOperationManager` trigger state transitions after we have left the `StopWithSavepoint` state. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointOperationHandlerImpl.java ########## @@ -113,91 +115,100 @@ public void handleExecutionsTermination(Collection<ExecutionState> terminatedExe } } - private void handleSavepointCreationSuccess(CompletedCheckpoint completedCheckpoint) { - final State oldState = state; - state = state.onSavepointCreation(completedCheckpoint); + @Override + public void abortOperation(Throwable cause) { + transitionToState(() -> state.onAbort(cause), String.format("abort for job %s.", jobId)); + } - log.debug( - "Stop-with-savepoint transitioned from {} to {} on savepoint creation handling for job {}.", - oldState.getName(), - state.getName(), - jobId); + private void handleSavepointCreationSuccess(CompletedCheckpoint completedCheckpoint) { + transitionToState( + () -> state.onSavepointCreation(completedCheckpoint), + String.format("savepoint creation handling for job %s.", 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.getName(), - state.getName(), - jobId); + transitionToState( + () -> state.onSavepointCreationFailure(throwable), + String.format("savepoint creation failure handling for job %s.", 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.getName(), - state.getName(), - jobId); + transitionToState( + () -> state.onExecutionsFinished(), + String.format( + "execution termination handling with all executions being finished for job %s.", + jobId)); } private void handleAnyExecutionNotFinished(Set<ExecutionState> notFinishedExecutionStates) { + transitionToState( + () -> state.onAnyExecutionNotFinished(notFinishedExecutionStates), + String.format( + "execution termination handling for job %s with some executions being in an not-finished state: %s", + jobId, notFinishedExecutionStates)); + } + + private void transitionToState(Supplier<? extends State> targetState, String message) { + Preconditions.checkState( + !inStateTransition, + "Can not transition to new state because we are already in a state transition"); final State oldState = state; - state = state.onAnyExecutionNotFinished(notFinishedExecutionStates); + inStateTransition = true; Review comment: Can it happen that we trigger a new state transition in the constructor of another state here? ########## 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 { Review comment: I would suggest to not let the `StopWithSavepoint` directly implement the `StopWithSavepointOperations`. That way it wouldn't be exposed to the users of the `StopWithSavepoint` class. Moreover, one could implement an inner class which controls whether to forward the calls or not depending on the internal state of `StopWithSavepoint`. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/Executing.java ########## @@ -206,6 +228,26 @@ void goToFailing( ExecutionGraphHandler executionGraphHandler, OperatorCoordinatorHandler operatorCoordinatorHandler, Throwable failureCause); + + /** + * Transitions into the {@link StopWithSavepoint} state. + * + * @param executionGraph executionGraph to pass to the {@link StopWithSavepoint} state + * @param executionGraphHandler executionGraphHandler to pass to the {@link + * StopWithSavepoint} state + * @param operatorCoordinatorHandler operatorCoordinatorHandler to pass to the {@link + * StopWithSavepoint} state + * @param targetDirectory target for the savepoint Review comment: What happens if `targetDirectory` is `null`? ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/StopWithSavepointTest.java ########## @@ -0,0 +1,458 @@ +/* + * 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.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.JobException; +import org.apache.flink.runtime.checkpoint.CheckpointProperties; +import org.apache.flink.runtime.checkpoint.CompletedCheckpoint; +import org.apache.flink.runtime.client.JobExecutionException; +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.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder; +import org.apache.flink.runtime.scheduler.ExecutionGraphHandler; +import org.apache.flink.runtime.scheduler.OperatorCoordinatorHandler; +import org.apache.flink.runtime.state.TestingStreamStateHandle; +import org.apache.flink.runtime.state.testutils.TestCompletedCheckpointStorageLocation; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; +import org.slf4j.Logger; + +import javax.annotation.Nullable; + +import java.time.Duration; +import java.util.Collections; +import java.util.HashMap; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ForkJoinPool; +import java.util.function.Consumer; +import java.util.function.Function; + +import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; +import static org.apache.flink.runtime.scheduler.adaptive.ExecutingTest.createFailingStateTransition; +import static org.apache.flink.runtime.scheduler.adaptive.WaitingForResourcesTest.assertNonNull; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.junit.Assert.assertThat; + +/** Tests for the {@link StopWithSavepoint} state. */ +public class StopWithSavepointTest extends TestLogger { + + @Test + public void testCancel() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + ctx.setExpectCancelling(assertNonNull()); + + sws.cancel(); + } + } + + @Test + public void testSuspend() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + ctx.setExpectFinished( + archivedExecutionGraph -> { + assertThat(archivedExecutionGraph.getState(), is(JobStatus.SUSPENDED)); + }); + + sws.suspend(new RuntimeException()); + } + } + + @Test + public void testRestartOnGlobalFailureIfRestartConfigured() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + ctx.setHowToHandleFailure( + (ignore) -> Executing.FailureResult.canRestart(Duration.ZERO)); + + ctx.setExpectRestarting(assertNonNull()); + + sws.handleGlobalFailure(new RuntimeException()); + } + } + + @Test + public void testFailingOnGlobalFailureIfNoRestartConfigured() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + ctx.setHowToHandleFailure(Executing.FailureResult::canNotRestart); + + ctx.setExpectFailing( + failingArguments -> { + assertThat( + failingArguments.getFailureCause(), + containsCause(RuntimeException.class)); + }); + + sws.handleGlobalFailure(new RuntimeException()); + } + } + + @Test + public void testFailingOnUpdateTaskExecutionStateWithNoRestart() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + + TestingStopWithSavepoint sws = + createStopWithSavepoint(ctx, new StateTrackingMockExecutionGraph()); + ctx.setHowToHandleFailure(Executing.FailureResult::canNotRestart); + + ctx.setExpectFailing( + failingArguments -> { + assertThat( + failingArguments.getFailureCause(), + containsCause(RuntimeException.class)); + }); + + sws.updateTaskExecutionState(createFailingStateTransition()); + } + } + + @Test + public void testRestartingOnUpdateTaskExecutionStateWithRestart() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + + TestingStopWithSavepoint sws = + createStopWithSavepoint(ctx, new StateTrackingMockExecutionGraph()); + ctx.setHowToHandleFailure( + (ignore) -> Executing.FailureResult.canRestart(Duration.ZERO)); + + ctx.setExpectRestarting(assertNonNull()); + + sws.updateTaskExecutionState(createFailingStateTransition()); + } + } + + @Test + public void testExceptionalFutureCompletionOnLeaveWhileWaitingOnSavepointCompletion() + throws Exception { + MockStopWithSavepointContext ctx = new MockStopWithSavepointContext(); + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + + sws.onLeave(Canceling.class); + + ctx.close(); + assertThat(sws.getOperationCompletionFuture().isCompletedExceptionally(), is(true)); + } + + @Test + public void testExceptionalFutureCompletionAndStateTransitionOnLeaveAfterSavepointCompletion() + throws Exception { + MockStopWithSavepointContext ctx = new MockStopWithSavepointContext(); + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + ctx.setGlobalFailureHandler(sws); + ctx.setHowToHandleFailure((ignore) -> Executing.FailureResult.canRestart(Duration.ZERO)); + ctx.setExpectRestarting(assertNonNull()); + sws.getSavepointFuture().complete(createCompletedSavepoint()); + + sws.onLeave(Canceling.class); + + ctx.close(); + assertThat(sws.getOperationCompletionFuture().isCompletedExceptionally(), is(true)); + } + + @Test + public void testExceptionalSavepointCompletion() throws Exception { + MockStopWithSavepointContext ctx = new MockStopWithSavepointContext(); + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + + sws.getSavepointFuture().completeExceptionally(new RuntimeException("Test error")); + + ctx.close(); + assertThat(sws.getOperationCompletionFuture().isCompletedExceptionally(), is(true)); + assertThat(ctx.hadStateTransition, is(false)); + } + + @Test + public void testRestartOnTaskFailureAfterSavepointCompletion() throws Exception { + MockStopWithSavepointContext ctx = new MockStopWithSavepointContext(); + TerminateExecutionFutureMockedExecutionGraph executionGraph = + new TerminateExecutionFutureMockedExecutionGraph(); + + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx, executionGraph); + ctx.setGlobalFailureHandler(sws); + + ctx.setHowToHandleFailure((ignore) -> Executing.FailureResult.canRestart(Duration.ZERO)); + + ctx.setExpectRestarting(assertNonNull()); + + sws.getSavepointFuture().complete(createCompletedSavepoint()); + executionGraph.getExecutionFuture().complete(ExecutionState.FAILED); Review comment: Technically, we rely on an implementation detail here. I think it would be correct to call `sws.updateTaskExecutionState` with a failed state. ########## 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( Review comment: I think the problem you are trying to work around with this construct is that the `stopWithSavepointOperationManager.abortOperation` can trigger a `handleGlobalFailure` if it happens after the savepoint has been created. With this change we will trigger a global failure on the next state. I think this is not correct as we should only leave the `StopWithSavepoint` state if the stop with savepoint operation has been completed (one way or the other). ########## File path: flink-tests/src/test/java/org/apache/flink/test/scheduling/AdaptiveSchedulerITCase.java ########## @@ -88,6 +123,189 @@ public void testGlobalFailoverCanRecoverState() throws Exception { env.execute(); } + private enum StopWithSavepointTestBehavior { + NO_FAILURE, + FAIL_ON_CHECKPOINT, + FAIL_ON_STOP, + FAIL_ON_FIRST_CHECKPOINT_ONLY + } + + @Test + public void testStopWithSavepointNoError() throws Exception { + StreamExecutionEnvironment env = getEnvWithSource(StopWithSavepointTestBehavior.NO_FAILURE); + + DummySource.resetForParallelism(PARALLELISM); + + JobClient client = env.executeAsync(); + + DummySource.awaitRunning(); + + final File savepointDirectory = tempFolder.newFolder("savepoint"); + final String savepoint = + client.stopWithSavepoint(false, savepointDirectory.getAbsolutePath()).get(); + assertThat(savepoint, containsString(savepointDirectory.getAbsolutePath())); + assertThat(client.getJobStatus().get(), is(JobStatus.FINISHED)); + } + + @Test + public void testStopWithSavepointFailOnCheckpoint() throws Exception { Review comment: Are we sure that a savepoint can only fail if a task fails as well? If not, then we are lacking a test where only the savepoint fails but not recovery will be triggered. ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/StopWithSavepointTest.java ########## @@ -0,0 +1,458 @@ +/* + * 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.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.JobException; +import org.apache.flink.runtime.checkpoint.CheckpointProperties; +import org.apache.flink.runtime.checkpoint.CompletedCheckpoint; +import org.apache.flink.runtime.client.JobExecutionException; +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.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder; +import org.apache.flink.runtime.scheduler.ExecutionGraphHandler; +import org.apache.flink.runtime.scheduler.OperatorCoordinatorHandler; +import org.apache.flink.runtime.state.TestingStreamStateHandle; +import org.apache.flink.runtime.state.testutils.TestCompletedCheckpointStorageLocation; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; +import org.slf4j.Logger; + +import javax.annotation.Nullable; + +import java.time.Duration; +import java.util.Collections; +import java.util.HashMap; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ForkJoinPool; +import java.util.function.Consumer; +import java.util.function.Function; + +import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; +import static org.apache.flink.runtime.scheduler.adaptive.ExecutingTest.createFailingStateTransition; +import static org.apache.flink.runtime.scheduler.adaptive.WaitingForResourcesTest.assertNonNull; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.junit.Assert.assertThat; + +/** Tests for the {@link StopWithSavepoint} state. */ +public class StopWithSavepointTest extends TestLogger { + + @Test + public void testCancel() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + ctx.setExpectCancelling(assertNonNull()); + + sws.cancel(); + } + } + + @Test + public void testSuspend() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + ctx.setExpectFinished( + archivedExecutionGraph -> { + assertThat(archivedExecutionGraph.getState(), is(JobStatus.SUSPENDED)); + }); + + sws.suspend(new RuntimeException()); + } + } + + @Test + public void testRestartOnGlobalFailureIfRestartConfigured() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + ctx.setHowToHandleFailure( + (ignore) -> Executing.FailureResult.canRestart(Duration.ZERO)); + + ctx.setExpectRestarting(assertNonNull()); + + sws.handleGlobalFailure(new RuntimeException()); + } + } + + @Test + public void testFailingOnGlobalFailureIfNoRestartConfigured() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + ctx.setHowToHandleFailure(Executing.FailureResult::canNotRestart); + + ctx.setExpectFailing( + failingArguments -> { + assertThat( + failingArguments.getFailureCause(), + containsCause(RuntimeException.class)); + }); + + sws.handleGlobalFailure(new RuntimeException()); + } + } + + @Test + public void testFailingOnUpdateTaskExecutionStateWithNoRestart() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + + TestingStopWithSavepoint sws = + createStopWithSavepoint(ctx, new StateTrackingMockExecutionGraph()); + ctx.setHowToHandleFailure(Executing.FailureResult::canNotRestart); + + ctx.setExpectFailing( + failingArguments -> { + assertThat( + failingArguments.getFailureCause(), + containsCause(RuntimeException.class)); + }); + + sws.updateTaskExecutionState(createFailingStateTransition()); + } + } + + @Test + public void testRestartingOnUpdateTaskExecutionStateWithRestart() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + + TestingStopWithSavepoint sws = + createStopWithSavepoint(ctx, new StateTrackingMockExecutionGraph()); + ctx.setHowToHandleFailure( + (ignore) -> Executing.FailureResult.canRestart(Duration.ZERO)); + + ctx.setExpectRestarting(assertNonNull()); + + sws.updateTaskExecutionState(createFailingStateTransition()); + } + } + + @Test + public void testExceptionalFutureCompletionOnLeaveWhileWaitingOnSavepointCompletion() + throws Exception { + MockStopWithSavepointContext ctx = new MockStopWithSavepointContext(); + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + + sws.onLeave(Canceling.class); + + ctx.close(); + assertThat(sws.getOperationCompletionFuture().isCompletedExceptionally(), is(true)); + } + + @Test + public void testExceptionalFutureCompletionAndStateTransitionOnLeaveAfterSavepointCompletion() + throws Exception { + MockStopWithSavepointContext ctx = new MockStopWithSavepointContext(); + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + ctx.setGlobalFailureHandler(sws); + ctx.setHowToHandleFailure((ignore) -> Executing.FailureResult.canRestart(Duration.ZERO)); + ctx.setExpectRestarting(assertNonNull()); + sws.getSavepointFuture().complete(createCompletedSavepoint()); + + sws.onLeave(Canceling.class); + + ctx.close(); + assertThat(sws.getOperationCompletionFuture().isCompletedExceptionally(), is(true)); + } + + @Test + public void testExceptionalSavepointCompletion() throws Exception { + MockStopWithSavepointContext ctx = new MockStopWithSavepointContext(); + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + + sws.getSavepointFuture().completeExceptionally(new RuntimeException("Test error")); + + ctx.close(); + assertThat(sws.getOperationCompletionFuture().isCompletedExceptionally(), is(true)); + assertThat(ctx.hadStateTransition, is(false)); + } + + @Test + public void testRestartOnTaskFailureAfterSavepointCompletion() throws Exception { + MockStopWithSavepointContext ctx = new MockStopWithSavepointContext(); + TerminateExecutionFutureMockedExecutionGraph executionGraph = + new TerminateExecutionFutureMockedExecutionGraph(); + + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx, executionGraph); + ctx.setGlobalFailureHandler(sws); + + ctx.setHowToHandleFailure((ignore) -> Executing.FailureResult.canRestart(Duration.ZERO)); + + ctx.setExpectRestarting(assertNonNull()); + + sws.getSavepointFuture().complete(createCompletedSavepoint()); + executionGraph.getExecutionFuture().complete(ExecutionState.FAILED); + + ctx.close(); + } + + @Test + public void testEnsureCheckpointSchedulerLifecycle() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + assertThat(sws.isCheckpointSchedulerStarted(), is(false)); + + sws.getSavepointFuture().completeExceptionally(new RuntimeException("Test error")); + + ctx.close(); + assertThat(sws.isCheckpointSchedulerStarted(), is(true)); + } + } + + private TestingStopWithSavepoint createStopWithSavepoint(MockStopWithSavepointContext ctx) + throws JobException, JobExecutionException { + return createStopWithSavepoint( + ctx, TestingDefaultExecutionGraphBuilder.newBuilder().build()); + } + + private TestingStopWithSavepoint createStopWithSavepoint( + MockStopWithSavepointContext ctx, ExecutionGraph executionGraph) { + final ExecutionGraphHandler executionGraphHandler = + new ExecutionGraphHandler( + executionGraph, + log, + ctx.getMainThreadExecutor(), + ctx.getMainThreadExecutor()); + OperatorCoordinatorHandler operatorCoordinatorHandler = + new OperatorCoordinatorHandler( + executionGraph, + (throwable) -> { + throw new RuntimeException("Error in test", throwable); + }); + + executionGraph.transitionToRunning(); + + return new TestingStopWithSavepoint( + ctx, + executionGraph, + executionGraphHandler, + operatorCoordinatorHandler, + log, + ClassLoader.getSystemClassLoader(), + "", + true); + } + + private static class TestingStopWithSavepoint extends StopWithSavepoint + implements GlobalFailureHandler { + + private CompletableFuture<CompletedCheckpoint> savepointFuture; + private boolean checkpointSchedulerStarted = false; + + TestingStopWithSavepoint( + Context context, + ExecutionGraph executionGraph, + ExecutionGraphHandler executionGraphHandler, + OperatorCoordinatorHandler operatorCoordinatorHandler, + Logger logger, + ClassLoader userCodeClassLoader, + String targetDirectory, + boolean terminate) { + super( + context, + executionGraph, + executionGraphHandler, + operatorCoordinatorHandler, + logger, + userCodeClassLoader, + targetDirectory, + terminate); + } + + public boolean isCheckpointSchedulerStarted() { + return checkpointSchedulerStarted; + } + + public CompletableFuture<CompletedCheckpoint> getSavepointFuture() { + // since triggerSynchronousSavepoint() gets called in the StopWithSavepoint constructor, + // we initialize the field lazily. + if (savepointFuture == null) { + savepointFuture = new CompletableFuture<>(); + } + return savepointFuture; Review comment: Could this be specified when we create the state? ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointOperationHandlerImpl.java ########## @@ -236,10 +253,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()); + return new FinalState(); + } Review comment: In the context of the `StopWithSavepointOperationHandlerImpl`, this method might be correct. However, in the context of the `StopWithSavepoint` state one should use it in the `onLeave` method because it can trigger a global failover. This can only work if the state is directly responsible for handling the global failover w/o going through the `AdaptiveScheduler`. That way the `StopWithSavepoint` state could say to ignore this signal if it is about to change state. Alternatively, a method which only fails the `result` could work. However, this would add a method which is very specific to the `StopWithSavepoint` state. A third option would be to not directly use the `StopWithSavepointOperationHandlerImpl` directly in the `StopWithSavepoint` state if it does not fit well (at least for the case of stopping the operation and not triggering a global failure and for handling the case of a failed savepoint). Only because you have hammer does not make everything a nail. ########## 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() { Review comment: I think this method shouldn't be executed unconditionally. What if we are no longer in the state `StopWithSavepoint`? Then a callback from the `StopWithSavepointOperationImpl` should not have any effect. ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/StopWithSavepointTest.java ########## @@ -0,0 +1,458 @@ +/* + * 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.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.JobException; +import org.apache.flink.runtime.checkpoint.CheckpointProperties; +import org.apache.flink.runtime.checkpoint.CompletedCheckpoint; +import org.apache.flink.runtime.client.JobExecutionException; +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.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder; +import org.apache.flink.runtime.scheduler.ExecutionGraphHandler; +import org.apache.flink.runtime.scheduler.OperatorCoordinatorHandler; +import org.apache.flink.runtime.state.TestingStreamStateHandle; +import org.apache.flink.runtime.state.testutils.TestCompletedCheckpointStorageLocation; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; +import org.slf4j.Logger; + +import javax.annotation.Nullable; + +import java.time.Duration; +import java.util.Collections; +import java.util.HashMap; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ForkJoinPool; +import java.util.function.Consumer; +import java.util.function.Function; + +import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; +import static org.apache.flink.runtime.scheduler.adaptive.ExecutingTest.createFailingStateTransition; +import static org.apache.flink.runtime.scheduler.adaptive.WaitingForResourcesTest.assertNonNull; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.junit.Assert.assertThat; + +/** Tests for the {@link StopWithSavepoint} state. */ +public class StopWithSavepointTest extends TestLogger { + + @Test + public void testCancel() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + ctx.setExpectCancelling(assertNonNull()); + + sws.cancel(); + } + } + + @Test + public void testSuspend() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + ctx.setExpectFinished( + archivedExecutionGraph -> { + assertThat(archivedExecutionGraph.getState(), is(JobStatus.SUSPENDED)); + }); + + sws.suspend(new RuntimeException()); + } + } + + @Test + public void testRestartOnGlobalFailureIfRestartConfigured() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + ctx.setHowToHandleFailure( + (ignore) -> Executing.FailureResult.canRestart(Duration.ZERO)); + + ctx.setExpectRestarting(assertNonNull()); + + sws.handleGlobalFailure(new RuntimeException()); + } + } + + @Test + public void testFailingOnGlobalFailureIfNoRestartConfigured() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + ctx.setHowToHandleFailure(Executing.FailureResult::canNotRestart); + + ctx.setExpectFailing( + failingArguments -> { + assertThat( + failingArguments.getFailureCause(), + containsCause(RuntimeException.class)); + }); + + sws.handleGlobalFailure(new RuntimeException()); + } + } + + @Test + public void testFailingOnUpdateTaskExecutionStateWithNoRestart() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + + TestingStopWithSavepoint sws = + createStopWithSavepoint(ctx, new StateTrackingMockExecutionGraph()); + ctx.setHowToHandleFailure(Executing.FailureResult::canNotRestart); + + ctx.setExpectFailing( + failingArguments -> { + assertThat( + failingArguments.getFailureCause(), + containsCause(RuntimeException.class)); + }); + + sws.updateTaskExecutionState(createFailingStateTransition()); + } + } + + @Test + public void testRestartingOnUpdateTaskExecutionStateWithRestart() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + + TestingStopWithSavepoint sws = + createStopWithSavepoint(ctx, new StateTrackingMockExecutionGraph()); + ctx.setHowToHandleFailure( + (ignore) -> Executing.FailureResult.canRestart(Duration.ZERO)); + + ctx.setExpectRestarting(assertNonNull()); + + sws.updateTaskExecutionState(createFailingStateTransition()); + } + } + + @Test + public void testExceptionalFutureCompletionOnLeaveWhileWaitingOnSavepointCompletion() + throws Exception { + MockStopWithSavepointContext ctx = new MockStopWithSavepointContext(); + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + + sws.onLeave(Canceling.class); + + ctx.close(); + assertThat(sws.getOperationCompletionFuture().isCompletedExceptionally(), is(true)); + } + + @Test + public void testExceptionalFutureCompletionAndStateTransitionOnLeaveAfterSavepointCompletion() + throws Exception { + MockStopWithSavepointContext ctx = new MockStopWithSavepointContext(); + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + ctx.setGlobalFailureHandler(sws); + ctx.setHowToHandleFailure((ignore) -> Executing.FailureResult.canRestart(Duration.ZERO)); + ctx.setExpectRestarting(assertNonNull()); + sws.getSavepointFuture().complete(createCompletedSavepoint()); + + sws.onLeave(Canceling.class); + + ctx.close(); + assertThat(sws.getOperationCompletionFuture().isCompletedExceptionally(), is(true)); + } Review comment: Why is this a valid case? If we call `onLeave` then we should already have a new target state. ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/StopWithSavepointTest.java ########## @@ -0,0 +1,458 @@ +/* + * 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.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.JobException; +import org.apache.flink.runtime.checkpoint.CheckpointProperties; +import org.apache.flink.runtime.checkpoint.CompletedCheckpoint; +import org.apache.flink.runtime.client.JobExecutionException; +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.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder; +import org.apache.flink.runtime.scheduler.ExecutionGraphHandler; +import org.apache.flink.runtime.scheduler.OperatorCoordinatorHandler; +import org.apache.flink.runtime.state.TestingStreamStateHandle; +import org.apache.flink.runtime.state.testutils.TestCompletedCheckpointStorageLocation; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; +import org.slf4j.Logger; + +import javax.annotation.Nullable; + +import java.time.Duration; +import java.util.Collections; +import java.util.HashMap; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ForkJoinPool; +import java.util.function.Consumer; +import java.util.function.Function; + +import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; +import static org.apache.flink.runtime.scheduler.adaptive.ExecutingTest.createFailingStateTransition; +import static org.apache.flink.runtime.scheduler.adaptive.WaitingForResourcesTest.assertNonNull; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.junit.Assert.assertThat; + +/** Tests for the {@link StopWithSavepoint} state. */ +public class StopWithSavepointTest extends TestLogger { + + @Test + public void testCancel() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + ctx.setExpectCancelling(assertNonNull()); + + sws.cancel(); + } + } + + @Test + public void testSuspend() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + ctx.setExpectFinished( + archivedExecutionGraph -> { + assertThat(archivedExecutionGraph.getState(), is(JobStatus.SUSPENDED)); + }); + + sws.suspend(new RuntimeException()); + } + } + + @Test + public void testRestartOnGlobalFailureIfRestartConfigured() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + ctx.setHowToHandleFailure( + (ignore) -> Executing.FailureResult.canRestart(Duration.ZERO)); + + ctx.setExpectRestarting(assertNonNull()); + + sws.handleGlobalFailure(new RuntimeException()); + } + } + + @Test + public void testFailingOnGlobalFailureIfNoRestartConfigured() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + ctx.setHowToHandleFailure(Executing.FailureResult::canNotRestart); + + ctx.setExpectFailing( + failingArguments -> { + assertThat( + failingArguments.getFailureCause(), + containsCause(RuntimeException.class)); + }); + + sws.handleGlobalFailure(new RuntimeException()); + } + } + + @Test + public void testFailingOnUpdateTaskExecutionStateWithNoRestart() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + + TestingStopWithSavepoint sws = + createStopWithSavepoint(ctx, new StateTrackingMockExecutionGraph()); + ctx.setHowToHandleFailure(Executing.FailureResult::canNotRestart); + + ctx.setExpectFailing( + failingArguments -> { + assertThat( + failingArguments.getFailureCause(), + containsCause(RuntimeException.class)); + }); + + sws.updateTaskExecutionState(createFailingStateTransition()); + } + } + + @Test + public void testRestartingOnUpdateTaskExecutionStateWithRestart() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + + TestingStopWithSavepoint sws = + createStopWithSavepoint(ctx, new StateTrackingMockExecutionGraph()); + ctx.setHowToHandleFailure( + (ignore) -> Executing.FailureResult.canRestart(Duration.ZERO)); + + ctx.setExpectRestarting(assertNonNull()); + + sws.updateTaskExecutionState(createFailingStateTransition()); + } + } + + @Test + public void testExceptionalFutureCompletionOnLeaveWhileWaitingOnSavepointCompletion() + throws Exception { + MockStopWithSavepointContext ctx = new MockStopWithSavepointContext(); + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + + sws.onLeave(Canceling.class); + + ctx.close(); + assertThat(sws.getOperationCompletionFuture().isCompletedExceptionally(), is(true)); + } + + @Test + public void testExceptionalFutureCompletionAndStateTransitionOnLeaveAfterSavepointCompletion() + throws Exception { + MockStopWithSavepointContext ctx = new MockStopWithSavepointContext(); + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + ctx.setGlobalFailureHandler(sws); + ctx.setHowToHandleFailure((ignore) -> Executing.FailureResult.canRestart(Duration.ZERO)); + ctx.setExpectRestarting(assertNonNull()); + sws.getSavepointFuture().complete(createCompletedSavepoint()); + + sws.onLeave(Canceling.class); + + ctx.close(); + assertThat(sws.getOperationCompletionFuture().isCompletedExceptionally(), is(true)); + } + + @Test + public void testExceptionalSavepointCompletion() throws Exception { + MockStopWithSavepointContext ctx = new MockStopWithSavepointContext(); + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + + sws.getSavepointFuture().completeExceptionally(new RuntimeException("Test error")); + + ctx.close(); + assertThat(sws.getOperationCompletionFuture().isCompletedExceptionally(), is(true)); + assertThat(ctx.hadStateTransition, is(false)); Review comment: I think this is a problem. If the savepoint fails, then we should stay in the `StopWithSavepoint` state. Instead we should return to `Executing`. ########## 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); + } + + interface Context extends StateWithExecutionGraph.Context, GlobalFailureHandler { Review comment: Why do we go through the `AdaptiveScheduler` to handle global failures from the `StopWithSavepointOperationHandlerImpl`? ########## 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: What happens if this future fails? ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/StopWithSavepointTest.java ########## @@ -0,0 +1,458 @@ +/* + * 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.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.JobException; +import org.apache.flink.runtime.checkpoint.CheckpointProperties; +import org.apache.flink.runtime.checkpoint.CompletedCheckpoint; +import org.apache.flink.runtime.client.JobExecutionException; +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.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder; +import org.apache.flink.runtime.scheduler.ExecutionGraphHandler; +import org.apache.flink.runtime.scheduler.OperatorCoordinatorHandler; +import org.apache.flink.runtime.state.TestingStreamStateHandle; +import org.apache.flink.runtime.state.testutils.TestCompletedCheckpointStorageLocation; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; +import org.slf4j.Logger; + +import javax.annotation.Nullable; + +import java.time.Duration; +import java.util.Collections; +import java.util.HashMap; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ForkJoinPool; +import java.util.function.Consumer; +import java.util.function.Function; + +import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; +import static org.apache.flink.runtime.scheduler.adaptive.ExecutingTest.createFailingStateTransition; +import static org.apache.flink.runtime.scheduler.adaptive.WaitingForResourcesTest.assertNonNull; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.junit.Assert.assertThat; + +/** Tests for the {@link StopWithSavepoint} state. */ +public class StopWithSavepointTest extends TestLogger { + + @Test + public void testCancel() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + ctx.setExpectCancelling(assertNonNull()); + + sws.cancel(); + } + } + + @Test + public void testSuspend() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + ctx.setExpectFinished( + archivedExecutionGraph -> { + assertThat(archivedExecutionGraph.getState(), is(JobStatus.SUSPENDED)); + }); + + sws.suspend(new RuntimeException()); + } + } + + @Test + public void testRestartOnGlobalFailureIfRestartConfigured() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + ctx.setHowToHandleFailure( + (ignore) -> Executing.FailureResult.canRestart(Duration.ZERO)); + + ctx.setExpectRestarting(assertNonNull()); + + sws.handleGlobalFailure(new RuntimeException()); + } + } + + @Test + public void testFailingOnGlobalFailureIfNoRestartConfigured() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + ctx.setHowToHandleFailure(Executing.FailureResult::canNotRestart); + + ctx.setExpectFailing( + failingArguments -> { + assertThat( + failingArguments.getFailureCause(), + containsCause(RuntimeException.class)); + }); + + sws.handleGlobalFailure(new RuntimeException()); + } + } + + @Test + public void testFailingOnUpdateTaskExecutionStateWithNoRestart() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + + TestingStopWithSavepoint sws = + createStopWithSavepoint(ctx, new StateTrackingMockExecutionGraph()); + ctx.setHowToHandleFailure(Executing.FailureResult::canNotRestart); + + ctx.setExpectFailing( + failingArguments -> { + assertThat( + failingArguments.getFailureCause(), + containsCause(RuntimeException.class)); + }); + + sws.updateTaskExecutionState(createFailingStateTransition()); + } + } + + @Test + public void testRestartingOnUpdateTaskExecutionStateWithRestart() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + + TestingStopWithSavepoint sws = + createStopWithSavepoint(ctx, new StateTrackingMockExecutionGraph()); + ctx.setHowToHandleFailure( + (ignore) -> Executing.FailureResult.canRestart(Duration.ZERO)); + + ctx.setExpectRestarting(assertNonNull()); + + sws.updateTaskExecutionState(createFailingStateTransition()); + } + } + + @Test + public void testExceptionalFutureCompletionOnLeaveWhileWaitingOnSavepointCompletion() + throws Exception { + MockStopWithSavepointContext ctx = new MockStopWithSavepointContext(); + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + + sws.onLeave(Canceling.class); + + ctx.close(); + assertThat(sws.getOperationCompletionFuture().isCompletedExceptionally(), is(true)); + } + + @Test + public void testExceptionalFutureCompletionAndStateTransitionOnLeaveAfterSavepointCompletion() + throws Exception { + MockStopWithSavepointContext ctx = new MockStopWithSavepointContext(); + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + ctx.setGlobalFailureHandler(sws); + ctx.setHowToHandleFailure((ignore) -> Executing.FailureResult.canRestart(Duration.ZERO)); + ctx.setExpectRestarting(assertNonNull()); + sws.getSavepointFuture().complete(createCompletedSavepoint()); + + sws.onLeave(Canceling.class); + + ctx.close(); + assertThat(sws.getOperationCompletionFuture().isCompletedExceptionally(), is(true)); + } + + @Test + public void testExceptionalSavepointCompletion() throws Exception { + MockStopWithSavepointContext ctx = new MockStopWithSavepointContext(); + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + + sws.getSavepointFuture().completeExceptionally(new RuntimeException("Test error")); + + ctx.close(); + assertThat(sws.getOperationCompletionFuture().isCompletedExceptionally(), is(true)); + assertThat(ctx.hadStateTransition, is(false)); + } + + @Test + public void testRestartOnTaskFailureAfterSavepointCompletion() throws Exception { + MockStopWithSavepointContext ctx = new MockStopWithSavepointContext(); + TerminateExecutionFutureMockedExecutionGraph executionGraph = + new TerminateExecutionFutureMockedExecutionGraph(); + + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx, executionGraph); + ctx.setGlobalFailureHandler(sws); + + ctx.setHowToHandleFailure((ignore) -> Executing.FailureResult.canRestart(Duration.ZERO)); + + ctx.setExpectRestarting(assertNonNull()); + + sws.getSavepointFuture().complete(createCompletedSavepoint()); + executionGraph.getExecutionFuture().complete(ExecutionState.FAILED); + + ctx.close(); + } + + @Test + public void testEnsureCheckpointSchedulerLifecycle() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + assertThat(sws.isCheckpointSchedulerStarted(), is(false)); + + sws.getSavepointFuture().completeExceptionally(new RuntimeException("Test error")); + + ctx.close(); + assertThat(sws.isCheckpointSchedulerStarted(), is(true)); + } Review comment: I think the contract would be better defined if we say we stop the `CheckpointCoordinator` if we enter `StopWithSavepoint` and if we go back into `Executing`, then we start the `CheckpointCoordinator` again. ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/StopWithSavepointTest.java ########## @@ -0,0 +1,458 @@ +/* + * 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.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.JobException; +import org.apache.flink.runtime.checkpoint.CheckpointProperties; +import org.apache.flink.runtime.checkpoint.CompletedCheckpoint; +import org.apache.flink.runtime.client.JobExecutionException; +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.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder; +import org.apache.flink.runtime.scheduler.ExecutionGraphHandler; +import org.apache.flink.runtime.scheduler.OperatorCoordinatorHandler; +import org.apache.flink.runtime.state.TestingStreamStateHandle; +import org.apache.flink.runtime.state.testutils.TestCompletedCheckpointStorageLocation; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; +import org.slf4j.Logger; + +import javax.annotation.Nullable; + +import java.time.Duration; +import java.util.Collections; +import java.util.HashMap; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ForkJoinPool; +import java.util.function.Consumer; +import java.util.function.Function; + +import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; +import static org.apache.flink.runtime.scheduler.adaptive.ExecutingTest.createFailingStateTransition; +import static org.apache.flink.runtime.scheduler.adaptive.WaitingForResourcesTest.assertNonNull; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.junit.Assert.assertThat; + +/** Tests for the {@link StopWithSavepoint} state. */ +public class StopWithSavepointTest extends TestLogger { + + @Test + public void testCancel() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + ctx.setExpectCancelling(assertNonNull()); + + sws.cancel(); + } + } + + @Test + public void testSuspend() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + ctx.setExpectFinished( + archivedExecutionGraph -> { + assertThat(archivedExecutionGraph.getState(), is(JobStatus.SUSPENDED)); + }); + + sws.suspend(new RuntimeException()); + } + } + + @Test + public void testRestartOnGlobalFailureIfRestartConfigured() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + ctx.setHowToHandleFailure( + (ignore) -> Executing.FailureResult.canRestart(Duration.ZERO)); + + ctx.setExpectRestarting(assertNonNull()); + + sws.handleGlobalFailure(new RuntimeException()); + } + } + + @Test + public void testFailingOnGlobalFailureIfNoRestartConfigured() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + ctx.setHowToHandleFailure(Executing.FailureResult::canNotRestart); + + ctx.setExpectFailing( + failingArguments -> { + assertThat( + failingArguments.getFailureCause(), + containsCause(RuntimeException.class)); + }); + + sws.handleGlobalFailure(new RuntimeException()); + } + } + + @Test + public void testFailingOnUpdateTaskExecutionStateWithNoRestart() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + + TestingStopWithSavepoint sws = + createStopWithSavepoint(ctx, new StateTrackingMockExecutionGraph()); + ctx.setHowToHandleFailure(Executing.FailureResult::canNotRestart); + + ctx.setExpectFailing( + failingArguments -> { + assertThat( + failingArguments.getFailureCause(), + containsCause(RuntimeException.class)); + }); + + sws.updateTaskExecutionState(createFailingStateTransition()); + } + } + + @Test + public void testRestartingOnUpdateTaskExecutionStateWithRestart() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + + TestingStopWithSavepoint sws = + createStopWithSavepoint(ctx, new StateTrackingMockExecutionGraph()); + ctx.setHowToHandleFailure( + (ignore) -> Executing.FailureResult.canRestart(Duration.ZERO)); + + ctx.setExpectRestarting(assertNonNull()); + + sws.updateTaskExecutionState(createFailingStateTransition()); + } + } + + @Test + public void testExceptionalFutureCompletionOnLeaveWhileWaitingOnSavepointCompletion() + throws Exception { + MockStopWithSavepointContext ctx = new MockStopWithSavepointContext(); + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + + sws.onLeave(Canceling.class); + + ctx.close(); + assertThat(sws.getOperationCompletionFuture().isCompletedExceptionally(), is(true)); + } + + @Test + public void testExceptionalFutureCompletionAndStateTransitionOnLeaveAfterSavepointCompletion() + throws Exception { + MockStopWithSavepointContext ctx = new MockStopWithSavepointContext(); + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + ctx.setGlobalFailureHandler(sws); + ctx.setHowToHandleFailure((ignore) -> Executing.FailureResult.canRestart(Duration.ZERO)); + ctx.setExpectRestarting(assertNonNull()); + sws.getSavepointFuture().complete(createCompletedSavepoint()); + + sws.onLeave(Canceling.class); + + ctx.close(); + assertThat(sws.getOperationCompletionFuture().isCompletedExceptionally(), is(true)); + } + + @Test + public void testExceptionalSavepointCompletion() throws Exception { + MockStopWithSavepointContext ctx = new MockStopWithSavepointContext(); + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + + sws.getSavepointFuture().completeExceptionally(new RuntimeException("Test error")); + + ctx.close(); + assertThat(sws.getOperationCompletionFuture().isCompletedExceptionally(), is(true)); + assertThat(ctx.hadStateTransition, is(false)); + } + + @Test + public void testRestartOnTaskFailureAfterSavepointCompletion() throws Exception { + MockStopWithSavepointContext ctx = new MockStopWithSavepointContext(); + TerminateExecutionFutureMockedExecutionGraph executionGraph = + new TerminateExecutionFutureMockedExecutionGraph(); + + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx, executionGraph); + ctx.setGlobalFailureHandler(sws); + + ctx.setHowToHandleFailure((ignore) -> Executing.FailureResult.canRestart(Duration.ZERO)); + + ctx.setExpectRestarting(assertNonNull()); + + sws.getSavepointFuture().complete(createCompletedSavepoint()); + executionGraph.getExecutionFuture().complete(ExecutionState.FAILED); + + ctx.close(); + } + + @Test + public void testEnsureCheckpointSchedulerLifecycle() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + assertThat(sws.isCheckpointSchedulerStarted(), is(false)); + + sws.getSavepointFuture().completeExceptionally(new RuntimeException("Test error")); + + ctx.close(); + assertThat(sws.isCheckpointSchedulerStarted(), is(true)); + } + } + + private TestingStopWithSavepoint createStopWithSavepoint(MockStopWithSavepointContext ctx) + throws JobException, JobExecutionException { + return createStopWithSavepoint( + ctx, TestingDefaultExecutionGraphBuilder.newBuilder().build()); + } + + private TestingStopWithSavepoint createStopWithSavepoint( + MockStopWithSavepointContext ctx, ExecutionGraph executionGraph) { + final ExecutionGraphHandler executionGraphHandler = + new ExecutionGraphHandler( + executionGraph, + log, + ctx.getMainThreadExecutor(), + ctx.getMainThreadExecutor()); + OperatorCoordinatorHandler operatorCoordinatorHandler = + new OperatorCoordinatorHandler( + executionGraph, + (throwable) -> { + throw new RuntimeException("Error in test", throwable); + }); + + executionGraph.transitionToRunning(); + + return new TestingStopWithSavepoint( + ctx, + executionGraph, + executionGraphHandler, + operatorCoordinatorHandler, + log, + ClassLoader.getSystemClassLoader(), + "", + true); + } + + private static class TestingStopWithSavepoint extends StopWithSavepoint + implements GlobalFailureHandler { + + private CompletableFuture<CompletedCheckpoint> savepointFuture; + private boolean checkpointSchedulerStarted = false; + + TestingStopWithSavepoint( + Context context, + ExecutionGraph executionGraph, + ExecutionGraphHandler executionGraphHandler, + OperatorCoordinatorHandler operatorCoordinatorHandler, + Logger logger, + ClassLoader userCodeClassLoader, + String targetDirectory, + boolean terminate) { + super( + context, + executionGraph, + executionGraphHandler, + operatorCoordinatorHandler, + logger, + userCodeClassLoader, + targetDirectory, + terminate); + } + + public boolean isCheckpointSchedulerStarted() { + return checkpointSchedulerStarted; + } + + public CompletableFuture<CompletedCheckpoint> getSavepointFuture() { Review comment: Can this be package private? ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/StopWithSavepointTest.java ########## @@ -0,0 +1,458 @@ +/* + * 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.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.JobException; +import org.apache.flink.runtime.checkpoint.CheckpointProperties; +import org.apache.flink.runtime.checkpoint.CompletedCheckpoint; +import org.apache.flink.runtime.client.JobExecutionException; +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.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder; +import org.apache.flink.runtime.scheduler.ExecutionGraphHandler; +import org.apache.flink.runtime.scheduler.OperatorCoordinatorHandler; +import org.apache.flink.runtime.state.TestingStreamStateHandle; +import org.apache.flink.runtime.state.testutils.TestCompletedCheckpointStorageLocation; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; +import org.slf4j.Logger; + +import javax.annotation.Nullable; + +import java.time.Duration; +import java.util.Collections; +import java.util.HashMap; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ForkJoinPool; +import java.util.function.Consumer; +import java.util.function.Function; + +import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; +import static org.apache.flink.runtime.scheduler.adaptive.ExecutingTest.createFailingStateTransition; +import static org.apache.flink.runtime.scheduler.adaptive.WaitingForResourcesTest.assertNonNull; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.junit.Assert.assertThat; + +/** Tests for the {@link StopWithSavepoint} state. */ +public class StopWithSavepointTest extends TestLogger { + + @Test + public void testCancel() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + ctx.setExpectCancelling(assertNonNull()); + + sws.cancel(); + } + } + + @Test + public void testSuspend() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + ctx.setExpectFinished( + archivedExecutionGraph -> { + assertThat(archivedExecutionGraph.getState(), is(JobStatus.SUSPENDED)); + }); + + sws.suspend(new RuntimeException()); + } + } + + @Test + public void testRestartOnGlobalFailureIfRestartConfigured() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + ctx.setHowToHandleFailure( + (ignore) -> Executing.FailureResult.canRestart(Duration.ZERO)); + + ctx.setExpectRestarting(assertNonNull()); + + sws.handleGlobalFailure(new RuntimeException()); + } + } + + @Test + public void testFailingOnGlobalFailureIfNoRestartConfigured() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + ctx.setHowToHandleFailure(Executing.FailureResult::canNotRestart); + + ctx.setExpectFailing( + failingArguments -> { + assertThat( + failingArguments.getFailureCause(), + containsCause(RuntimeException.class)); + }); + + sws.handleGlobalFailure(new RuntimeException()); + } + } + + @Test + public void testFailingOnUpdateTaskExecutionStateWithNoRestart() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + + TestingStopWithSavepoint sws = + createStopWithSavepoint(ctx, new StateTrackingMockExecutionGraph()); + ctx.setHowToHandleFailure(Executing.FailureResult::canNotRestart); + + ctx.setExpectFailing( + failingArguments -> { + assertThat( + failingArguments.getFailureCause(), + containsCause(RuntimeException.class)); + }); + + sws.updateTaskExecutionState(createFailingStateTransition()); + } + } + + @Test + public void testRestartingOnUpdateTaskExecutionStateWithRestart() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + + TestingStopWithSavepoint sws = + createStopWithSavepoint(ctx, new StateTrackingMockExecutionGraph()); + ctx.setHowToHandleFailure( + (ignore) -> Executing.FailureResult.canRestart(Duration.ZERO)); + + ctx.setExpectRestarting(assertNonNull()); + + sws.updateTaskExecutionState(createFailingStateTransition()); + } + } + + @Test + public void testExceptionalFutureCompletionOnLeaveWhileWaitingOnSavepointCompletion() + throws Exception { + MockStopWithSavepointContext ctx = new MockStopWithSavepointContext(); + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + + sws.onLeave(Canceling.class); + + ctx.close(); + assertThat(sws.getOperationCompletionFuture().isCompletedExceptionally(), is(true)); + } + + @Test + public void testExceptionalFutureCompletionAndStateTransitionOnLeaveAfterSavepointCompletion() + throws Exception { + MockStopWithSavepointContext ctx = new MockStopWithSavepointContext(); + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + ctx.setGlobalFailureHandler(sws); + ctx.setHowToHandleFailure((ignore) -> Executing.FailureResult.canRestart(Duration.ZERO)); + ctx.setExpectRestarting(assertNonNull()); + sws.getSavepointFuture().complete(createCompletedSavepoint()); + + sws.onLeave(Canceling.class); + + ctx.close(); + assertThat(sws.getOperationCompletionFuture().isCompletedExceptionally(), is(true)); + } + + @Test + public void testExceptionalSavepointCompletion() throws Exception { + MockStopWithSavepointContext ctx = new MockStopWithSavepointContext(); + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + + sws.getSavepointFuture().completeExceptionally(new RuntimeException("Test error")); + + ctx.close(); + assertThat(sws.getOperationCompletionFuture().isCompletedExceptionally(), is(true)); + assertThat(ctx.hadStateTransition, is(false)); + } + + @Test + public void testRestartOnTaskFailureAfterSavepointCompletion() throws Exception { + MockStopWithSavepointContext ctx = new MockStopWithSavepointContext(); + TerminateExecutionFutureMockedExecutionGraph executionGraph = + new TerminateExecutionFutureMockedExecutionGraph(); + + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx, executionGraph); + ctx.setGlobalFailureHandler(sws); + + ctx.setHowToHandleFailure((ignore) -> Executing.FailureResult.canRestart(Duration.ZERO)); + + ctx.setExpectRestarting(assertNonNull()); + + sws.getSavepointFuture().complete(createCompletedSavepoint()); + executionGraph.getExecutionFuture().complete(ExecutionState.FAILED); + + ctx.close(); + } + + @Test + public void testEnsureCheckpointSchedulerLifecycle() throws Exception { + try (MockStopWithSavepointContext ctx = new MockStopWithSavepointContext()) { + TestingStopWithSavepoint sws = createStopWithSavepoint(ctx); + assertThat(sws.isCheckpointSchedulerStarted(), is(false)); + + sws.getSavepointFuture().completeExceptionally(new RuntimeException("Test error")); + + ctx.close(); + assertThat(sws.isCheckpointSchedulerStarted(), is(true)); + } + } + Review comment: I think we need some tests which ensure that the `StopWithSavepointOperationsHandlerImpl` won't trigger any methods once we have left the `StopWithSavepoint` state. ---------------------------------------------------------------- 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