You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/07/08 15:19:54 UTC

[GitHub] [flink] reswqa commented on a diff in pull request #20222: [FLINK-28137] Introduce SpeculativeScheduler

reswqa commented on code in PR #20222:
URL: https://github.com/apache/flink/pull/20222#discussion_r916889626


##########
flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandlerTest.java:
##########
@@ -71,122 +74,125 @@ public void setUp() {
 
     /** Tests the case that task restarting is accepted. */
     @Test
-    public void testNormalFailureHandling() {
+    void testNormalFailureHandling() throws Exception {
         final Set<ExecutionVertexID> tasksToRestart =
                 Collections.singleton(new ExecutionVertexID(new JobVertexID(), 0));
         failoverStrategy.setTasksToRestart(tasksToRestart);
 
+        Execution execution = createExecution(EXECUTOR_RESOURCE.getExecutor());
         Exception cause = new Exception("test failure");
         long timestamp = System.currentTimeMillis();
         // trigger a task failure
         final FailureHandlingResult result =
-                executionFailureHandler.getFailureHandlingResult(
-                        new ExecutionVertexID(new JobVertexID(), 0), cause, timestamp);
+                executionFailureHandler.getFailureHandlingResult(execution, cause, timestamp);
 
         // verify results
-        assertTrue(result.canRestart());
-        assertEquals(RESTART_DELAY_MS, result.getRestartDelayMS());
-        assertEquals(tasksToRestart, result.getVerticesToRestart());
-        assertThat(result.getError(), is(cause));
-        assertThat(result.getTimestamp(), is(timestamp));
-        assertEquals(1, executionFailureHandler.getNumberOfRestarts());
+        assertThat(result.canRestart()).isTrue();
+        assertThat(result.getFailedExecution().isPresent()).isTrue();
+        assertThat(result.getFailedExecution().get()).isSameAs(execution);
+        assertThat(result.getRestartDelayMS()).isEqualTo(RESTART_DELAY_MS);
+        assertThat(result.getVerticesToRestart()).isEqualTo(tasksToRestart);
+        assertThat(result.getError()).isSameAs(cause);
+        assertThat(result.getTimestamp()).isEqualTo(timestamp);
+        assertThat(executionFailureHandler.getNumberOfRestarts()).isEqualTo(1);
     }
 
     /** Tests the case that task restarting is suppressed. */
     @Test
-    public void testRestartingSuppressedFailureHandlingResult() {
+    void testRestartingSuppressedFailureHandlingResult() throws Exception {
         // restart strategy suppresses restarting
         backoffTimeStrategy.setCanRestart(false);
 
         // trigger a task failure
+        Execution execution = createExecution(EXECUTOR_RESOURCE.getExecutor());
         final Throwable error = new Exception("expected test failure");
         final long timestamp = System.currentTimeMillis();
         final FailureHandlingResult result =
-                executionFailureHandler.getFailureHandlingResult(
-                        new ExecutionVertexID(new JobVertexID(), 0), error, timestamp);
+                executionFailureHandler.getFailureHandlingResult(execution, error, timestamp);
 
         // verify results
-        assertFalse(result.canRestart());
-        assertThat(result.getError(), containsCause(error));
-        assertThat(result.getTimestamp(), is(timestamp));
-        assertFalse(ExecutionFailureHandler.isUnrecoverableError(result.getError()));
-        try {
-            result.getVerticesToRestart();
-            fail("get tasks to restart is not allowed when restarting is suppressed");
-        } catch (IllegalStateException ex) {
-            // expected
-        }
-        try {
-            result.getRestartDelayMS();
-            fail("get restart delay is not allowed when restarting is suppressed");
-        } catch (IllegalStateException ex) {
-            // expected
-        }
-        assertEquals(0, executionFailureHandler.getNumberOfRestarts());
+        assertThat(result.canRestart()).isFalse();
+        assertThat(result.getFailedExecution().isPresent()).isTrue();
+        assertThat(result.getFailedExecution().get()).isSameAs(execution);
+        assertThat(result.getError()).hasCause(error);
+        assertThat(result.getTimestamp()).isEqualTo(timestamp);
+        assertThat(ExecutionFailureHandler.isUnrecoverableError(result.getError())).isFalse();
+
+        assertThatThrownBy(() -> result.getVerticesToRestart())
+                .as("getVerticesToRestart is not allowed when restarting is suppressed")
+                .isInstanceOf(IllegalStateException.class);
+
+        assertThatThrownBy(() -> result.getRestartDelayMS())
+                .as("getRestartDelayMS is not allowed when restarting is suppressed")
+                .isInstanceOf(IllegalStateException.class);
+
+        assertThat(executionFailureHandler.getNumberOfRestarts()).isZero();
     }
 
     /** Tests the case that the failure is non-recoverable type. */
     @Test
-    public void testNonRecoverableFailureHandlingResult() {
+    void testNonRecoverableFailureHandlingResult() throws Exception {
+
         // trigger an unrecoverable task failure
+        Execution execution = createExecution(EXECUTOR_RESOURCE.getExecutor());
         final Throwable error =
                 new Exception(new SuppressRestartsException(new Exception("test failure")));
         final long timestamp = System.currentTimeMillis();
         final FailureHandlingResult result =
-                executionFailureHandler.getFailureHandlingResult(
-                        new ExecutionVertexID(new JobVertexID(), 0), error, timestamp);
+                executionFailureHandler.getFailureHandlingResult(execution, error, timestamp);
 
         // verify results
-        assertFalse(result.canRestart());
-        assertNotNull(result.getError());
-        assertTrue(ExecutionFailureHandler.isUnrecoverableError(result.getError()));
-        assertThat(result.getTimestamp(), is(timestamp));
-        try {
-            result.getVerticesToRestart();
-            fail("get tasks to restart is not allowed when restarting is suppressed");
-        } catch (IllegalStateException ex) {
-            // expected
-        }
-        try {
-            result.getRestartDelayMS();
-            fail("get restart delay is not allowed when restarting is suppressed");
-        } catch (IllegalStateException ex) {
-            // expected
-        }
-        assertEquals(0, executionFailureHandler.getNumberOfRestarts());
+        assertThat(result.canRestart()).isFalse();
+        assertThat(result.getFailedExecution().isPresent()).isTrue();
+        assertThat(result.getFailedExecution().get()).isSameAs(execution);
+        assertThat(result.getError()).isNotNull();
+        assertThat(ExecutionFailureHandler.isUnrecoverableError(result.getError())).isTrue();
+        assertThat(result.getTimestamp()).isEqualTo(timestamp);
+
+        assertThatThrownBy(() -> result.getVerticesToRestart())
+                .as("getVerticesToRestart is not allowed when restarting is suppressed")
+                .isInstanceOf(IllegalStateException.class);
+
+        assertThatThrownBy(() -> result.getRestartDelayMS())

Review Comment:
   ```suggestion
           assertThatThrownBy(result::getRestartDelayMS)
   ```



##########
flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandlerTest.java:
##########
@@ -71,122 +74,125 @@ public void setUp() {
 
     /** Tests the case that task restarting is accepted. */
     @Test
-    public void testNormalFailureHandling() {
+    void testNormalFailureHandling() throws Exception {
         final Set<ExecutionVertexID> tasksToRestart =
                 Collections.singleton(new ExecutionVertexID(new JobVertexID(), 0));
         failoverStrategy.setTasksToRestart(tasksToRestart);
 
+        Execution execution = createExecution(EXECUTOR_RESOURCE.getExecutor());
         Exception cause = new Exception("test failure");
         long timestamp = System.currentTimeMillis();
         // trigger a task failure
         final FailureHandlingResult result =
-                executionFailureHandler.getFailureHandlingResult(
-                        new ExecutionVertexID(new JobVertexID(), 0), cause, timestamp);
+                executionFailureHandler.getFailureHandlingResult(execution, cause, timestamp);
 
         // verify results
-        assertTrue(result.canRestart());
-        assertEquals(RESTART_DELAY_MS, result.getRestartDelayMS());
-        assertEquals(tasksToRestart, result.getVerticesToRestart());
-        assertThat(result.getError(), is(cause));
-        assertThat(result.getTimestamp(), is(timestamp));
-        assertEquals(1, executionFailureHandler.getNumberOfRestarts());
+        assertThat(result.canRestart()).isTrue();
+        assertThat(result.getFailedExecution().isPresent()).isTrue();
+        assertThat(result.getFailedExecution().get()).isSameAs(execution);
+        assertThat(result.getRestartDelayMS()).isEqualTo(RESTART_DELAY_MS);
+        assertThat(result.getVerticesToRestart()).isEqualTo(tasksToRestart);
+        assertThat(result.getError()).isSameAs(cause);
+        assertThat(result.getTimestamp()).isEqualTo(timestamp);
+        assertThat(executionFailureHandler.getNumberOfRestarts()).isEqualTo(1);
     }
 
     /** Tests the case that task restarting is suppressed. */
     @Test
-    public void testRestartingSuppressedFailureHandlingResult() {
+    void testRestartingSuppressedFailureHandlingResult() throws Exception {
         // restart strategy suppresses restarting
         backoffTimeStrategy.setCanRestart(false);
 
         // trigger a task failure
+        Execution execution = createExecution(EXECUTOR_RESOURCE.getExecutor());
         final Throwable error = new Exception("expected test failure");
         final long timestamp = System.currentTimeMillis();
         final FailureHandlingResult result =
-                executionFailureHandler.getFailureHandlingResult(
-                        new ExecutionVertexID(new JobVertexID(), 0), error, timestamp);
+                executionFailureHandler.getFailureHandlingResult(execution, error, timestamp);
 
         // verify results
-        assertFalse(result.canRestart());
-        assertThat(result.getError(), containsCause(error));
-        assertThat(result.getTimestamp(), is(timestamp));
-        assertFalse(ExecutionFailureHandler.isUnrecoverableError(result.getError()));
-        try {
-            result.getVerticesToRestart();
-            fail("get tasks to restart is not allowed when restarting is suppressed");
-        } catch (IllegalStateException ex) {
-            // expected
-        }
-        try {
-            result.getRestartDelayMS();
-            fail("get restart delay is not allowed when restarting is suppressed");
-        } catch (IllegalStateException ex) {
-            // expected
-        }
-        assertEquals(0, executionFailureHandler.getNumberOfRestarts());
+        assertThat(result.canRestart()).isFalse();
+        assertThat(result.getFailedExecution().isPresent()).isTrue();
+        assertThat(result.getFailedExecution().get()).isSameAs(execution);
+        assertThat(result.getError()).hasCause(error);
+        assertThat(result.getTimestamp()).isEqualTo(timestamp);
+        assertThat(ExecutionFailureHandler.isUnrecoverableError(result.getError())).isFalse();
+
+        assertThatThrownBy(() -> result.getVerticesToRestart())
+                .as("getVerticesToRestart is not allowed when restarting is suppressed")
+                .isInstanceOf(IllegalStateException.class);
+
+        assertThatThrownBy(() -> result.getRestartDelayMS())

Review Comment:
   ```suggestion
           assertThatThrownBy(result::getRestartDelayMS)
   ```



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/SpeculativeScheduler.java:
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.adaptivebatch;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.runtime.blocklist.BlockedNode;
+import org.apache.flink.runtime.blocklist.BlocklistHandler;
+import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
+import org.apache.flink.runtime.checkpoint.CheckpointsCleaner;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.JobStatusListener;
+import org.apache.flink.runtime.executiongraph.SpeculativeExecutionVertex;
+import org.apache.flink.runtime.executiongraph.failover.flip1.FailoverStrategy;
+import org.apache.flink.runtime.executiongraph.failover.flip1.RestartBackoffTimeStrategy;
+import org.apache.flink.runtime.io.network.partition.PartitionException;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.metrics.groups.JobManagerJobMetricGroup;
+import org.apache.flink.runtime.scheduler.ExecutionGraphFactory;
+import org.apache.flink.runtime.scheduler.ExecutionOperations;
+import org.apache.flink.runtime.scheduler.ExecutionSlotAllocatorFactory;
+import org.apache.flink.runtime.scheduler.ExecutionVertexVersioner;
+import org.apache.flink.runtime.scheduler.slowtaskdetector.ExecutionTimeBasedSlowTaskDetector;
+import org.apache.flink.runtime.scheduler.slowtaskdetector.SlowTaskDetector;
+import org.apache.flink.runtime.scheduler.slowtaskdetector.SlowTaskDetectorListener;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.scheduler.strategy.SchedulingStrategyFactory;
+import org.apache.flink.runtime.shuffle.ShuffleMaster;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+import org.apache.flink.runtime.topology.Vertex;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.IterableUtils;
+import org.apache.flink.util.concurrent.FutureUtils;
+import org.apache.flink.util.concurrent.ScheduledExecutor;
+
+import org.slf4j.Logger;
+
+import javax.annotation.Nullable;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** The speculative scheduler. */
+public class SpeculativeScheduler extends AdaptiveBatchScheduler
+        implements SlowTaskDetectorListener {
+
+    private final int maxConcurrentExecutions;
+
+    private final Duration blockSlowNodeDuration;
+
+    private final BlocklistHandler blocklistHandler;
+
+    private final SlowTaskDetector slowTaskDetector;
+
+    public SpeculativeScheduler(
+            final Logger log,
+            final JobGraph jobGraph,
+            final Executor ioExecutor,
+            final Configuration jobMasterConfiguration,
+            final Consumer<ComponentMainThreadExecutor> startUpAction,
+            final ScheduledExecutor delayExecutor,
+            final ClassLoader userCodeLoader,
+            final CheckpointsCleaner checkpointsCleaner,
+            final CheckpointRecoveryFactory checkpointRecoveryFactory,
+            final JobManagerJobMetricGroup jobManagerJobMetricGroup,
+            final SchedulingStrategyFactory schedulingStrategyFactory,
+            final FailoverStrategy.Factory failoverStrategyFactory,
+            final RestartBackoffTimeStrategy restartBackoffTimeStrategy,
+            final ExecutionOperations executionOperations,
+            final ExecutionVertexVersioner executionVertexVersioner,
+            final ExecutionSlotAllocatorFactory executionSlotAllocatorFactory,
+            long initializationTimestamp,
+            final ComponentMainThreadExecutor mainThreadExecutor,
+            final JobStatusListener jobStatusListener,
+            final ExecutionGraphFactory executionGraphFactory,
+            final ShuffleMaster<?> shuffleMaster,
+            final Time rpcTimeout,
+            final VertexParallelismDecider vertexParallelismDecider,
+            final int defaultMaxParallelism,
+            final BlocklistHandler blocklistHandler)
+            throws Exception {
+
+        super(
+                log,
+                jobGraph,
+                ioExecutor,
+                jobMasterConfiguration,
+                startUpAction,
+                delayExecutor,
+                userCodeLoader,
+                checkpointsCleaner,
+                checkpointRecoveryFactory,
+                jobManagerJobMetricGroup,
+                schedulingStrategyFactory,
+                failoverStrategyFactory,
+                restartBackoffTimeStrategy,
+                executionOperations,
+                executionVertexVersioner,
+                executionSlotAllocatorFactory,
+                initializationTimestamp,
+                mainThreadExecutor,
+                jobStatusListener,
+                executionGraphFactory,
+                shuffleMaster,
+                rpcTimeout,
+                vertexParallelismDecider,
+                defaultMaxParallelism);
+
+        this.maxConcurrentExecutions =
+                jobMasterConfiguration.getInteger(
+                        JobManagerOptions.SPECULATIVE_MAX_CONCURRENT_EXECUTIONS);
+
+        this.blockSlowNodeDuration =
+                jobMasterConfiguration.get(JobManagerOptions.BLOCK_SLOW_NODE_DURATION);
+
+        this.blocklistHandler = checkNotNull(blocklistHandler);
+
+        this.slowTaskDetector = new ExecutionTimeBasedSlowTaskDetector(jobMasterConfiguration);
+    }
+
+    @Override
+    protected void startSchedulingInternal() {
+        super.startSchedulingInternal();
+        slowTaskDetector.start(getExecutionGraph(), this, getMainThreadExecutor());
+    }
+
+    @Override
+    public CompletableFuture<Void> closeAsync() {
+        CompletableFuture<Void> future = super.closeAsync();
+        slowTaskDetector.stop();
+        return future;
+    }
+
+    @Override
+    public SpeculativeExecutionVertex getExecutionVertex(ExecutionVertexID executionVertexId) {
+        return (SpeculativeExecutionVertex) super.getExecutionVertex(executionVertexId);
+    }
+
+    @Override
+    protected void onTaskFinished(final Execution execution) {
+        // cancel all un-terminated executions because the execution vertex has finished
+        FutureUtils.assertNoException(cancelPendingExecutions(execution.getVertex().getID()));
+
+        initializeVerticesIfPossible();
+
+        super.onTaskFinished(execution);
+    }
+
+    private CompletableFuture<?> cancelPendingExecutions(
+            final ExecutionVertexID executionVertexId) {
+        final SpeculativeExecutionVertex executionVertex = getExecutionVertex(executionVertexId);
+
+        final List<CompletableFuture<?>> cancelingFutures = new ArrayList<>();
+        for (Execution execution : executionVertex.getCurrentExecutions()) {
+            if (!execution.getState().isTerminal()) {
+                execution.cancel();
+                cancelingFutures.add(execution.getReleaseFuture());
+            }
+        }
+        cancelAllPendingSlotRequests(executionVertexId);
+        return FutureUtils.combineAll(cancelingFutures);
+    }
+
+    @Override
+    protected void onTaskFailed(final Execution execution) {
+        final SpeculativeExecutionVertex executionVertex =
+                getExecutionVertex(execution.getVertex().getID());
+        final ExecutionAttemptID attemptId = execution.getAttemptId();
+
+        // when an execution fails, remove it from current executions to make room for future
+        // speculative executions
+        executionVertex.archiveFailedExecution(attemptId);
+        executionSlotAllocator.cancel(attemptId);
+
+        super.onTaskFailed(execution);
+    }
+
+    @Override
+    protected void handleTaskFailure(
+            final Execution failedExecution, @Nullable final Throwable error) {
+
+        final SpeculativeExecutionVertex executionVertex =
+                getExecutionVertex(failedExecution.getVertex().getID());
+
+        // if the execution vertex is not possible finish or a PartitionException occurred, trigger
+        // an execution vertex failover to recover
+        if (!isExecutionVertexPossibleToFinish(executionVertex)
+                || ExceptionUtils.findThrowable(error, PartitionException.class).isPresent()) {
+            super.handleTaskFailure(failedExecution, error);
+        } else {
+            // add the execution failure to exception history even though not restarting the entire
+            // execution vertex
+            final long timestamp = System.currentTimeMillis();
+            setGlobalFailureCause(error, timestamp);
+            archiveFromFailureHandlingResult(
+                    createFailureHandlingResultSnapshot(
+                            executionFailureHandler.getFailureHandlingResult(
+                                    failedExecution, error, timestamp)));
+        }
+    }
+
+    private static boolean isExecutionVertexPossibleToFinish(
+            final SpeculativeExecutionVertex executionVertex) {
+        boolean anyExecutionPossibleToFinish = false;
+        for (Execution execution : executionVertex.getCurrentExecutions()) {
+            // if any execution has finished, no execution of the same execution vertex should fail
+            // after that
+            checkState(execution.getState() != ExecutionState.FINISHED);
+
+            if (execution.getState() == ExecutionState.CREATED
+                    || execution.getState() == ExecutionState.SCHEDULED
+                    || execution.getState() == ExecutionState.DEPLOYING
+                    || execution.getState() == ExecutionState.INITIALIZING
+                    || execution.getState() == ExecutionState.RUNNING) {
+                anyExecutionPossibleToFinish = true;
+            }
+        }
+        return anyExecutionPossibleToFinish;
+    }
+
+    @Override
+    protected void cancelAllPendingSlotRequestsInternal() {
+        IterableUtils.toStream(getSchedulingTopology().getVertices())
+                .map(Vertex::getId)
+                .forEach(this::cancelAllPendingSlotRequests);
+    }
+
+    @Override
+    protected void cancelAllPendingSlotRequestsForVertices(
+            final Set<ExecutionVertexID> executionVertices) {
+        executionVertices.forEach(this::cancelAllPendingSlotRequests);
+    }
+
+    private void cancelAllPendingSlotRequests(final ExecutionVertexID executionVertexId) {
+        final SpeculativeExecutionVertex executionVertex = getExecutionVertex(executionVertexId);
+        executionVertex
+                .getCurrentExecutions()
+                .forEach(e -> executionSlotAllocator.cancel(e.getAttemptId()));
+    }
+
+    @Override
+    public void notifySlowTasks(Map<ExecutionVertexID, Collection<ExecutionAttemptID>> slowTasks) {
+        // add slow nodes to blocklist before scheduling new speculative executions
+        final long blockedEndTimestamp =
+                System.currentTimeMillis() + blockSlowNodeDuration.toMillis();
+        final Collection<BlockedNode> nodesToBlock =
+                getSlowNodeIds(slowTasks).stream()
+                        .map(
+                                nodeId ->
+                                        new BlockedNode(
+                                                nodeId,
+                                                "Node is detected to be slow.",
+                                                blockedEndTimestamp))
+                        .collect(Collectors.toList());
+        blocklistHandler.addNewBlockedNodes(nodesToBlock);
+
+        final List<Execution> newSpeculativeExecutions = new ArrayList<>();
+        final Set<ExecutionVertexID> verticesToDeploy = new HashSet<>();
+        for (ExecutionVertexID executionVertexId : slowTasks.keySet()) {
+            final SpeculativeExecutionVertex executionVertex =
+                    getExecutionVertex(executionVertexId);
+
+            if (executionVertex.containsSources() || executionVertex.containsSinks()) {
+                continue;
+            }
+
+            final int currentConcurrentExecutions = executionVertex.getCurrentExecutions().size();
+            final int newSpeculativeExecutionsToDeploy =
+                    maxConcurrentExecutions - currentConcurrentExecutions;
+            if (newSpeculativeExecutionsToDeploy > 0) {
+                log.info(
+                        "{} ({}) is detected as a slow vertex, create and deploy {} new speculative executions for it.",
+                        executionVertex.getTaskNameWithSubtaskIndex(),
+                        executionVertex.getID(),
+                        newSpeculativeExecutionsToDeploy);
+
+                verticesToDeploy.add(executionVertexId);
+                IntStream.range(0, newSpeculativeExecutionsToDeploy)
+                        .mapToObj(executionVertex::createNewSpeculativeExecution)
+                        .forEach(newSpeculativeExecutions::add);
+            }
+        }
+
+        executionDeployer.allocateSlotsAndDeploy(
+                newSpeculativeExecutions,
+                executionVertexVersioner.getExecutionVertexVersions(verticesToDeploy));
+    }
+
+    private Set<String> getSlowNodeIds(
+            Map<ExecutionVertexID, Collection<ExecutionAttemptID>> slowTasks) {
+        final Set<ExecutionAttemptID> slowExecutions =
+                slowTasks.values().stream()
+                        .flatMap(ids -> ids.stream())
+                        .collect(Collectors.toSet());

Review Comment:
   ```suggestion
         slowTasks.values().stream().flatMap(Collection::stream).collect(Collectors.toSet());
   ```



##########
flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandlerTest.java:
##########
@@ -71,122 +74,125 @@ public void setUp() {
 
     /** Tests the case that task restarting is accepted. */
     @Test
-    public void testNormalFailureHandling() {
+    void testNormalFailureHandling() throws Exception {
         final Set<ExecutionVertexID> tasksToRestart =
                 Collections.singleton(new ExecutionVertexID(new JobVertexID(), 0));
         failoverStrategy.setTasksToRestart(tasksToRestart);
 
+        Execution execution = createExecution(EXECUTOR_RESOURCE.getExecutor());
         Exception cause = new Exception("test failure");
         long timestamp = System.currentTimeMillis();
         // trigger a task failure
         final FailureHandlingResult result =
-                executionFailureHandler.getFailureHandlingResult(
-                        new ExecutionVertexID(new JobVertexID(), 0), cause, timestamp);
+                executionFailureHandler.getFailureHandlingResult(execution, cause, timestamp);
 
         // verify results
-        assertTrue(result.canRestart());
-        assertEquals(RESTART_DELAY_MS, result.getRestartDelayMS());
-        assertEquals(tasksToRestart, result.getVerticesToRestart());
-        assertThat(result.getError(), is(cause));
-        assertThat(result.getTimestamp(), is(timestamp));
-        assertEquals(1, executionFailureHandler.getNumberOfRestarts());
+        assertThat(result.canRestart()).isTrue();
+        assertThat(result.getFailedExecution().isPresent()).isTrue();
+        assertThat(result.getFailedExecution().get()).isSameAs(execution);
+        assertThat(result.getRestartDelayMS()).isEqualTo(RESTART_DELAY_MS);
+        assertThat(result.getVerticesToRestart()).isEqualTo(tasksToRestart);
+        assertThat(result.getError()).isSameAs(cause);
+        assertThat(result.getTimestamp()).isEqualTo(timestamp);
+        assertThat(executionFailureHandler.getNumberOfRestarts()).isEqualTo(1);
     }
 
     /** Tests the case that task restarting is suppressed. */
     @Test
-    public void testRestartingSuppressedFailureHandlingResult() {
+    void testRestartingSuppressedFailureHandlingResult() throws Exception {
         // restart strategy suppresses restarting
         backoffTimeStrategy.setCanRestart(false);
 
         // trigger a task failure
+        Execution execution = createExecution(EXECUTOR_RESOURCE.getExecutor());
         final Throwable error = new Exception("expected test failure");
         final long timestamp = System.currentTimeMillis();
         final FailureHandlingResult result =
-                executionFailureHandler.getFailureHandlingResult(
-                        new ExecutionVertexID(new JobVertexID(), 0), error, timestamp);
+                executionFailureHandler.getFailureHandlingResult(execution, error, timestamp);
 
         // verify results
-        assertFalse(result.canRestart());
-        assertThat(result.getError(), containsCause(error));
-        assertThat(result.getTimestamp(), is(timestamp));
-        assertFalse(ExecutionFailureHandler.isUnrecoverableError(result.getError()));
-        try {
-            result.getVerticesToRestart();
-            fail("get tasks to restart is not allowed when restarting is suppressed");
-        } catch (IllegalStateException ex) {
-            // expected
-        }
-        try {
-            result.getRestartDelayMS();
-            fail("get restart delay is not allowed when restarting is suppressed");
-        } catch (IllegalStateException ex) {
-            // expected
-        }
-        assertEquals(0, executionFailureHandler.getNumberOfRestarts());
+        assertThat(result.canRestart()).isFalse();
+        assertThat(result.getFailedExecution().isPresent()).isTrue();
+        assertThat(result.getFailedExecution().get()).isSameAs(execution);
+        assertThat(result.getError()).hasCause(error);
+        assertThat(result.getTimestamp()).isEqualTo(timestamp);
+        assertThat(ExecutionFailureHandler.isUnrecoverableError(result.getError())).isFalse();
+
+        assertThatThrownBy(() -> result.getVerticesToRestart())
+                .as("getVerticesToRestart is not allowed when restarting is suppressed")
+                .isInstanceOf(IllegalStateException.class);
+
+        assertThatThrownBy(() -> result.getRestartDelayMS())
+                .as("getRestartDelayMS is not allowed when restarting is suppressed")
+                .isInstanceOf(IllegalStateException.class);
+
+        assertThat(executionFailureHandler.getNumberOfRestarts()).isZero();
     }
 
     /** Tests the case that the failure is non-recoverable type. */
     @Test
-    public void testNonRecoverableFailureHandlingResult() {
+    void testNonRecoverableFailureHandlingResult() throws Exception {
+
         // trigger an unrecoverable task failure
+        Execution execution = createExecution(EXECUTOR_RESOURCE.getExecutor());
         final Throwable error =
                 new Exception(new SuppressRestartsException(new Exception("test failure")));
         final long timestamp = System.currentTimeMillis();
         final FailureHandlingResult result =
-                executionFailureHandler.getFailureHandlingResult(
-                        new ExecutionVertexID(new JobVertexID(), 0), error, timestamp);
+                executionFailureHandler.getFailureHandlingResult(execution, error, timestamp);
 
         // verify results
-        assertFalse(result.canRestart());
-        assertNotNull(result.getError());
-        assertTrue(ExecutionFailureHandler.isUnrecoverableError(result.getError()));
-        assertThat(result.getTimestamp(), is(timestamp));
-        try {
-            result.getVerticesToRestart();
-            fail("get tasks to restart is not allowed when restarting is suppressed");
-        } catch (IllegalStateException ex) {
-            // expected
-        }
-        try {
-            result.getRestartDelayMS();
-            fail("get restart delay is not allowed when restarting is suppressed");
-        } catch (IllegalStateException ex) {
-            // expected
-        }
-        assertEquals(0, executionFailureHandler.getNumberOfRestarts());
+        assertThat(result.canRestart()).isFalse();
+        assertThat(result.getFailedExecution().isPresent()).isTrue();
+        assertThat(result.getFailedExecution().get()).isSameAs(execution);
+        assertThat(result.getError()).isNotNull();
+        assertThat(ExecutionFailureHandler.isUnrecoverableError(result.getError())).isTrue();
+        assertThat(result.getTimestamp()).isEqualTo(timestamp);
+
+        assertThatThrownBy(() -> result.getVerticesToRestart())

Review Comment:
   ```suggestion
           assertThatThrownBy(result::getVerticesToRestart)
   ```



##########
flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandlerTest.java:
##########
@@ -71,122 +74,125 @@ public void setUp() {
 
     /** Tests the case that task restarting is accepted. */
     @Test
-    public void testNormalFailureHandling() {
+    void testNormalFailureHandling() throws Exception {
         final Set<ExecutionVertexID> tasksToRestart =
                 Collections.singleton(new ExecutionVertexID(new JobVertexID(), 0));
         failoverStrategy.setTasksToRestart(tasksToRestart);
 
+        Execution execution = createExecution(EXECUTOR_RESOURCE.getExecutor());
         Exception cause = new Exception("test failure");
         long timestamp = System.currentTimeMillis();
         // trigger a task failure
         final FailureHandlingResult result =
-                executionFailureHandler.getFailureHandlingResult(
-                        new ExecutionVertexID(new JobVertexID(), 0), cause, timestamp);
+                executionFailureHandler.getFailureHandlingResult(execution, cause, timestamp);
 
         // verify results
-        assertTrue(result.canRestart());
-        assertEquals(RESTART_DELAY_MS, result.getRestartDelayMS());
-        assertEquals(tasksToRestart, result.getVerticesToRestart());
-        assertThat(result.getError(), is(cause));
-        assertThat(result.getTimestamp(), is(timestamp));
-        assertEquals(1, executionFailureHandler.getNumberOfRestarts());
+        assertThat(result.canRestart()).isTrue();
+        assertThat(result.getFailedExecution().isPresent()).isTrue();
+        assertThat(result.getFailedExecution().get()).isSameAs(execution);
+        assertThat(result.getRestartDelayMS()).isEqualTo(RESTART_DELAY_MS);
+        assertThat(result.getVerticesToRestart()).isEqualTo(tasksToRestart);
+        assertThat(result.getError()).isSameAs(cause);
+        assertThat(result.getTimestamp()).isEqualTo(timestamp);
+        assertThat(executionFailureHandler.getNumberOfRestarts()).isEqualTo(1);
     }
 
     /** Tests the case that task restarting is suppressed. */
     @Test
-    public void testRestartingSuppressedFailureHandlingResult() {
+    void testRestartingSuppressedFailureHandlingResult() throws Exception {
         // restart strategy suppresses restarting
         backoffTimeStrategy.setCanRestart(false);
 
         // trigger a task failure
+        Execution execution = createExecution(EXECUTOR_RESOURCE.getExecutor());
         final Throwable error = new Exception("expected test failure");
         final long timestamp = System.currentTimeMillis();
         final FailureHandlingResult result =
-                executionFailureHandler.getFailureHandlingResult(
-                        new ExecutionVertexID(new JobVertexID(), 0), error, timestamp);
+                executionFailureHandler.getFailureHandlingResult(execution, error, timestamp);
 
         // verify results
-        assertFalse(result.canRestart());
-        assertThat(result.getError(), containsCause(error));
-        assertThat(result.getTimestamp(), is(timestamp));
-        assertFalse(ExecutionFailureHandler.isUnrecoverableError(result.getError()));
-        try {
-            result.getVerticesToRestart();
-            fail("get tasks to restart is not allowed when restarting is suppressed");
-        } catch (IllegalStateException ex) {
-            // expected
-        }
-        try {
-            result.getRestartDelayMS();
-            fail("get restart delay is not allowed when restarting is suppressed");
-        } catch (IllegalStateException ex) {
-            // expected
-        }
-        assertEquals(0, executionFailureHandler.getNumberOfRestarts());
+        assertThat(result.canRestart()).isFalse();
+        assertThat(result.getFailedExecution().isPresent()).isTrue();
+        assertThat(result.getFailedExecution().get()).isSameAs(execution);
+        assertThat(result.getError()).hasCause(error);
+        assertThat(result.getTimestamp()).isEqualTo(timestamp);
+        assertThat(ExecutionFailureHandler.isUnrecoverableError(result.getError())).isFalse();
+
+        assertThatThrownBy(() -> result.getVerticesToRestart())

Review Comment:
   ```suggestion
           assertThatThrownBy(result::getVerticesToRestart)
   ```



##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptivebatch/SpeculativeSchedulerTest.java:
##########
@@ -0,0 +1,401 @@
+/*
+ * 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.adaptivebatch;
+
+import org.apache.flink.runtime.blocklist.BlockedNode;
+import org.apache.flink.runtime.blocklist.BlocklistHandler;
+import org.apache.flink.runtime.blocklist.BlocklistListener;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.DefaultExecutionGraph;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.IOMetrics;
+import org.apache.flink.runtime.executiongraph.failover.flip1.TestRestartBackoffTimeStrategy;
+import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobGraphTestUtils;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.scheduler.DefaultExecutionOperations;
+import org.apache.flink.runtime.scheduler.DefaultSchedulerBuilder;
+import org.apache.flink.runtime.scheduler.TestExecutionOperationsDecorator;
+import org.apache.flink.runtime.scheduler.exceptionhistory.RootExceptionHistoryEntry;
+import org.apache.flink.runtime.taskmanager.TaskExecutionState;
+import org.apache.flink.runtime.testutils.DirectScheduledExecutorService;
+import org.apache.flink.testutils.TestingUtils;
+import org.apache.flink.testutils.executor.TestExecutorExtension;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.concurrent.ManuallyTriggeredScheduledExecutor;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableMap;
+import org.apache.flink.shaded.guava30.com.google.common.collect.Iterables;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.runtime.execution.ExecutionState.CANCELING;
+import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.completeCancellingForAllVertices;
+import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.createNoOpVertex;
+import static org.apache.flink.runtime.scheduler.DefaultSchedulerTest.createFailedTaskExecutionState;
+import static org.apache.flink.runtime.scheduler.DefaultSchedulerTest.singleNonParallelJobVertexJobGraph;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for {@link SpeculativeScheduler}. */
+class SpeculativeSchedulerTest {
+
+    @RegisterExtension
+    private static final TestExecutorExtension<ScheduledExecutorService> EXECUTOR_RESOURCE =
+            TestingUtils.defaultExecutorExtension();
+
+    private ScheduledExecutorService futureExecutor;
+
+    private ManuallyTriggeredScheduledExecutor taskRestartExecutor;
+    private TestExecutionOperationsDecorator testExecutionOperations;
+    private TestBlocklistHandler testBlocklistHandler;
+
+    @BeforeEach
+    void setUp() {
+        futureExecutor = new DirectScheduledExecutorService();
+
+        taskRestartExecutor = new ManuallyTriggeredScheduledExecutor();
+        testExecutionOperations =
+                new TestExecutionOperationsDecorator(new DefaultExecutionOperations());
+        testBlocklistHandler = new TestBlocklistHandler();
+    }
+
+    @AfterEach
+    void tearDown() {
+        if (futureExecutor != null) {
+            ExecutorUtils.gracefulShutdown(10, TimeUnit.SECONDS, futureExecutor);
+        }
+    }
+
+    @Test
+    void testStartScheduling() {
+        createSchedulerAndStartScheduling();
+        final List<ExecutionAttemptID> deployedExecutions =
+                testExecutionOperations.getDeployedExecutions();
+        assertThat(deployedExecutions).hasSize(1);
+    }
+
+    @Test
+    void testNotifySlowTasks() {
+        final SpeculativeScheduler scheduler = createSchedulerAndStartScheduling();
+        final ExecutionVertex ev = getOnlyExecutionVertex(scheduler);
+        final Execution attempt1 = ev.getCurrentExecutionAttempt();
+
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(1);
+
+        notifySlowTask(scheduler, attempt1);
+
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(2);
+        assertThat(testBlocklistHandler.getAllBlockedNodeIds())
+                .containsExactly(attempt1.getAssignedResourceLocation().getNodeId());
+    }
+
+    @Test
+    void testNotifyDuplicatedSlowTasks() {
+        final SpeculativeScheduler scheduler = createSchedulerAndStartScheduling();
+        final ExecutionVertex ev = getOnlyExecutionVertex(scheduler);
+        final Execution attempt1 = ev.getCurrentExecutionAttempt();
+
+        notifySlowTask(scheduler, attempt1);
+
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(2);
+
+        // notify the execution as a slow task again
+        notifySlowTask(scheduler, attempt1);
+
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(2);
+
+        // fail attempt2 to make room for a new speculative execution
+        final Execution attempt2 = getExecution(ev, 1);
+        scheduler.updateTaskExecutionState(createFailedTaskExecutionState(attempt2.getAttemptId()));
+
+        // notify the execution as a slow task again
+        notifySlowTask(scheduler, attempt1);
+
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(3);
+    }
+
+    @Test
+    void testRestartVertexIfAllSpeculativeExecutionFailed() {
+        final SpeculativeScheduler scheduler = createSchedulerAndStartScheduling();
+        final ExecutionVertex ev = getOnlyExecutionVertex(scheduler);
+        final Execution attempt1 = ev.getCurrentExecutionAttempt();
+
+        notifySlowTask(scheduler, attempt1);
+
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(2);
+
+        final ExecutionAttemptID attemptId1 = attempt1.getAttemptId();
+        final ExecutionAttemptID attemptId2 = getExecution(ev, 1).getAttemptId();
+
+        scheduler.updateTaskExecutionState(createFailedTaskExecutionState(attemptId1));
+        scheduler.updateTaskExecutionState(createFailedTaskExecutionState(attemptId2));
+        taskRestartExecutor.triggerScheduledTasks();
+
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(3);
+    }
+
+    @Test
+    void testNoRestartIfNotAllSpeculativeExecutionFailed() {
+        final SpeculativeScheduler scheduler = createSchedulerAndStartScheduling();
+        final ExecutionVertex ev = getOnlyExecutionVertex(scheduler);
+        final Execution attempt1 = ev.getCurrentExecutionAttempt();
+
+        notifySlowTask(scheduler, attempt1);
+        scheduler.updateTaskExecutionState(createFailedTaskExecutionState(attempt1.getAttemptId()));
+        taskRestartExecutor.triggerScheduledTasks();
+
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(2);
+    }
+
+    @Test
+    void testRestartVertexIfPartitionExceptionHappened() {
+        final SpeculativeScheduler scheduler = createSchedulerAndStartScheduling();
+        final ExecutionVertex ev = getOnlyExecutionVertex(scheduler);
+        final Execution attempt1 = ev.getCurrentExecutionAttempt();
+
+        notifySlowTask(scheduler, attempt1);
+        final Execution attempt2 = getExecution(ev, 1);
+        scheduler.updateTaskExecutionState(
+                new TaskExecutionState(
+                        attempt1.getAttemptId(),
+                        ExecutionState.FAILED,
+                        new PartitionNotFoundException(new ResultPartitionID())));
+
+        assertThat(attempt2.getState()).isEqualTo(CANCELING);
+
+        completeCancellingForAllVertices(scheduler.getExecutionGraph());
+        taskRestartExecutor.triggerScheduledTasks();
+
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(3);
+    }
+
+    @Test
+    void testCancelOtherCurrentExecutionsWhenAnyExecutionFinished() {
+        final SpeculativeScheduler scheduler = createSchedulerAndStartScheduling();
+        final ExecutionVertex ev = getOnlyExecutionVertex(scheduler);
+        final Execution attempt1 = ev.getCurrentExecutionAttempt();
+
+        notifySlowTask(scheduler, attempt1);
+        final Execution attempt2 = getExecution(ev, 1);
+        scheduler.updateTaskExecutionState(
+                new TaskExecutionState(attempt1.getAttemptId(), ExecutionState.FINISHED));
+
+        assertThat(attempt2.getState()).isEqualTo(CANCELING);
+    }
+
+    @Test
+    void testExceptionHistoryIfPartitionExceptionHappened() {
+        final SpeculativeScheduler scheduler = createSchedulerAndStartScheduling();
+        final ExecutionVertex ev = getOnlyExecutionVertex(scheduler);
+        final Execution attempt1 = ev.getCurrentExecutionAttempt();
+
+        notifySlowTask(scheduler, attempt1);
+
+        // A partition exception can result in a restart of the whole execution vertex.
+        scheduler.updateTaskExecutionState(
+                new TaskExecutionState(
+                        attempt1.getAttemptId(),
+                        ExecutionState.FAILED,
+                        new PartitionNotFoundException(new ResultPartitionID())));
+
+        completeCancellingForAllVertices(scheduler.getExecutionGraph());
+        taskRestartExecutor.triggerScheduledTasks();
+
+        assertThat(scheduler.getExceptionHistory()).hasSize(1);
+
+        final RootExceptionHistoryEntry entry = scheduler.getExceptionHistory().iterator().next();
+        // the current execution attempt before the restarting should be attempt2 but the failure
+        // root exception should be attempt1
+        assertThat(entry.getFailingTaskName()).isEqualTo(attempt1.getVertexWithAttempt());
+    }
+
+    @Test
+    void testLocalExecutionAttemptFailureIsCorrectlyRecorded() {
+        final SpeculativeScheduler scheduler = createSchedulerAndStartScheduling();
+        final ExecutionVertex ev = getOnlyExecutionVertex(scheduler);
+        final Execution attempt1 = ev.getCurrentExecutionAttempt();
+
+        notifySlowTask(scheduler, attempt1);
+
+        // the execution vertex will not be restarted if we only fails attempt1, but it still should
+        // be recorded in the execution graph and in exception history
+        final TaskExecutionState failedState =
+                createFailedTaskExecutionState(attempt1.getAttemptId());
+        scheduler.updateTaskExecutionState(failedState);
+
+        final ClassLoader classLoader = SpeculativeSchedulerTest.class.getClassLoader();
+        assertThat(scheduler.getExecutionGraph().getFailureInfo()).isNotNull();
+        assertThat(scheduler.getExecutionGraph().getFailureInfo().getExceptionAsString())
+                .contains(failedState.getError(classLoader).getMessage());
+
+        assertThat(scheduler.getExceptionHistory()).hasSize(1);
+
+        final RootExceptionHistoryEntry entry = scheduler.getExceptionHistory().iterator().next();
+        assertThat(entry.getFailingTaskName()).isEqualTo(attempt1.getVertexWithAttempt());
+    }
+
+    @Test
+    public void testSpeculativeExecutionCombinedWithAdaptiveScheduling() throws Exception {
+        final JobVertex source = createNoOpVertex("source", 1);
+        final JobVertex sink = createNoOpVertex("sink", -1);
+        sink.connectNewDataSetAsInput(
+                source, DistributionPattern.ALL_TO_ALL, ResultPartitionType.BLOCKING);
+        final JobGraph jobGraph = JobGraphTestUtils.batchJobGraph(source, sink);
+
+        final ComponentMainThreadExecutor mainThreadExecutor =
+                ComponentMainThreadExecutorServiceAdapter.forMainThread();
+        final SpeculativeScheduler scheduler =
+                createSchedulerBuilder(jobGraph, mainThreadExecutor)
+                        .setVertexParallelismDecider(ignore -> 3)
+                        .buildSpeculativeScheduler();
+        mainThreadExecutor.execute(scheduler::startScheduling);
+
+        final DefaultExecutionGraph graph = (DefaultExecutionGraph) scheduler.getExecutionGraph();
+        final ExecutionJobVertex sourceExecutionJobVertex = graph.getJobVertex(source.getID());
+        final ExecutionJobVertex sinkExecutionJobVertex = graph.getJobVertex(sink.getID());
+
+        final ExecutionVertex sourceExecutionVertex = sourceExecutionJobVertex.getTaskVertices()[0];
+        assertThat(sourceExecutionVertex.getCurrentExecutions()).hasSize(1);
+
+        // trigger source vertex speculation
+        final Execution sourceAttempt1 = sourceExecutionVertex.getCurrentExecutionAttempt();
+        notifySlowTask(scheduler, sourceAttempt1);
+        assertThat(sourceExecutionVertex.getCurrentExecutions()).hasSize(2);
+
+        assertThat(sinkExecutionJobVertex.getParallelism()).isEqualTo(-1);
+
+        // Finishing any source execution attempt will finish the source execution vertex, and then
+        // finish the job vertex.
+        scheduler.updateTaskExecutionState(
+                new TaskExecutionState(
+                        sourceAttempt1.getAttemptId(),
+                        ExecutionState.FINISHED,
+                        null,
+                        null,
+                        new IOMetrics(0, 0, 0, 0, 0, 0, 0)));
+        assertThat(sinkExecutionJobVertex.getParallelism()).isEqualTo(3);
+
+        // trigger sink vertex speculation
+        final ExecutionVertex sinkExecutionVertex = sinkExecutionJobVertex.getTaskVertices()[0];
+        final Execution sinkAttempt1 = sinkExecutionVertex.getCurrentExecutionAttempt();
+        notifySlowTask(scheduler, sinkAttempt1);
+        assertThat(sinkExecutionVertex.getCurrentExecutions()).hasSize(2);
+    }
+
+    private static Execution getExecution(ExecutionVertex executionVertex, int attemptNumber) {
+        return executionVertex.getCurrentExecutions().stream()
+                .filter(e -> e.getAttemptNumber() == attemptNumber)
+                .findFirst()
+                .get();
+    }
+
+    private static ExecutionVertex getOnlyExecutionVertex(SpeculativeScheduler scheduler) {
+        return Iterables.getOnlyElement(scheduler.getExecutionGraph().getAllExecutionVertices());
+    }
+
+    private SpeculativeScheduler createSchedulerAndStartScheduling() {
+        return createSchedulerAndStartScheduling(singleNonParallelJobVertexJobGraph());
+    }
+
+    private SpeculativeScheduler createSchedulerAndStartScheduling(final JobGraph jobGraph) {
+        final ComponentMainThreadExecutor mainThreadExecutor =
+                ComponentMainThreadExecutorServiceAdapter.forMainThread();
+
+        try {
+            final SpeculativeScheduler scheduler = createScheduler(jobGraph, mainThreadExecutor);
+            mainThreadExecutor.execute(scheduler::startScheduling);
+            return scheduler;
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private SpeculativeScheduler createScheduler(
+            final JobGraph jobGraph, final ComponentMainThreadExecutor mainThreadExecutor)
+            throws Exception {
+        return createSchedulerBuilder(jobGraph, mainThreadExecutor).buildSpeculativeScheduler();
+    }
+
+    private DefaultSchedulerBuilder createSchedulerBuilder(
+            final JobGraph jobGraph, final ComponentMainThreadExecutor mainThreadExecutor) {
+        return new DefaultSchedulerBuilder(
+                        jobGraph, mainThreadExecutor, EXECUTOR_RESOURCE.getExecutor())
+                .setBlocklistHandler(testBlocklistHandler)
+                .setExecutionOperations(testExecutionOperations)
+                .setFutureExecutor(futureExecutor)
+                .setDelayExecutor(taskRestartExecutor)
+                .setRestartBackoffTimeStrategy(new TestRestartBackoffTimeStrategy(true, 0));
+    }
+
+    private static void notifySlowTask(
+            final SpeculativeScheduler scheduler, final Execution slowTask) {
+        scheduler.notifySlowTasks(
+                ImmutableMap.of(
+                        slowTask.getVertex().getID(),
+                        Collections.singleton(slowTask.getAttemptId())));
+    }
+
+    private static class TestBlocklistHandler implements BlocklistHandler {
+        private List<BlockedNode> blockedNodes = new ArrayList<>();

Review Comment:
   can be final



##########
flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailureHandlingResultTest.java:
##########
@@ -18,73 +18,81 @@
 
 package org.apache.flink.runtime.executiongraph.failover.flip1;
 
-import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
 import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
-import org.apache.flink.util.TestLogger;
+import org.apache.flink.testutils.TestingUtils;
+import org.apache.flink.testutils.executor.TestExecutorExtension;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
 
 import java.util.HashSet;
 import java.util.Set;
+import java.util.concurrent.ScheduledExecutorService;
 
-import static org.hamcrest.CoreMatchers.is;
-import static org.hamcrest.core.IsSame.sameInstance;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.createExecutionGraph;
+import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.createNoOpVertex;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
 /** Tests for {@link FailureHandlingResult}. */
-public class FailureHandlingResultTest extends TestLogger {
+class FailureHandlingResultTest {
+
+    @RegisterExtension
+    private static final TestExecutorExtension<ScheduledExecutorService> EXECUTOR_RESOURCE =
+            TestingUtils.defaultExecutorExtension();
 
     /** Tests normal FailureHandlingResult. */
     @Test
-    public void testNormalFailureHandlingResult() {
+    void testNormalFailureHandlingResult() throws Exception {
+
         // create a normal FailureHandlingResult
-        ExecutionVertexID executionVertexID = new ExecutionVertexID(new JobVertexID(), 0);
+        Execution execution = createExecution(EXECUTOR_RESOURCE.getExecutor());
+
         Set<ExecutionVertexID> tasks = new HashSet<>();
-        tasks.add(executionVertexID);
+        tasks.add(execution.getVertex().getID());
+
         long delay = 1234;
         Throwable error = new RuntimeException();
         long timestamp = System.currentTimeMillis();
         FailureHandlingResult result =
-                FailureHandlingResult.restartable(
-                        executionVertexID, error, timestamp, tasks, delay, false);
-
-        assertTrue(result.canRestart());
-        assertEquals(delay, result.getRestartDelayMS());
-        assertEquals(tasks, result.getVerticesToRestart());
-        assertThat(result.getError(), sameInstance(error));
-        assertThat(result.getTimestamp(), is(timestamp));
-        assertTrue(result.getExecutionVertexIdOfFailedTask().isPresent());
-        assertThat(result.getExecutionVertexIdOfFailedTask().get(), is(executionVertexID));
+                FailureHandlingResult.restartable(execution, error, timestamp, tasks, delay, false);
+
+        assertThat(result.canRestart()).isTrue();
+        assertThat(delay).isEqualTo(result.getRestartDelayMS());
+        assertThat(tasks).isEqualTo(result.getVerticesToRestart());
+        assertThat(result.getError()).isSameAs(error);
+        assertThat(result.getTimestamp()).isEqualTo(timestamp);
+        assertThat(result.getFailedExecution().isPresent()).isTrue();

Review Comment:
   ```suggestion
           assertThat(result.getFailedExecution()).isPresent();
   ```
   Other assertions of optional should also be changed like this
   



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org