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



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointTerminationHandler.java
##########
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.scheduler.stopwithsavepoint;
+
+import org.apache.flink.runtime.checkpoint.CompletedCheckpoint;
+import org.apache.flink.runtime.execution.ExecutionState;
+
+import javax.annotation.Nullable;
+
+import java.util.Collection;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * {@code StopWithSavepointTerminationHandler} handles the termination steps 
necessary for the
+ * stop-with-savepoint operation to finish. It includes:
+ *
+ * <ol>
+ *   <li>Creating a savepoint needs to be completed
+ *   <li>Waiting for the executions of the underlying job to finish
+ * </ol>
+ */
+public interface StopWithSavepointTerminationHandler {
+
+    /**
+     * Returns the a {@code CompletableFuture} referring to the result of the 
stop-with-savepoint
+     * operation.
+     *
+     * @return the {@code CompletableFuture} containing the path to the 
created savepoint in case of
+     *     success.
+     */
+    CompletableFuture<String> getSavepointPath();
+
+    /**
+     * Handles the result of a {@code CompletableFuture} holding a {@link 
CompletedCheckpoint}. Only
+     * one of the two parameters are allowed to be set.
+     *
+     * @param completedSavepoint the {@code CompletedCheckpoint} referring to 
the created savepoint
+     * @param throwable an error that was caught during savepoint creation
+     * @throws IllegalArgumentException if {@code throwable} and {@code 
completedSavepoint} are set
+     * @throws NullPointerException if none of the parameters are set

Review comment:
       ```suggestion
        * @throws NullPointerException if none of the parameters is set
   ```

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointTerminationHandlerImplTest.java
##########
@@ -0,0 +1,228 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.scheduler.stopwithsavepoint;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.core.testutils.FlinkMatchers;
+import org.apache.flink.runtime.checkpoint.CheckpointProperties;
+import org.apache.flink.runtime.checkpoint.CompletedCheckpoint;
+import org.apache.flink.runtime.checkpoint.TestingCheckpointScheduling;
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.scheduler.SchedulerBase;
+import org.apache.flink.runtime.scheduler.SchedulerNG;
+import org.apache.flink.runtime.scheduler.TestingSchedulerNG;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.testutils.EmptyStreamStateHandle;
+import 
org.apache.flink.runtime.state.testutils.TestCompletedCheckpointStorageLocation;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.function.Consumer;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * {@code StopWithSavepointTerminationHandlerImplTest} tests the 
stop-with-savepoint functionality
+ * of {@link SchedulerBase#stopWithSavepoint(String, boolean)}.

Review comment:
       Don't we test the `StopWithSavepointTerminationHandlerImpl` here?

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointTerminationHandlerImplTest.java
##########
@@ -0,0 +1,228 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.scheduler.stopwithsavepoint;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.core.testutils.FlinkMatchers;
+import org.apache.flink.runtime.checkpoint.CheckpointProperties;
+import org.apache.flink.runtime.checkpoint.CompletedCheckpoint;
+import org.apache.flink.runtime.checkpoint.TestingCheckpointScheduling;
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.scheduler.SchedulerBase;
+import org.apache.flink.runtime.scheduler.SchedulerNG;
+import org.apache.flink.runtime.scheduler.TestingSchedulerNG;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.testutils.EmptyStreamStateHandle;
+import 
org.apache.flink.runtime.state.testutils.TestCompletedCheckpointStorageLocation;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.function.Consumer;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * {@code StopWithSavepointTerminationHandlerImplTest} tests the 
stop-with-savepoint functionality
+ * of {@link SchedulerBase#stopWithSavepoint(String, boolean)}.
+ */
+public class StopWithSavepointTerminationHandlerImplTest extends TestLogger {
+
+    private static final JobID JOB_ID = new JobID();
+
+    private final TestingCheckpointScheduling checkpointScheduling =
+            new TestingCheckpointScheduling(false);
+
+    private StopWithSavepointTerminationHandlerImpl 
createTestInstanceFailingOnGlobalFailOver() {
+        return createTestInstance(
+                throwableCausingGlobalFailOver -> fail("No global failover 
should be triggered."));
+    }
+
+    private StopWithSavepointTerminationHandlerImpl createTestInstance(
+            Consumer<Throwable> handleGlobalFailureConsumer) {
+        // checkpointing should be always stopped before initiating 
stop-with-savepoint
+        checkpointScheduling.stopCheckpointScheduler();
+
+        final SchedulerNG scheduler =
+                TestingSchedulerNG.newBuilder()
+                        
.setHandleGlobalFailureConsumer(handleGlobalFailureConsumer)
+                        .build();
+        return new StopWithSavepointTerminationHandlerImpl(
+                JOB_ID, scheduler, checkpointScheduling, 
Executors.directExecutor(), log);
+    }
+
+    @Test
+    public void testHappyPath() throws ExecutionException, 
InterruptedException {
+        final StopWithSavepointTerminationHandlerImpl testInstance =
+                createTestInstanceFailingOnGlobalFailOver();
+
+        final EmptyStreamStateHandle streamStateHandle = new 
EmptyStreamStateHandle();
+        final CompletedCheckpoint completedSavepoint = 
createCompletedSavepoint(streamStateHandle);
+        testInstance.handleSavepointCreation(completedSavepoint, null);
+        
testInstance.handleExecutionsTermination(Collections.singleton(ExecutionState.FINISHED));
+
+        assertThat(
+                testInstance.getSavepointPath().get(), 
is(completedSavepoint.getExternalPointer()));
+
+        assertFalse(
+                "The savepoint should not have been discarded.", 
streamStateHandle.isDisposed());
+        assertFalse("Checkpoint scheduling should be disabled.", 
checkpointScheduling.isEnabled());
+    }
+
+    @Test
+    public void testSavepointCreationFailureWithoutExecutionTermination() {
+        // savepoint creation failure is handled as expected if no execution 
termination happens
+        assertSavepointCreationFailure(testInstance -> {});
+    }
+
+    @Test
+    public void testSavepointCreationFailureWithFailingExecutions() {
+        // no global fail-over is expected to be triggered by the 
stop-with-savepoint despite the
+        // execution failure
+        assertSavepointCreationFailure(
+                testInstance ->
+                        testInstance.handleExecutionsTermination(
+                                
Collections.singletonList(ExecutionState.FAILED)));
+    }
+
+    @Test
+    public void testSavepointCreationFailureWithFinishingExecutions() {
+        // checkpoint scheduling should be still enabled despite the finished 
executions
+        assertSavepointCreationFailure(
+                testInstance ->
+                        testInstance.handleExecutionsTermination(
+                                
Collections.singletonList(ExecutionState.FINISHED)));
+    }
+
+    public void assertSavepointCreationFailure(
+            Consumer<StopWithSavepointTerminationHandler> 
handleExecutionsTermination) {
+        final StopWithSavepointTerminationHandlerImpl testInstance =
+                createTestInstanceFailingOnGlobalFailOver();
+
+        final String expectedErrorMessage = "Expected exception during 
savepoint creation.";
+        testInstance.handleSavepointCreation(null, new 
Exception(expectedErrorMessage));
+        handleExecutionsTermination.accept(testInstance);
+
+        try {
+            testInstance.getSavepointPath().get();
+            fail("An ExecutionException is expected.");
+        } catch (Throwable e) {
+            final Optional<Throwable> actualException =
+                    ExceptionUtils.findThrowableWithMessage(e, 
expectedErrorMessage);
+            assertTrue(
+                    "An exception with the expected error message should have 
been thrown.",
+                    actualException.isPresent());
+        }
+
+        // the checkpoint scheduling should be enabled in case of failure
+        assertTrue("Checkpoint scheduling should be enabled.", 
checkpointScheduling.isEnabled());
+    }
+
+    @Test
+    public void testNoTerminationHandling() throws ExecutionException, 
InterruptedException {
+        final CompletableFuture<Throwable> globalFailOverTriggered = new 
CompletableFuture<>();
+        final StopWithSavepointTerminationHandlerImpl testInstance =
+                createTestInstance(globalFailOverTriggered::complete);
+
+        final ExecutionState expectedNonFinishedState = ExecutionState.FAILED;
+        final String expectedErrorMessage =
+                String.format(
+                        "Inconsistent execution state after stopping with 
savepoint. At least one execution is still in one of the following states: %s. 
A global fail-over is triggered to recover the job %s.",
+                        expectedNonFinishedState, JOB_ID);
+
+        final EmptyStreamStateHandle streamStateHandle = new 
EmptyStreamStateHandle();
+        final CompletedCheckpoint completedSavepoint = 
createCompletedSavepoint(streamStateHandle);
+
+        testInstance.handleSavepointCreation(completedSavepoint, null);
+        testInstance.handleExecutionsTermination(
+                Collections.singletonList(expectedNonFinishedState));
+
+        try {
+            testInstance.getSavepointPath().get();
+            fail("An ExecutionException is expected.");
+        } catch (Throwable e) {
+            final Optional<FlinkException> actualFlinkException =
+                    ExceptionUtils.findThrowable(e, FlinkException.class);
+            assertTrue(
+                    "A FlinkException should have been thrown.", 
actualFlinkException.isPresent());
+            assertThat(
+                    actualFlinkException.get(),
+                    FlinkMatchers.containsMessage(expectedErrorMessage));
+        }
+
+        assertTrue("Global fail-over was not triggered.", 
globalFailOverTriggered.isDone());
+        assertThat(
+                globalFailOverTriggered.get(), 
FlinkMatchers.containsMessage(expectedErrorMessage));
+
+        assertTrue("Savepoint should be discarded.", 
streamStateHandle.isDisposed());
+
+        // the checkpoint scheduling should be enabled in case of failure
+        assertTrue("Checkpoint scheduling should be enabled.", 
checkpointScheduling.isEnabled());

Review comment:
       In case of a global failover, the checkpointing should actually be 
stopped for the time until the job has been restarted. Normally the 
de-/activation should happen via `CheckpointCoordinatorDeActivator`. In the 
current code what happens is that by calling the global failover the job will 
go to `RESTARTING` and then we go into 
`StopWithSavepointTerminationHandlerImpl.terminateExceptionally` and restart 
the checkpoint scheduling.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointTerminationHandlerImpl.java
##########
@@ -0,0 +1,300 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.scheduler.stopwithsavepoint;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.checkpoint.CheckpointScheduling;
+import org.apache.flink.runtime.checkpoint.CompletedCheckpoint;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.scheduler.SchedulerNG;
+import org.apache.flink.util.FlinkException;
+
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+
+import java.util.Collection;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@code StopWithSavepointTerminationHandlerImpl} implements {@link
+ * StopWithSavepointTerminationHandler}.
+ *
+ * <p>The operation only succeeds if both steps, the savepoint creation and 
the successful
+ * termination of the job, succeed. If the former step fails, the operation 
fails exceptionally
+ * without any further actions. If the latter one fails, a global fail-over is 
triggered before
+ * failing the operation.
+ */
+public class StopWithSavepointTerminationHandlerImpl
+        implements StopWithSavepointTerminationHandler {
+
+    private final Logger log;
+
+    private final SchedulerNG scheduler;
+    private final CheckpointScheduling checkpointScheduling;
+    private final JobID jobId;
+    private final Executor ioExecutor;
+
+    private final CompletableFuture<String> result = new CompletableFuture<>();
+
+    private State state = new WaitingForSavepoint();
+
+    public <S extends SchedulerNG & CheckpointScheduling> 
StopWithSavepointTerminationHandlerImpl(
+            JobID jobId, S schedulerWithCheckpointing, Executor ioExecutor, 
Logger log) {
+        this(jobId, schedulerWithCheckpointing, schedulerWithCheckpointing, 
ioExecutor, log);
+    }
+
+    @VisibleForTesting
+    StopWithSavepointTerminationHandlerImpl(
+            JobID jobId,
+            SchedulerNG scheduler,
+            CheckpointScheduling checkpointScheduling,
+            Executor ioExecutor,
+            Logger log) {
+        this.jobId = checkNotNull(jobId);
+        this.scheduler = checkNotNull(scheduler);
+        this.checkpointScheduling = checkNotNull(checkpointScheduling);
+        this.ioExecutor = checkNotNull(ioExecutor);
+        this.log = checkNotNull(log);
+    }
+
+    @Override
+    public CompletableFuture<String> getSavepointPath() {
+        return result;
+    }
+
+    @Override
+    public void handleSavepointCreation(
+            CompletedCheckpoint completedSavepoint, Throwable throwable) {
+        if (throwable != null) {
+            checkArgument(
+                    completedSavepoint == null,
+                    "No savepoint should be provided if a throwable is 
passed.");
+            handleSavepointCreationFailure(throwable);
+        } else {
+            handleSavepointCreationSuccess(checkNotNull(completedSavepoint));
+        }
+    }
+
+    @Override
+    public void handleExecutionsTermination(Collection<ExecutionState> 
terminatedExecutionStates) {
+        final Set<ExecutionState> notFinishedExecutionStates =
+                checkNotNull(terminatedExecutionStates).stream()
+                        .filter(state -> state != ExecutionState.FINISHED)
+                        .collect(Collectors.toSet());
+
+        if (notFinishedExecutionStates.isEmpty()) {
+            handleExecutionsFinished();
+        } else {
+            handleAnyExecutionNotFinished(notFinishedExecutionStates);
+        }
+    }
+
+    private void handleSavepointCreationSuccess(CompletedCheckpoint 
completedCheckpoint) {
+        final State oldState = state;
+        state = state.onSavepointCreation(completedCheckpoint);
+
+        log.debug(
+                "Stop-with-savepoint transitioned from {} to {} on savepoint 
creation handling for job {}.",
+                oldState,
+                state,
+                jobId);
+    }
+
+    private void handleSavepointCreationFailure(Throwable throwable) {
+        final State oldState = state;
+        state = state.onSavepointCreationFailure(throwable);
+
+        log.debug(
+                "Stop-with-savepoint transitioned from {} to {} on savepoint 
creation failure handling for job {}.",
+                oldState,
+                state,
+                jobId);
+    }
+
+    private void handleExecutionsFinished() {
+        final State oldState = state;
+        state = state.onExecutionsFinished();
+
+        log.debug(
+                "Stop-with-savepoint transitioned from {} to {} on execution 
termination handling with all executions being finished for job {}.",
+                oldState,
+                state,
+                jobId);
+    }
+
+    private void handleAnyExecutionNotFinished(Set<ExecutionState> 
notFinishedExecutionStates) {
+        final State oldState = state;
+        state = state.onAnyExecutionNotFinished(notFinishedExecutionStates);
+
+        log.warn(
+                "Stop-with-savepoint transitioned from {} to {} on execution 
termination handling for job {} with some executions being in an not-finished 
state: {}",
+                oldState,
+                state,
+                jobId,
+                notFinishedExecutionStates);
+    }
+
+    /**
+     * Handles the termination of the {@code 
StopWithSavepointTerminationHandler} exceptionally
+     * after triggering a global job fail-over.
+     *
+     * @param completedSavepoint the completed savepoint that needs to be 
discarded.
+     * @param unfinishedExecutionStates the unfinished states that caused the 
failure.
+     */
+    private void terminateExceptionallyWithGlobalFailover(
+            CompletedCheckpoint completedSavepoint,
+            Iterable<ExecutionState> unfinishedExecutionStates) {
+        String errorMessage =
+                String.format(
+                        "Inconsistent execution state after stopping with 
savepoint. At least one execution is still in one of the following states: %s. 
A global fail-over is triggered to recover the job %s.",
+                        StringUtils.join(unfinishedExecutionStates, ", "), 
jobId);
+        FlinkException inconsistentFinalStateException = new 
FlinkException(errorMessage);
+
+        scheduler.handleGlobalFailure(inconsistentFinalStateException);
+        discardSavepoint(completedSavepoint);

Review comment:
       I think we must not discard the savepoint because it might be used by 
Flink for recoveries. Sorry for the bad advice.




----------------------------------------------------------------
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]


Reply via email to