rmetzger commented on a change in pull request #14879:
URL: https://github.com/apache/flink/pull/14879#discussion_r572086187



##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/declarative/ExecutingTest.java
##########
@@ -0,0 +1,451 @@
+/*
+ * 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.declarative;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.JobStatus;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.JobException;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.blob.VoidBlobWriter;
+import org.apache.flink.runtime.client.JobExecutionException;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionVertexDeploymentTest;
+import org.apache.flink.runtime.executiongraph.JobInformation;
+import org.apache.flink.runtime.executiongraph.NoOpExecutionDeploymentListener;
+import org.apache.flink.runtime.executiongraph.TaskExecutionStateTransition;
+import org.apache.flink.runtime.executiongraph.TestingExecutionGraphBuilder;
+import 
org.apache.flink.runtime.executiongraph.failover.flip1.partitionrelease.PartitionReleaseStrategyFactoryLoader;
+import 
org.apache.flink.runtime.io.network.partition.NoOpJobMasterPartitionTracker;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.ScheduleMode;
+import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway;
+import org.apache.flink.runtime.jobmaster.LogicalSlot;
+import org.apache.flink.runtime.jobmaster.TestingLogicalSlotBuilder;
+import org.apache.flink.runtime.scheduler.ExecutionGraphHandler;
+import org.apache.flink.runtime.scheduler.OperatorCoordinatorHandler;
+import org.apache.flink.runtime.shuffle.NettyShuffleMaster;
+import org.apache.flink.runtime.taskmanager.TaskExecutionState;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.SerializedValue;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Collections;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Supplier;
+
+import static 
org.apache.flink.runtime.scheduler.declarative.WaitingForResourcesTest.assertNonNull;
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.junit.Assert.assertThat;
+
+/** Tests for declarative scheduler's {@link Executing} state. */
+public class ExecutingTest extends TestLogger {
+    @Test
+    public void testTransitionToFailing() throws Exception {
+        final String failureMsg = "test exception";
+        try (MockExecutingContext ctx = new MockExecutingContext()) {
+            Executing exec = getExecutingState(ctx);
+            exec.onEnter();
+            ctx.setExpectFailing(
+                    (failingArguments -> {
+                        assertThat(failingArguments.getExecutionGraph(), 
notNullValue());
+                        
assertThat(failingArguments.getFailureCause().getMessage(), is(failureMsg));
+                    }));
+            ctx.setHowToHandleFailure(Executing.FailureResult::canNotRestart);
+            exec.handleGlobalFailure(new RuntimeException(failureMsg));
+        }
+    }
+
+    @Test
+    public void testTransitionToRestarting() throws Exception {
+        final Duration duration = Duration.ZERO;
+        try (MockExecutingContext ctx = new MockExecutingContext()) {
+            Executing exec = getExecutingState(ctx);
+            exec.onEnter();
+            ctx.setExpectRestarting(
+                    (restartingArguments ->
+                            assertThat(restartingArguments.getBackoffTime(), 
is(duration))));
+            ctx.setHowToHandleFailure((t) -> 
Executing.FailureResult.canRestart(duration));
+            exec.handleGlobalFailure(new RuntimeException("Recoverable 
error"));
+        }
+    }
+
+    @Test
+    public void testTransitionToCancelling() throws Exception {
+        try (MockExecutingContext ctx = new MockExecutingContext()) {
+            Executing exec = getExecutingState(ctx);
+            exec.onEnter();
+            ctx.setExpectCancelling(assertNonNull());
+            exec.cancel();
+        }
+    }
+
+    @Test
+    public void testTransitionToFinishedOnTerminalState() throws Exception {
+        try (MockExecutingContext ctx = new MockExecutingContext()) {
+            Executing exec = getExecutingState(ctx);
+            exec.onEnter();
+            ctx.setExpectFinished(
+                    archivedExecutionGraph ->
+                            assertThat(archivedExecutionGraph.getState(), 
is(JobStatus.FAILED)));
+
+            ctx.setExpectedStateChecker((state) -> state == exec);
+            // transition EG into terminal state, which will notify the 
Executing state about the
+            // failure (async via the supplied executor)
+            exec.getExecutionGraph().failJob(new RuntimeException("test 
failure"));
+        }
+    }
+
+    @Test
+    public void testTransitionToFinishedOnSuspend() throws Exception {
+        try (MockExecutingContext ctx = new MockExecutingContext()) {
+            Executing exec = getExecutingState(ctx);
+            ctx.setExpectedStateChecker((state) -> state == exec);
+
+            ctx.setExpectFinished(
+                    archivedExecutionGraph -> {
+                        assertThat(archivedExecutionGraph.getState(), 
is(JobStatus.SUSPENDED));
+                    });
+            exec.onEnter();
+            exec.suspend(new RuntimeException("suspend"));
+        }
+    }
+
+    @Test
+    public void testScaleUp() throws Exception {
+        try (MockExecutingContext ctx = new MockExecutingContext()) {
+            Executing exec = getExecutingState(ctx);
+            exec.onEnter();
+
+            ctx.setExpectRestarting(
+                    restartingArguments -> {
+                        // expect immediate restart on scale up
+                        assertThat(restartingArguments.getBackoffTime(), 
is(Duration.ZERO));
+                    });
+            ctx.setCanScaleUp(() -> true);
+            exec.notifyNewResourcesAvailable();
+        }
+    }
+
+    @Test
+    public void testNoScaleUp() throws Exception {
+        try (MockExecutingContext ctx = new MockExecutingContext()) {
+            Executing exec = getExecutingState(ctx);
+            ctx.setCanScaleUp(() -> false);
+            exec.onEnter();
+            exec.notifyNewResourcesAvailable();
+        }
+    }
+
+    @Test
+    public void testFailingOnDeploymentFailure() throws Exception {
+        try (MockExecutingContext ctx = new MockExecutingContext()) {
+            ctx.setCanScaleUp(() -> false);
+            ctx.setHowToHandleFailure(Executing.FailureResult::canNotRestart);
+            ctx.setExpectFailing(assertNonNull());
+
+            // create ExecutionGraph with one ExecutionVertex, which fails 
during deployment.
+            JobGraph jobGraph = new JobGraph(new JobVertex("test"));
+            Executing exec = getExecutingState(ctx, null, jobGraph);
+            TestingLogicalSlotBuilder slotBuilder = new 
TestingLogicalSlotBuilder();
+            TaskManagerGateway taskManagerGateway =
+                    new 
ExecutionVertexDeploymentTest.SubmitFailingSimpleAckingTaskManagerGateway();
+            slotBuilder.setTaskManagerGateway(taskManagerGateway);
+            LogicalSlot slot = slotBuilder.createTestingLogicalSlot();
+            exec.getExecutionGraph()
+                    .getAllExecutionVertices()
+                    .forEach(executionVertex -> 
executionVertex.tryAssignResource(slot));
+
+            // trigger deployment
+            exec.onEnter();
+        }
+    }

Review comment:
       I believe the test above (and the additional test you are proposing) are 
not needed, as it is not the responsibility of the Executing state to handle 
failures during deployment. Of course it is, but it is indirect. Failures 
during deployment are reported as `updateTaskExecutionState()` or 
`handleGlobalFailure` via the `UpdateSchedulerNgOnInternalFailuresListener`, 
which should be tested separately (as part of tests for the declarative 
scheduler itself).




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