rmetzger commented on a change in pull request #14948: URL: https://github.com/apache/flink/pull/14948#discussion_r593090535
########## 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: Maybe this test should not make any assumptions about state transitions. A savepoint failure leads to a transition of the Task from RUNNING to FAILED, hence we get a notification via `updateTaskExecutionState` about this failure, which we handle accordingly. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected]
