zhuzhurk commented on code in PR #24771:
URL: https://github.com/apache/flink/pull/24771#discussion_r1612989352


##########
flink-tests/src/test/java/org/apache/flink/test/scheduling/JMFailoverITCase.java:
##########
@@ -532,9 +508,9 @@ private JobGraph 
createJobGraphWithUnsupportedBatchSnapshotOperatorCoordinator(
         return StreamingJobGraphGenerator.createJobGraph(streamGraph);
     }
 
-    private static void fillKeepGoing(
-            List<Integer> indices, boolean going, Map<Integer, Boolean> 
keepGoing) {
-        indices.forEach(index -> keepGoing.put(index, going));
+    private static void fillBlockSubTasks(

Review Comment:
   fillBlockSubTasks -> setSubtaskBlocked



##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptivebatch/BatchJobRecoveryTest.java:
##########
@@ -0,0 +1,1173 @@
+/*
+ * 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.JobID;
+import org.apache.flink.api.common.eventtime.WatermarkAlignmentParams;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.mocks.MockSource;
+import org.apache.flink.api.connector.source.mocks.MockSourceSplit;
+import org.apache.flink.api.connector.source.mocks.MockSplitEnumerator;
+import org.apache.flink.configuration.BatchExecutionOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+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.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.IntermediateResultPartition;
+import org.apache.flink.runtime.executiongraph.InternalExecutionGraphAccessor;
+import org.apache.flink.runtime.executiongraph.ResultPartitionBytes;
+import 
org.apache.flink.runtime.executiongraph.TestingComponentMainThreadExecutor;
+import 
org.apache.flink.runtime.executiongraph.failover.FixedDelayRestartBackoffTimeStrategy;
+import org.apache.flink.runtime.io.network.partition.JobMasterPartitionTracker;
+import 
org.apache.flink.runtime.io.network.partition.JobMasterPartitionTrackerImpl;
+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.IntermediateResultPartitionID;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.jobmaster.event.ExecutionVertexFinishedEvent;
+import org.apache.flink.runtime.jobmaster.event.FileSystemJobEventStore;
+import org.apache.flink.runtime.jobmaster.event.JobEvent;
+import org.apache.flink.runtime.jobmaster.event.JobEventManager;
+import org.apache.flink.runtime.jobmaster.event.JobEventStore;
+import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGateway;
+import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGatewayBuilder;
+import org.apache.flink.runtime.operators.coordination.EventReceivingTasks;
+import 
org.apache.flink.runtime.operators.coordination.OperatorCoordinatorHolder;
+import 
org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator;
+import 
org.apache.flink.runtime.operators.coordination.TestingOperatorCoordinator;
+import org.apache.flink.runtime.scheduler.DefaultSchedulerBuilder;
+import org.apache.flink.runtime.scheduler.SchedulerBase;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.scheduler.strategy.SchedulingResultPartition;
+import org.apache.flink.runtime.scheduler.strategy.SchedulingTopology;
+import org.apache.flink.runtime.shuffle.DefaultShuffleMetrics;
+import org.apache.flink.runtime.shuffle.JobShuffleContextImpl;
+import org.apache.flink.runtime.shuffle.NettyShuffleDescriptor;
+import org.apache.flink.runtime.shuffle.NettyShuffleMaster;
+import org.apache.flink.runtime.shuffle.PartitionWithMetrics;
+import org.apache.flink.runtime.shuffle.ShuffleDescriptor;
+import org.apache.flink.runtime.shuffle.ShuffleMaster;
+import org.apache.flink.runtime.shuffle.ShuffleMetrics;
+import org.apache.flink.runtime.source.coordinator.SourceCoordinator;
+import org.apache.flink.runtime.source.coordinator.SourceCoordinatorProvider;
+import org.apache.flink.runtime.source.event.ReaderRegistrationEvent;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.runtime.testutils.CommonTestUtils;
+import org.apache.flink.testutils.TestingUtils;
+import org.apache.flink.testutils.executor.TestExecutorExtension;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameter;
+import 
org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameters;
+import org.apache.flink.testutils.junit.utils.TempDirUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.SerializedValue;
+import org.apache.flink.util.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.util.concurrent.ScheduledExecutor;
+import org.apache.flink.util.concurrent.ScheduledExecutorServiceAdapter;
+import org.apache.flink.util.function.SupplierWithException;
+import org.apache.flink.util.function.ThrowingRunnable;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.junit.jupiter.api.io.TempDir;
+
+import javax.annotation.Nonnull;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+import static 
org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.waitUntilExecutionVertexState;
+import static 
org.apache.flink.runtime.scheduler.DefaultSchedulerBuilder.createCustomParallelismDecider;
+import static org.apache.flink.util.Preconditions.checkState;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.fail;
+
+/** Test for batch job recovery. */
+@ExtendWith(ParameterizedTestExtension.class)
+public class BatchJobRecoveryTest {
+
+    private final Duration previousWorkerRecoveryTimeout = 
Duration.ofSeconds(1);
+
+    @TempDir private java.nio.file.Path temporaryFolder;
+
+    // ---- Mocks for the underlying Operator Coordinator Context ---
+    protected EventReceivingTasks receivingTasks;
+
+    @RegisterExtension
+    static final TestExecutorExtension<ScheduledExecutorService> 
EXECUTOR_RESOURCE =
+            TestingUtils.defaultExecutorExtension();
+
+    @RegisterExtension
+    static final TestingComponentMainThreadExecutor.Extension 
MAIN_EXECUTOR_RESOURCE =
+            new TestingComponentMainThreadExecutor.Extension();
+
+    private final TestingComponentMainThreadExecutor mainThreadExecutor =
+            MAIN_EXECUTOR_RESOURCE.getComponentMainThreadTestExecutor();
+
+    private ScheduledExecutor delayedExecutor =
+            new 
ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor());
+
+    private static final OperatorID OPERATOR_ID = new OperatorID(1234L, 5678L);
+    private static final int NUM_SPLITS = 10;
+    private static final int SOURCE_PARALLELISM = 5;
+    private static final int MIDDLE_PARALLELISM = 5;
+    private static final int DECIDED_SINK_PARALLELISM = 2;
+    private static final JobVertexID SOURCE_ID = new JobVertexID();
+    private static final JobVertexID MIDDLE_ID = new JobVertexID();
+    private static final JobVertexID SINK_ID = new JobVertexID();
+    private static final JobID JOB_ID = new JobID();
+
+    private SourceCoordinatorProvider<MockSourceSplit> provider;
+    private FileSystemJobEventStore jobEventStore;
+    private List<JobEvent> persistedJobEventList;
+
+    private byte[] serializedJobGraph;
+
+    private final Collection<PartitionWithMetrics> allPartitionWithMetrics = 
new ArrayList<>();
+
+    @Parameter public boolean enableSpeculativeExecution;
+
+    @Parameters(name = "enableSpeculativeExecution={0}")
+    public static Collection<Boolean> parameters() {
+        return Arrays.asList(false, true);
+    }
+
+    @BeforeEach
+    void setUp() throws IOException {
+        final Path rootPath = new 
Path(TempDirUtils.newFolder(temporaryFolder).getAbsolutePath());
+        delayedExecutor = new 
ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor());
+        receivingTasks = EventReceivingTasks.createForRunningTasks();
+        persistedJobEventList = new ArrayList<>();
+        jobEventStore =
+                new TestingFileSystemJobEventStore(
+                        rootPath, new Configuration(), persistedJobEventList);
+
+        provider =
+                new SourceCoordinatorProvider<>(
+                        "AdaptiveBatchSchedulerTest",
+                        OPERATOR_ID,
+                        new MockSource(Boundedness.BOUNDED, NUM_SPLITS),
+                        1,
+                        WatermarkAlignmentParams.WATERMARK_ALIGNMENT_DISABLED,
+                        null);
+
+        this.serializedJobGraph = serializeJobGraph(createDefaultJobGraph());
+        allPartitionWithMetrics.clear();
+    }
+
+    @AfterEach
+    void after() {
+        jobEventStore.stop(true);
+    }
+
+    // This case will use job graph with the following topology:
+    // Source (p=5) -- POINTWISE --> Middle (p=5) -- ALLTOALL --> Sink (p=2)
+    //
+    // This case will experience the following stages:
+    // 1. All source tasks are finished and all middle tasks are running
+    // 2. JM failover
+    // 3. After the failover, all source tasks are expected to be recovered to
+    //  finished, and their produced partitions should also be restored. And 
middle vertex is
+    // redeployed.
+    @TestTemplate
+    void testRecoverFromJMFailover() throws Exception {
+        AdaptiveBatchScheduler scheduler = 
createScheduler(deserializeJobGraph(serializedJobGraph));
+
+        // start scheduling.
+        runInMainThread(scheduler::startScheduling);
+
+        runInMainThread(
+                () -> {
+                    // trigger all source finished
+                    transitionExecutionsState(scheduler, 
ExecutionState.FINISHED, SOURCE_ID);
+                });
+        runInMainThread(
+                () -> {
+                    // trigger all middle tasks to RUNNING state
+                    transitionExecutionsState(scheduler, 
ExecutionState.INITIALIZING, MIDDLE_ID);
+                    transitionExecutionsState(scheduler, 
ExecutionState.RUNNING, MIDDLE_ID);
+                });
+        List<ExecutionAttemptID> sourceExecutions =
+                
getCurrentAttemptIds(scheduler.getExecutionJobVertex(SOURCE_ID));
+        List<ExecutionAttemptID> middleExecutions =
+                
getCurrentAttemptIds(scheduler.getExecutionJobVertex(MIDDLE_ID));
+
+        Map<IntermediateResultPartitionID, Integer> subpartitionNums = new 
HashMap<>();
+        for (ExecutionVertex vertex :
+                getExecutionVertices(SOURCE_ID, 
scheduler.getExecutionGraph())) {
+
+            IntermediateResultPartition partition =
+                    vertex.getProducedPartitions().values().iterator().next();
+            subpartitionNums.put(partition.getPartitionId(), 
partition.getNumberOfSubpartitions());
+        }
+
+        waitWriteJobFinishedEventCompleted(5);
+        runInMainThread(
+                () -> {
+                    // flush events.
+                    jobEventStore.stop(false);
+                });
+
+        // register partitions
+        registerPartitions(scheduler);
+
+        // start a new scheduler and try to recover.
+        AdaptiveBatchScheduler newScheduler =
+                createScheduler(deserializeJobGraph(serializedJobGraph));
+        startSchedulingAndWaitRecoverFinish(newScheduler);
+
+        // check source vertices state were recovered.
+        for (ExecutionVertex vertex :
+                getExecutionVertices(SOURCE_ID, 
newScheduler.getExecutionGraph())) {
+            // check state.
+            assertThat(sourceExecutions)
+                    
.contains(vertex.getCurrentExecutionAttempt().getAttemptId());
+            
assertThat(vertex.getExecutionState()).isEqualTo(ExecutionState.FINISHED);
+
+            // check partition tracker was rebuild.
+            JobMasterPartitionTracker partitionTracker =
+                    ((InternalExecutionGraphAccessor) 
newScheduler.getExecutionGraph())
+                            .getPartitionTracker();
+            List<ResultPartitionID> resultPartitionIds =
+                    vertex.getProducedPartitions().keySet().stream()
+                            .map(
+                                    ((DefaultExecutionGraph) 
newScheduler.getExecutionGraph())
+                                            ::createResultPartitionId)
+                            .collect(Collectors.toList());
+            for (ResultPartitionID partitionID : resultPartitionIds) {
+                
assertThat(partitionTracker.isPartitionTracked(partitionID)).isTrue();
+            }
+
+            // check partitions are recovered
+            IntermediateResultPartition partition =
+                    vertex.getProducedPartitions().values().iterator().next();
+            assertThat(partition.getNumberOfSubpartitions())
+                    
.isEqualTo(subpartitionNums.get(partition.getPartitionId()));
+        }
+
+        // check middle vertices state were not recovered.
+        for (ExecutionVertex vertex :
+                getExecutionVertices(MIDDLE_ID, 
newScheduler.getExecutionGraph())) {
+            assertThat(middleExecutions)
+                    
.doesNotContain(vertex.getCurrentExecutionAttempt().getAttemptId());
+            
assertThat(vertex.getExecutionState()).isEqualTo(ExecutionState.DEPLOYING);
+        }
+    }
+
+    // This case will use job graph with the following topology:
+    // Source (p=5) -- POINTWISE --> Middle (p=5) -- ALLTOALL --> Sink (p=2)
+    //
+    // This case will undergo the following stages:
+    // 1. All source tasks are finished, as well as middle task 0, while other 
middle tasks are
+    // still running.
+    // The middle vertex contains an operator coordinator that does not 
support batch snapshot.
+    // 2. The partition belonging to source task0 is released because middle 
task0 has finished.
+    // 3. The JobMaster undergoes a failover.
+    // 4. After failover, middle task0 and source task0 are expected be reset. 
Other source tasks
+    // should be restored to finished and their produced partitions should 
also be restored.
+    @TestTemplate
+    void testOpCoordUnsupportedBatchSnapshotWithJobVertexUnFinished() throws 
Exception {
+        JobGraph jobGraph = deserializeJobGraph(serializedJobGraph);
+        JobVertex jobVertex = jobGraph.findVertexByID(MIDDLE_ID);
+        jobVertex.addOperatorCoordinator(
+                new SerializedValue<>(
+                        new TestingOperatorCoordinator.Provider(
+                                
jobVertex.getOperatorIDs().get(0).getGeneratedOperatorID())));
+        AdaptiveBatchScheduler scheduler =
+                createScheduler(
+                        jobGraph,
+                        Duration.ZERO /* make sure every finished event can 
flush on time.*/);
+
+        // start scheduling.
+        runInMainThread(scheduler::startScheduling);
+
+        runInMainThread(
+                () -> {
+                    // trigger all source finished
+                    transitionExecutionsState(scheduler, 
ExecutionState.FINISHED, SOURCE_ID);
+                });
+        runInMainThread(
+                () -> {
+                    // trigger first middle finished.
+                    ExecutionVertex firstMiddle =
+                            getExecutionVertex(MIDDLE_ID, 0, 
scheduler.getExecutionGraph());
+                    AdaptiveBatchSchedulerTest.transitionExecutionsState(
+                            scheduler,
+                            ExecutionState.FINISHED,
+                            
Collections.singletonList(firstMiddle.getCurrentExecutionAttempt()),
+                            null);
+                });
+        List<ExecutionAttemptID> sourceExecutions =
+                
getCurrentAttemptIds(scheduler.getExecutionJobVertex(SOURCE_ID));
+        List<ExecutionAttemptID> middleExecutions =
+                
getCurrentAttemptIds(scheduler.getExecutionJobVertex(MIDDLE_ID));
+
+        waitWriteJobFinishedEventCompleted(6);
+        runInMainThread(
+                () -> {
+                    // flush events.
+                    jobEventStore.stop(false);
+                });
+
+        // register partitions, the partition of source task 0 is lost, and it 
will be restarted
+        // if middle task 0 need be restarted.
+        int subtaskIndex = 0;
+        registerPartitions(
+                scheduler,
+                Collections.emptySet(),
+                Collections.singleton(
+                        scheduler
+                                .getExecutionJobVertex(SOURCE_ID)
+                                .getTaskVertices()[subtaskIndex]
+                                .getID()));
+
+        // start a new scheduler and try to recover.
+        AdaptiveBatchScheduler newScheduler = createScheduler(jobGraph);
+        startSchedulingAndWaitRecoverFinish(newScheduler);
+
+        // check source vertices state were recovered.
+        for (ExecutionVertex vertex :
+                getExecutionVertices(SOURCE_ID, 
newScheduler.getExecutionGraph())) {
+            if (vertex.getParallelSubtaskIndex() == subtaskIndex) {
+                
assertThat(vertex.getExecutionState()).isEqualTo(ExecutionState.DEPLOYING);
+                continue;
+            }
+
+            // check state.
+            assertThat(sourceExecutions)
+                    
.contains(vertex.getCurrentExecutionAttempt().getAttemptId());
+            
assertThat(vertex.getExecutionState()).isEqualTo(ExecutionState.FINISHED);
+
+            // check partition tracker was rebuild.
+            JobMasterPartitionTracker partitionTracker =
+                    ((InternalExecutionGraphAccessor) 
newScheduler.getExecutionGraph())
+                            .getPartitionTracker();
+            List<ResultPartitionID> resultPartitionIds =
+                    vertex.getProducedPartitions().keySet().stream()
+                            .map(
+                                    ((DefaultExecutionGraph) 
newScheduler.getExecutionGraph())
+                                            ::createResultPartitionId)
+                            .collect(Collectors.toList());
+            for (ResultPartitionID partitionID : resultPartitionIds) {
+                
assertThat(partitionTracker.isPartitionTracked(partitionID)).isTrue();
+            }
+        }
+
+        // check middle vertices state were not recovered.
+        for (ExecutionVertex vertex :
+                getExecutionVertices(MIDDLE_ID, 
newScheduler.getExecutionGraph())) {
+            if (vertex.getParallelSubtaskIndex() == subtaskIndex) {
+                
assertThat(vertex.getExecutionState()).isEqualTo(ExecutionState.CREATED);
+                continue;
+            }
+
+            assertThat(middleExecutions)
+                    
.doesNotContain(vertex.getCurrentExecutionAttempt().getAttemptId());
+            
assertThat(vertex.getExecutionState()).isEqualTo(ExecutionState.DEPLOYING);
+        }
+    }
+
+    // This case will use job graph with the following topology:
+    // Source (p=5) -- POINTWISE --> Middle (p=5) -- ALLTOALL --> Sink (p=2)
+    //
+    // This case will undergo the following stages:
+    // 1. All source tasks are finished.
+    // The source vertex contains an operator coordinator that does not 
support batch snapshot.
+    // 2. The jobMaster undergoes a failover.
+    // 3. After the failover, all source tasks are expected to be recovered to 
finished, and their
+    // produced partitions should also be restored.
+    // 4. Trigger all middle task running
+    // 5. All source task should be restarted.
+    @TestTemplate
+    void 
testOpCoordUnsupportedBatchSnapshotWithJobVertexFinishedAndPartitionNotFound()

Review Comment:
   The name needs to be refined.



##########
flink-tests/src/test/java/org/apache/flink/test/scheduling/JMFailoverITCase.java:
##########
@@ -0,0 +1,815 @@
+/*
+ * 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.test.scheduling;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.api.common.eventtime.WatermarkStrategy;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.configuration.BatchExecutionOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.configuration.ExecutionOptions;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.configuration.RestOptions;
+import org.apache.flink.configuration.RestartStrategyOptions;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.connector.file.src.FileSource;
+import org.apache.flink.connector.file.src.reader.TextLineInputFormat;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.execution.Environment;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import 
org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedHaServicesWithLeadershipControl;
+import org.apache.flink.runtime.io.network.partition.PartitionedFile;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobType;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobmaster.JobResult;
+import org.apache.flink.runtime.minicluster.MiniCluster;
+import org.apache.flink.runtime.minicluster.TestingMiniCluster;
+import org.apache.flink.runtime.minicluster.TestingMiniClusterConfiguration;
+import org.apache.flink.runtime.testutils.CommonTestUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.graph.GlobalStreamExchangeMode;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.graph.StreamGraph;
+import org.apache.flink.streaming.api.graph.StreamingJobGraphGenerator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import org.apache.flink.testutils.TestingUtils;
+import org.apache.flink.testutils.executor.TestExecutorResource;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.NetUtils;
+import org.apache.flink.util.function.SupplierWithException;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.rules.TestName;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static org.apache.flink.util.Preconditions.checkState;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** ITCase for JM failover. */
+public class JMFailoverITCase {
+
+    // to speed up recovery
+    private final Duration previousWorkerRecoveryTimeout = 
Duration.ofSeconds(3);
+
+    @ClassRule
+    public static final TestExecutorResource<ScheduledExecutorService> 
EXECUTOR_RESOURCE =
+            TestingUtils.defaultExecutorResource();
+
+    private static final int DEFAULT_MAX_PARALLELISM = 4;
+    private static final int SOURCE_PARALLELISM = 8;
+
+    private static final int NUMBER_KEYS = 10000;
+    private static final int NUMBER_OF_EACH_KEY = 4;
+
+    private EmbeddedHaServicesWithLeadershipControl highAvailabilityServices;
+
+    @Rule public TestName name = new TestName();
+
+    @Rule public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+    protected int numTaskManagers = 4;
+
+    protected int numSlotsPerTaskManager = 4;
+
+    protected Configuration flinkConfiguration = new Configuration();
+
+    protected MiniCluster flinkCluster;
+
+    protected Supplier<HighAvailabilityServices> 
highAvailabilityServicesSupplier = null;
+
+    @Before
+    public void before() throws Exception {
+        flinkConfiguration = new Configuration();
+        SourceTail.clear();
+        StubMapFunction.clear();
+        StubRecordSink.clear();
+    }
+
+    @After
+    public void after() {
+        Throwable exception = null;
+
+        try {
+            if (flinkCluster != null) {
+                flinkCluster.close();
+            }
+        } catch (Throwable throwable) {
+            exception = throwable;
+        }
+
+        if (exception != null) {
+            ExceptionUtils.rethrow(exception);
+        }
+    }
+
+    public void setup() throws Exception {
+        SourceTail.clear();
+        StubMapFunction.clear();
+        StubRecordSink.clear();
+    }
+
+    @Test
+    public void testRecoverFromJMFailover() throws Exception {
+        JobGraph jobGraph = prepareEnvAndGetJobGraph();
+
+        // blocking all sink
+        StubRecordSink.blockSubTasks(0, 1, 2, 3);
+
+        JobID jobId = flinkCluster.submitJob(jobGraph).get().getJobID();
+
+        // wait until sink is running.
+        tryWaitUntilCondition(() -> StubRecordSink.attemptIds.size() > 0);
+
+        triggerJMFailover(jobId);
+
+        // going all sink.
+        StubRecordSink.unblockSubTasks(0, 1, 2, 3);
+
+        JobResult jobResult = flinkCluster.requestJobResult(jobId).get();
+        assertThat(jobResult.getSerializedThrowable()).isEmpty();
+
+        checkCountResults();
+    }
+
+    @Test
+    public void testSourceNotAllFinished() throws Exception {
+        JobGraph jobGraph = prepareEnvAndGetJobGraph();
+
+        // blocking source 0
+        SourceTail.blockSubTasks(0);
+
+        JobID jobId = flinkCluster.submitJob(jobGraph).get().getJobID();
+
+        // wait until source is running.
+        tryWaitUntilCondition(() -> SourceTail.attemptIds.size() == 
SOURCE_PARALLELISM);
+
+        JobVertex source = 
jobGraph.getVerticesSortedTopologicallyFromSources().get(0);
+        while (true) {
+            AccessExecutionGraph executionGraph = 
flinkCluster.getExecutionGraph(jobId).get();
+            long finishedTasks =
+                    
Arrays.stream(executionGraph.getJobVertex(source.getID()).getTaskVertices())
+                            .filter(task -> task.getExecutionState() == 
ExecutionState.FINISHED)
+                            .count();
+            if (finishedTasks == SOURCE_PARALLELISM - 1) {
+                break;
+            }
+
+            Thread.sleep(100L);
+        }
+
+        triggerJMFailover(jobId);
+
+        // going source 0.

Review Comment:
   this comment should be updated, as well as a few other similar comments.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptivebatch/BatchJobRecoveryTest.java:
##########
@@ -0,0 +1,1173 @@
+/*
+ * 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.JobID;
+import org.apache.flink.api.common.eventtime.WatermarkAlignmentParams;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.mocks.MockSource;
+import org.apache.flink.api.connector.source.mocks.MockSourceSplit;
+import org.apache.flink.api.connector.source.mocks.MockSplitEnumerator;
+import org.apache.flink.configuration.BatchExecutionOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+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.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.IntermediateResultPartition;
+import org.apache.flink.runtime.executiongraph.InternalExecutionGraphAccessor;
+import org.apache.flink.runtime.executiongraph.ResultPartitionBytes;
+import 
org.apache.flink.runtime.executiongraph.TestingComponentMainThreadExecutor;
+import 
org.apache.flink.runtime.executiongraph.failover.FixedDelayRestartBackoffTimeStrategy;
+import org.apache.flink.runtime.io.network.partition.JobMasterPartitionTracker;
+import 
org.apache.flink.runtime.io.network.partition.JobMasterPartitionTrackerImpl;
+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.IntermediateResultPartitionID;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.jobmaster.event.ExecutionVertexFinishedEvent;
+import org.apache.flink.runtime.jobmaster.event.FileSystemJobEventStore;
+import org.apache.flink.runtime.jobmaster.event.JobEvent;
+import org.apache.flink.runtime.jobmaster.event.JobEventManager;
+import org.apache.flink.runtime.jobmaster.event.JobEventStore;
+import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGateway;
+import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGatewayBuilder;
+import org.apache.flink.runtime.operators.coordination.EventReceivingTasks;
+import 
org.apache.flink.runtime.operators.coordination.OperatorCoordinatorHolder;
+import 
org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator;
+import 
org.apache.flink.runtime.operators.coordination.TestingOperatorCoordinator;
+import org.apache.flink.runtime.scheduler.DefaultSchedulerBuilder;
+import org.apache.flink.runtime.scheduler.SchedulerBase;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.scheduler.strategy.SchedulingResultPartition;
+import org.apache.flink.runtime.scheduler.strategy.SchedulingTopology;
+import org.apache.flink.runtime.shuffle.DefaultShuffleMetrics;
+import org.apache.flink.runtime.shuffle.JobShuffleContextImpl;
+import org.apache.flink.runtime.shuffle.NettyShuffleDescriptor;
+import org.apache.flink.runtime.shuffle.NettyShuffleMaster;
+import org.apache.flink.runtime.shuffle.PartitionWithMetrics;
+import org.apache.flink.runtime.shuffle.ShuffleDescriptor;
+import org.apache.flink.runtime.shuffle.ShuffleMaster;
+import org.apache.flink.runtime.shuffle.ShuffleMetrics;
+import org.apache.flink.runtime.source.coordinator.SourceCoordinator;
+import org.apache.flink.runtime.source.coordinator.SourceCoordinatorProvider;
+import org.apache.flink.runtime.source.event.ReaderRegistrationEvent;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.runtime.testutils.CommonTestUtils;
+import org.apache.flink.testutils.TestingUtils;
+import org.apache.flink.testutils.executor.TestExecutorExtension;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameter;
+import 
org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameters;
+import org.apache.flink.testutils.junit.utils.TempDirUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.SerializedValue;
+import org.apache.flink.util.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.util.concurrent.ScheduledExecutor;
+import org.apache.flink.util.concurrent.ScheduledExecutorServiceAdapter;
+import org.apache.flink.util.function.SupplierWithException;
+import org.apache.flink.util.function.ThrowingRunnable;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.junit.jupiter.api.io.TempDir;
+
+import javax.annotation.Nonnull;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+import static 
org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.waitUntilExecutionVertexState;
+import static 
org.apache.flink.runtime.scheduler.DefaultSchedulerBuilder.createCustomParallelismDecider;
+import static org.apache.flink.util.Preconditions.checkState;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.fail;
+
+/** Test for batch job recovery. */
+@ExtendWith(ParameterizedTestExtension.class)
+public class BatchJobRecoveryTest {
+
+    private final Duration previousWorkerRecoveryTimeout = 
Duration.ofSeconds(1);
+
+    @TempDir private java.nio.file.Path temporaryFolder;
+
+    // ---- Mocks for the underlying Operator Coordinator Context ---
+    protected EventReceivingTasks receivingTasks;
+
+    @RegisterExtension
+    static final TestExecutorExtension<ScheduledExecutorService> 
EXECUTOR_RESOURCE =
+            TestingUtils.defaultExecutorExtension();
+
+    @RegisterExtension
+    static final TestingComponentMainThreadExecutor.Extension 
MAIN_EXECUTOR_RESOURCE =
+            new TestingComponentMainThreadExecutor.Extension();
+
+    private final TestingComponentMainThreadExecutor mainThreadExecutor =
+            MAIN_EXECUTOR_RESOURCE.getComponentMainThreadTestExecutor();
+
+    private ScheduledExecutor delayedExecutor =
+            new 
ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor());
+
+    private static final OperatorID OPERATOR_ID = new OperatorID(1234L, 5678L);
+    private static final int NUM_SPLITS = 10;
+    private static final int SOURCE_PARALLELISM = 5;
+    private static final int MIDDLE_PARALLELISM = 5;
+    private static final int DECIDED_SINK_PARALLELISM = 2;
+    private static final JobVertexID SOURCE_ID = new JobVertexID();
+    private static final JobVertexID MIDDLE_ID = new JobVertexID();
+    private static final JobVertexID SINK_ID = new JobVertexID();
+    private static final JobID JOB_ID = new JobID();
+
+    private SourceCoordinatorProvider<MockSourceSplit> provider;
+    private FileSystemJobEventStore jobEventStore;
+    private List<JobEvent> persistedJobEventList;
+
+    private byte[] serializedJobGraph;
+
+    private final Collection<PartitionWithMetrics> allPartitionWithMetrics = 
new ArrayList<>();
+
+    @Parameter public boolean enableSpeculativeExecution;
+
+    @Parameters(name = "enableSpeculativeExecution={0}")
+    public static Collection<Boolean> parameters() {
+        return Arrays.asList(false, true);
+    }
+
+    @BeforeEach
+    void setUp() throws IOException {
+        final Path rootPath = new 
Path(TempDirUtils.newFolder(temporaryFolder).getAbsolutePath());
+        delayedExecutor = new 
ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor());
+        receivingTasks = EventReceivingTasks.createForRunningTasks();
+        persistedJobEventList = new ArrayList<>();
+        jobEventStore =
+                new TestingFileSystemJobEventStore(
+                        rootPath, new Configuration(), persistedJobEventList);
+
+        provider =
+                new SourceCoordinatorProvider<>(
+                        "AdaptiveBatchSchedulerTest",
+                        OPERATOR_ID,
+                        new MockSource(Boundedness.BOUNDED, NUM_SPLITS),
+                        1,
+                        WatermarkAlignmentParams.WATERMARK_ALIGNMENT_DISABLED,
+                        null);
+
+        this.serializedJobGraph = serializeJobGraph(createDefaultJobGraph());
+        allPartitionWithMetrics.clear();
+    }
+
+    @AfterEach
+    void after() {
+        jobEventStore.stop(true);
+    }
+
+    // This case will use job graph with the following topology:
+    // Source (p=5) -- POINTWISE --> Middle (p=5) -- ALLTOALL --> Sink (p=2)
+    //
+    // This case will experience the following stages:
+    // 1. All source tasks are finished and all middle tasks are running
+    // 2. JM failover
+    // 3. After the failover, all source tasks are expected to be recovered to
+    //  finished, and their produced partitions should also be restored. And 
middle vertex is
+    // redeployed.
+    @TestTemplate
+    void testRecoverFromJMFailover() throws Exception {
+        AdaptiveBatchScheduler scheduler = 
createScheduler(deserializeJobGraph(serializedJobGraph));
+
+        // start scheduling.
+        runInMainThread(scheduler::startScheduling);
+
+        runInMainThread(
+                () -> {
+                    // trigger all source finished
+                    transitionExecutionsState(scheduler, 
ExecutionState.FINISHED, SOURCE_ID);
+                });
+        runInMainThread(
+                () -> {
+                    // trigger all middle tasks to RUNNING state
+                    transitionExecutionsState(scheduler, 
ExecutionState.INITIALIZING, MIDDLE_ID);
+                    transitionExecutionsState(scheduler, 
ExecutionState.RUNNING, MIDDLE_ID);
+                });
+        List<ExecutionAttemptID> sourceExecutions =
+                
getCurrentAttemptIds(scheduler.getExecutionJobVertex(SOURCE_ID));
+        List<ExecutionAttemptID> middleExecutions =
+                
getCurrentAttemptIds(scheduler.getExecutionJobVertex(MIDDLE_ID));
+
+        Map<IntermediateResultPartitionID, Integer> subpartitionNums = new 
HashMap<>();
+        for (ExecutionVertex vertex :
+                getExecutionVertices(SOURCE_ID, 
scheduler.getExecutionGraph())) {
+
+            IntermediateResultPartition partition =
+                    vertex.getProducedPartitions().values().iterator().next();
+            subpartitionNums.put(partition.getPartitionId(), 
partition.getNumberOfSubpartitions());
+        }
+
+        waitWriteJobFinishedEventCompleted(5);
+        runInMainThread(
+                () -> {
+                    // flush events.
+                    jobEventStore.stop(false);
+                });
+
+        // register partitions
+        registerPartitions(scheduler);
+
+        // start a new scheduler and try to recover.
+        AdaptiveBatchScheduler newScheduler =
+                createScheduler(deserializeJobGraph(serializedJobGraph));
+        startSchedulingAndWaitRecoverFinish(newScheduler);
+
+        // check source vertices state were recovered.
+        for (ExecutionVertex vertex :
+                getExecutionVertices(SOURCE_ID, 
newScheduler.getExecutionGraph())) {
+            // check state.
+            assertThat(sourceExecutions)
+                    
.contains(vertex.getCurrentExecutionAttempt().getAttemptId());
+            
assertThat(vertex.getExecutionState()).isEqualTo(ExecutionState.FINISHED);
+
+            // check partition tracker was rebuild.
+            JobMasterPartitionTracker partitionTracker =
+                    ((InternalExecutionGraphAccessor) 
newScheduler.getExecutionGraph())
+                            .getPartitionTracker();
+            List<ResultPartitionID> resultPartitionIds =
+                    vertex.getProducedPartitions().keySet().stream()
+                            .map(
+                                    ((DefaultExecutionGraph) 
newScheduler.getExecutionGraph())
+                                            ::createResultPartitionId)
+                            .collect(Collectors.toList());
+            for (ResultPartitionID partitionID : resultPartitionIds) {
+                
assertThat(partitionTracker.isPartitionTracked(partitionID)).isTrue();
+            }
+
+            // check partitions are recovered
+            IntermediateResultPartition partition =
+                    vertex.getProducedPartitions().values().iterator().next();
+            assertThat(partition.getNumberOfSubpartitions())
+                    
.isEqualTo(subpartitionNums.get(partition.getPartitionId()));
+        }
+
+        // check middle vertices state were not recovered.
+        for (ExecutionVertex vertex :
+                getExecutionVertices(MIDDLE_ID, 
newScheduler.getExecutionGraph())) {
+            assertThat(middleExecutions)
+                    
.doesNotContain(vertex.getCurrentExecutionAttempt().getAttemptId());
+            
assertThat(vertex.getExecutionState()).isEqualTo(ExecutionState.DEPLOYING);
+        }
+    }
+
+    // This case will use job graph with the following topology:
+    // Source (p=5) -- POINTWISE --> Middle (p=5) -- ALLTOALL --> Sink (p=2)
+    //
+    // This case will undergo the following stages:
+    // 1. All source tasks are finished, as well as middle task 0, while other 
middle tasks are
+    // still running.
+    // The middle vertex contains an operator coordinator that does not 
support batch snapshot.
+    // 2. The partition belonging to source task0 is released because middle 
task0 has finished.
+    // 3. The JobMaster undergoes a failover.
+    // 4. After failover, middle task0 and source task0 are expected be reset. 
Other source tasks
+    // should be restored to finished and their produced partitions should 
also be restored.
+    @TestTemplate
+    void testOpCoordUnsupportedBatchSnapshotWithJobVertexUnFinished() throws 
Exception {
+        JobGraph jobGraph = deserializeJobGraph(serializedJobGraph);
+        JobVertex jobVertex = jobGraph.findVertexByID(MIDDLE_ID);
+        jobVertex.addOperatorCoordinator(
+                new SerializedValue<>(
+                        new TestingOperatorCoordinator.Provider(
+                                
jobVertex.getOperatorIDs().get(0).getGeneratedOperatorID())));
+        AdaptiveBatchScheduler scheduler =
+                createScheduler(
+                        jobGraph,
+                        Duration.ZERO /* make sure every finished event can 
flush on time.*/);
+
+        // start scheduling.
+        runInMainThread(scheduler::startScheduling);
+
+        runInMainThread(
+                () -> {
+                    // trigger all source finished
+                    transitionExecutionsState(scheduler, 
ExecutionState.FINISHED, SOURCE_ID);
+                });
+        runInMainThread(
+                () -> {
+                    // trigger first middle finished.
+                    ExecutionVertex firstMiddle =
+                            getExecutionVertex(MIDDLE_ID, 0, 
scheduler.getExecutionGraph());
+                    AdaptiveBatchSchedulerTest.transitionExecutionsState(
+                            scheduler,
+                            ExecutionState.FINISHED,
+                            
Collections.singletonList(firstMiddle.getCurrentExecutionAttempt()),
+                            null);
+                });
+        List<ExecutionAttemptID> sourceExecutions =
+                
getCurrentAttemptIds(scheduler.getExecutionJobVertex(SOURCE_ID));
+        List<ExecutionAttemptID> middleExecutions =
+                
getCurrentAttemptIds(scheduler.getExecutionJobVertex(MIDDLE_ID));
+
+        waitWriteJobFinishedEventCompleted(6);
+        runInMainThread(
+                () -> {
+                    // flush events.
+                    jobEventStore.stop(false);
+                });
+
+        // register partitions, the partition of source task 0 is lost, and it 
will be restarted
+        // if middle task 0 need be restarted.
+        int subtaskIndex = 0;
+        registerPartitions(
+                scheduler,
+                Collections.emptySet(),
+                Collections.singleton(
+                        scheduler
+                                .getExecutionJobVertex(SOURCE_ID)
+                                .getTaskVertices()[subtaskIndex]
+                                .getID()));
+
+        // start a new scheduler and try to recover.
+        AdaptiveBatchScheduler newScheduler = createScheduler(jobGraph);
+        startSchedulingAndWaitRecoverFinish(newScheduler);
+
+        // check source vertices state were recovered.
+        for (ExecutionVertex vertex :
+                getExecutionVertices(SOURCE_ID, 
newScheduler.getExecutionGraph())) {
+            if (vertex.getParallelSubtaskIndex() == subtaskIndex) {
+                
assertThat(vertex.getExecutionState()).isEqualTo(ExecutionState.DEPLOYING);
+                continue;
+            }
+
+            // check state.
+            assertThat(sourceExecutions)
+                    
.contains(vertex.getCurrentExecutionAttempt().getAttemptId());
+            
assertThat(vertex.getExecutionState()).isEqualTo(ExecutionState.FINISHED);
+
+            // check partition tracker was rebuild.
+            JobMasterPartitionTracker partitionTracker =
+                    ((InternalExecutionGraphAccessor) 
newScheduler.getExecutionGraph())
+                            .getPartitionTracker();
+            List<ResultPartitionID> resultPartitionIds =
+                    vertex.getProducedPartitions().keySet().stream()
+                            .map(
+                                    ((DefaultExecutionGraph) 
newScheduler.getExecutionGraph())
+                                            ::createResultPartitionId)
+                            .collect(Collectors.toList());
+            for (ResultPartitionID partitionID : resultPartitionIds) {
+                
assertThat(partitionTracker.isPartitionTracked(partitionID)).isTrue();
+            }
+        }
+
+        // check middle vertices state were not recovered.
+        for (ExecutionVertex vertex :
+                getExecutionVertices(MIDDLE_ID, 
newScheduler.getExecutionGraph())) {
+            if (vertex.getParallelSubtaskIndex() == subtaskIndex) {
+                
assertThat(vertex.getExecutionState()).isEqualTo(ExecutionState.CREATED);
+                continue;
+            }
+
+            assertThat(middleExecutions)
+                    
.doesNotContain(vertex.getCurrentExecutionAttempt().getAttemptId());
+            
assertThat(vertex.getExecutionState()).isEqualTo(ExecutionState.DEPLOYING);
+        }
+    }
+
+    // This case will use job graph with the following topology:
+    // Source (p=5) -- POINTWISE --> Middle (p=5) -- ALLTOALL --> Sink (p=2)
+    //
+    // This case will undergo the following stages:
+    // 1. All source tasks are finished.
+    // The source vertex contains an operator coordinator that does not 
support batch snapshot.
+    // 2. The jobMaster undergoes a failover.
+    // 3. After the failover, all source tasks are expected to be recovered to 
finished, and their
+    // produced partitions should also be restored.
+    // 4. Trigger all middle task running
+    // 5. All source task should be restarted.
+    @TestTemplate
+    void 
testOpCoordUnsupportedBatchSnapshotWithJobVertexFinishedAndPartitionNotFound()
+            throws Exception {
+        JobGraph jobGraph = deserializeJobGraph(serializedJobGraph);
+        JobVertex jobVertex = jobGraph.findVertexByID(SOURCE_ID);
+        jobVertex.addOperatorCoordinator(
+                new SerializedValue<>(
+                        new TestingOperatorCoordinator.Provider(
+                                
jobVertex.getOperatorIDs().get(0).getGeneratedOperatorID())));
+        AdaptiveBatchScheduler scheduler = createScheduler(jobGraph);
+
+        // start scheduling.
+        runInMainThread(scheduler::startScheduling);
+
+        runInMainThread(
+                () -> {
+                    // trigger all source finished
+                    transitionExecutionsState(scheduler, 
ExecutionState.FINISHED, SOURCE_ID);
+                });
+        List<ExecutionAttemptID> sourceExecutions =
+                
getCurrentAttemptIds(scheduler.getExecutionJobVertex(SOURCE_ID));
+
+        waitWriteJobFinishedEventCompleted(5);
+        runInMainThread(
+                () -> {
+                    // flush events.
+                    jobEventStore.stop(false);
+                });
+
+        // register partitions
+        registerPartitions(scheduler);
+
+        // start a new scheduler and try to recover.
+        AdaptiveBatchScheduler newScheduler = createScheduler(jobGraph);
+        startSchedulingAndWaitRecoverFinish(newScheduler);
+
+        // check source vertices state were recovered.
+        for (ExecutionVertex vertex :
+                getExecutionVertices(SOURCE_ID, 
newScheduler.getExecutionGraph())) {
+            // check state.
+            assertThat(sourceExecutions)
+                    
.contains(vertex.getCurrentExecutionAttempt().getAttemptId());
+            
assertThat(vertex.getExecutionState()).isEqualTo(ExecutionState.FINISHED);
+
+            // check partition tracker was rebuild.
+            JobMasterPartitionTracker partitionTracker =
+                    ((InternalExecutionGraphAccessor) 
newScheduler.getExecutionGraph())
+                            .getPartitionTracker();
+            List<ResultPartitionID> resultPartitionIds =
+                    vertex.getProducedPartitions().keySet().stream()
+                            .map(
+                                    ((DefaultExecutionGraph) 
newScheduler.getExecutionGraph())
+                                            ::createResultPartitionId)
+                            .collect(Collectors.toList());
+            for (ResultPartitionID partitionID : resultPartitionIds) {
+                
assertThat(partitionTracker.isPartitionTracked(partitionID)).isTrue();
+            }
+        }
+
+        runInMainThread(
+                () -> {
+                    // trigger all middle running
+                    transitionExecutionsState(scheduler, 
ExecutionState.RUNNING, SOURCE_ID);
+                });
+
+        // trigger partition not found
+        ExecutionVertex firstTask =
+                getExecutionVertex(MIDDLE_ID, 0, 
newScheduler.getExecutionGraph());
+        triggerFailedByDataConsumptionException(newScheduler, firstTask);
+
+        waitUntilExecutionVertexState(firstTask, ExecutionState.CREATED, 
5000L);
+
+        // verify all middle executionVertices were restarted
+        for (int i = 0; i < 5; i++) {
+            assertThat(
+                            getExecutionVertex(SOURCE_ID, i, 
newScheduler.getExecutionGraph())
+                                    .getExecutionState())
+                    .isNotEqualTo(ExecutionState.FINISHED);
+        }
+    }
+
+    // This case will use job graph with the following topology:
+    // Source (p=5) -- POINTWISE --> Middle (p=5) -- ALLTOALL --> Sink (p=2)
+    //
+    // This case will undergo the following stages:
+    // 1. All source tasks are finished. source task0 lose its partitions.
+    // 2. The jobMaster undergoes a failover.
+    // 3. After the failover, all source tasks are expected to be recovered to 
finished, and their
+    // produced partitions should also be restored.
+    // 4. Trigger all middle task running and task0 consumed partitions loss.
+    // 5. All source task should be restarted.
+    @TestTemplate
+    void testRecoverFromJMFailoverWithPartitionsUnavailable() throws Exception 
{
+        AdaptiveBatchScheduler scheduler = 
createScheduler(deserializeJobGraph(serializedJobGraph));
+
+        // start scheduling.
+        runInMainThread(scheduler::startScheduling);
+
+        runInMainThread(
+                () -> {
+                    // trigger all source finished
+                    transitionExecutionsState(scheduler, 
ExecutionState.FINISHED, SOURCE_ID);
+                });
+        List<ExecutionAttemptID> sourceExecutions =
+                
getCurrentAttemptIds(scheduler.getExecutionJobVertex(SOURCE_ID));
+
+        waitWriteJobFinishedEventCompleted(5);
+        runInMainThread(
+                () -> {
+                    // flush events.
+                    jobEventStore.stop(false);
+                });
+
+        // register partitions
+        registerPartitions(scheduler, Collections.singleton(SOURCE_ID), 
Collections.emptySet());
+
+        // start a new scheduler and try to recover.
+        AdaptiveBatchScheduler newScheduler =
+                createScheduler(deserializeJobGraph(serializedJobGraph));
+        startSchedulingAndWaitRecoverFinish(newScheduler);
+
+        // check middle vertices state were not recovered.
+        for (ExecutionVertex vertex :
+                getExecutionVertices(SOURCE_ID, 
newScheduler.getExecutionGraph())) {
+            assertThat(sourceExecutions)
+                    
.doesNotContain(vertex.getCurrentExecutionAttempt().getAttemptId());
+            
assertThat(vertex.getExecutionState()).isEqualTo(ExecutionState.DEPLOYING);
+        }
+    }
+
+    // This case will use job graph with the following topology:
+    // Source (p=5) -- POINTWISE --> Middle (p=5) -- ALLTOALL --> Sink (p=2)
+    @TestTemplate
+    void testRecoverDecidedParallelismFromDifferentJobGraphInstance() throws 
Exception {
+        testRecoverDecidedParallelism(() -> 
deserializeJobGraph(serializedJobGraph));
+    }
+
+    // This case will use job graph with the following topology:
+    // Source (p=5) -- POINTWISE --> Middle (p=5) -- ALLTOALL --> Sink (p=2)
+    @TestTemplate
+    void testRecoverDecidedParallelismFromTheSameJobGraphInstance() throws 
Exception {
+        JobGraph jobGraph = deserializeJobGraph(serializedJobGraph);
+        testRecoverDecidedParallelism(() -> jobGraph);
+    }
+
+    void testRecoverDecidedParallelism(SupplierWithException<JobGraph, 
Exception> jobGraphSupplier)

Review Comment:
   This method is only used by 
`testRecoverDecidedParallelismFromTheSameJobGraphInstance`.



##########
flink-tests/src/test/java/org/apache/flink/test/scheduling/JMFailoverITCase.java:
##########
@@ -98,7 +98,7 @@
 import java.util.stream.IntStream;
 
 import static org.apache.flink.util.Preconditions.checkState;
-import static org.junit.Assert.assertEquals;
+import static org.assertj.core.api.Assertions.assertThat;
 
 /** ITCase for JM failover. */
 public class JMFailoverITCase {

Review Comment:
   It is not the only choice to use `TestExecutorResource` I guess.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptivebatch/BatchJobRecoveryTest.java:
##########
@@ -0,0 +1,1173 @@
+/*
+ * 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.JobID;
+import org.apache.flink.api.common.eventtime.WatermarkAlignmentParams;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.mocks.MockSource;
+import org.apache.flink.api.connector.source.mocks.MockSourceSplit;
+import org.apache.flink.api.connector.source.mocks.MockSplitEnumerator;
+import org.apache.flink.configuration.BatchExecutionOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+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.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.IntermediateResultPartition;
+import org.apache.flink.runtime.executiongraph.InternalExecutionGraphAccessor;
+import org.apache.flink.runtime.executiongraph.ResultPartitionBytes;
+import 
org.apache.flink.runtime.executiongraph.TestingComponentMainThreadExecutor;
+import 
org.apache.flink.runtime.executiongraph.failover.FixedDelayRestartBackoffTimeStrategy;
+import org.apache.flink.runtime.io.network.partition.JobMasterPartitionTracker;
+import 
org.apache.flink.runtime.io.network.partition.JobMasterPartitionTrackerImpl;
+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.IntermediateResultPartitionID;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.jobmaster.event.ExecutionVertexFinishedEvent;
+import org.apache.flink.runtime.jobmaster.event.FileSystemJobEventStore;
+import org.apache.flink.runtime.jobmaster.event.JobEvent;
+import org.apache.flink.runtime.jobmaster.event.JobEventManager;
+import org.apache.flink.runtime.jobmaster.event.JobEventStore;
+import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGateway;
+import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGatewayBuilder;
+import org.apache.flink.runtime.operators.coordination.EventReceivingTasks;
+import 
org.apache.flink.runtime.operators.coordination.OperatorCoordinatorHolder;
+import 
org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator;
+import 
org.apache.flink.runtime.operators.coordination.TestingOperatorCoordinator;
+import org.apache.flink.runtime.scheduler.DefaultSchedulerBuilder;
+import org.apache.flink.runtime.scheduler.SchedulerBase;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.scheduler.strategy.SchedulingResultPartition;
+import org.apache.flink.runtime.scheduler.strategy.SchedulingTopology;
+import org.apache.flink.runtime.shuffle.DefaultShuffleMetrics;
+import org.apache.flink.runtime.shuffle.JobShuffleContextImpl;
+import org.apache.flink.runtime.shuffle.NettyShuffleDescriptor;
+import org.apache.flink.runtime.shuffle.NettyShuffleMaster;
+import org.apache.flink.runtime.shuffle.PartitionWithMetrics;
+import org.apache.flink.runtime.shuffle.ShuffleDescriptor;
+import org.apache.flink.runtime.shuffle.ShuffleMaster;
+import org.apache.flink.runtime.shuffle.ShuffleMetrics;
+import org.apache.flink.runtime.source.coordinator.SourceCoordinator;
+import org.apache.flink.runtime.source.coordinator.SourceCoordinatorProvider;
+import org.apache.flink.runtime.source.event.ReaderRegistrationEvent;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.runtime.testutils.CommonTestUtils;
+import org.apache.flink.testutils.TestingUtils;
+import org.apache.flink.testutils.executor.TestExecutorExtension;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameter;
+import 
org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameters;
+import org.apache.flink.testutils.junit.utils.TempDirUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.SerializedValue;
+import org.apache.flink.util.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.util.concurrent.ScheduledExecutor;
+import org.apache.flink.util.concurrent.ScheduledExecutorServiceAdapter;
+import org.apache.flink.util.function.SupplierWithException;
+import org.apache.flink.util.function.ThrowingRunnable;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.junit.jupiter.api.io.TempDir;
+
+import javax.annotation.Nonnull;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+import static 
org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.waitUntilExecutionVertexState;
+import static 
org.apache.flink.runtime.scheduler.DefaultSchedulerBuilder.createCustomParallelismDecider;
+import static org.apache.flink.util.Preconditions.checkState;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.fail;
+
+/** Test for batch job recovery. */
+@ExtendWith(ParameterizedTestExtension.class)
+public class BatchJobRecoveryTest {
+
+    private final Duration previousWorkerRecoveryTimeout = 
Duration.ofSeconds(1);
+
+    @TempDir private java.nio.file.Path temporaryFolder;
+
+    // ---- Mocks for the underlying Operator Coordinator Context ---
+    protected EventReceivingTasks receivingTasks;
+
+    @RegisterExtension
+    static final TestExecutorExtension<ScheduledExecutorService> 
EXECUTOR_RESOURCE =
+            TestingUtils.defaultExecutorExtension();
+
+    @RegisterExtension
+    static final TestingComponentMainThreadExecutor.Extension 
MAIN_EXECUTOR_RESOURCE =
+            new TestingComponentMainThreadExecutor.Extension();
+
+    private final TestingComponentMainThreadExecutor mainThreadExecutor =
+            MAIN_EXECUTOR_RESOURCE.getComponentMainThreadTestExecutor();
+
+    private ScheduledExecutor delayedExecutor =
+            new 
ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor());
+
+    private static final OperatorID OPERATOR_ID = new OperatorID(1234L, 5678L);
+    private static final int NUM_SPLITS = 10;
+    private static final int SOURCE_PARALLELISM = 5;
+    private static final int MIDDLE_PARALLELISM = 5;
+    private static final int DECIDED_SINK_PARALLELISM = 2;
+    private static final JobVertexID SOURCE_ID = new JobVertexID();
+    private static final JobVertexID MIDDLE_ID = new JobVertexID();
+    private static final JobVertexID SINK_ID = new JobVertexID();
+    private static final JobID JOB_ID = new JobID();
+
+    private SourceCoordinatorProvider<MockSourceSplit> provider;
+    private FileSystemJobEventStore jobEventStore;
+    private List<JobEvent> persistedJobEventList;
+
+    private byte[] serializedJobGraph;
+
+    private final Collection<PartitionWithMetrics> allPartitionWithMetrics = 
new ArrayList<>();
+
+    @Parameter public boolean enableSpeculativeExecution;
+
+    @Parameters(name = "enableSpeculativeExecution={0}")
+    public static Collection<Boolean> parameters() {
+        return Arrays.asList(false, true);
+    }
+
+    @BeforeEach
+    void setUp() throws IOException {
+        final Path rootPath = new 
Path(TempDirUtils.newFolder(temporaryFolder).getAbsolutePath());
+        delayedExecutor = new 
ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor());
+        receivingTasks = EventReceivingTasks.createForRunningTasks();
+        persistedJobEventList = new ArrayList<>();
+        jobEventStore =
+                new TestingFileSystemJobEventStore(
+                        rootPath, new Configuration(), persistedJobEventList);
+
+        provider =
+                new SourceCoordinatorProvider<>(
+                        "AdaptiveBatchSchedulerTest",
+                        OPERATOR_ID,
+                        new MockSource(Boundedness.BOUNDED, NUM_SPLITS),
+                        1,
+                        WatermarkAlignmentParams.WATERMARK_ALIGNMENT_DISABLED,
+                        null);
+
+        this.serializedJobGraph = serializeJobGraph(createDefaultJobGraph());
+        allPartitionWithMetrics.clear();
+    }
+
+    @AfterEach
+    void after() {
+        jobEventStore.stop(true);
+    }
+
+    // This case will use job graph with the following topology:
+    // Source (p=5) -- POINTWISE --> Middle (p=5) -- ALLTOALL --> Sink (p=2)
+    //
+    // This case will experience the following stages:
+    // 1. All source tasks are finished and all middle tasks are running
+    // 2. JM failover
+    // 3. After the failover, all source tasks are expected to be recovered to
+    //  finished, and their produced partitions should also be restored. And 
middle vertex is
+    // redeployed.
+    @TestTemplate
+    void testRecoverFromJMFailover() throws Exception {
+        AdaptiveBatchScheduler scheduler = 
createScheduler(deserializeJobGraph(serializedJobGraph));
+
+        // start scheduling.
+        runInMainThread(scheduler::startScheduling);
+
+        runInMainThread(
+                () -> {
+                    // trigger all source finished
+                    transitionExecutionsState(scheduler, 
ExecutionState.FINISHED, SOURCE_ID);
+                });
+        runInMainThread(
+                () -> {
+                    // trigger all middle tasks to RUNNING state
+                    transitionExecutionsState(scheduler, 
ExecutionState.INITIALIZING, MIDDLE_ID);
+                    transitionExecutionsState(scheduler, 
ExecutionState.RUNNING, MIDDLE_ID);
+                });
+        List<ExecutionAttemptID> sourceExecutions =
+                
getCurrentAttemptIds(scheduler.getExecutionJobVertex(SOURCE_ID));
+        List<ExecutionAttemptID> middleExecutions =
+                
getCurrentAttemptIds(scheduler.getExecutionJobVertex(MIDDLE_ID));
+
+        Map<IntermediateResultPartitionID, Integer> subpartitionNums = new 
HashMap<>();
+        for (ExecutionVertex vertex :
+                getExecutionVertices(SOURCE_ID, 
scheduler.getExecutionGraph())) {
+
+            IntermediateResultPartition partition =
+                    vertex.getProducedPartitions().values().iterator().next();
+            subpartitionNums.put(partition.getPartitionId(), 
partition.getNumberOfSubpartitions());
+        }
+
+        waitWriteJobFinishedEventCompleted(5);
+        runInMainThread(
+                () -> {
+                    // flush events.
+                    jobEventStore.stop(false);
+                });
+
+        // register partitions
+        registerPartitions(scheduler);
+
+        // start a new scheduler and try to recover.
+        AdaptiveBatchScheduler newScheduler =
+                createScheduler(deserializeJobGraph(serializedJobGraph));
+        startSchedulingAndWaitRecoverFinish(newScheduler);
+
+        // check source vertices state were recovered.
+        for (ExecutionVertex vertex :
+                getExecutionVertices(SOURCE_ID, 
newScheduler.getExecutionGraph())) {
+            // check state.
+            assertThat(sourceExecutions)
+                    
.contains(vertex.getCurrentExecutionAttempt().getAttemptId());
+            
assertThat(vertex.getExecutionState()).isEqualTo(ExecutionState.FINISHED);
+
+            // check partition tracker was rebuild.
+            JobMasterPartitionTracker partitionTracker =
+                    ((InternalExecutionGraphAccessor) 
newScheduler.getExecutionGraph())
+                            .getPartitionTracker();
+            List<ResultPartitionID> resultPartitionIds =
+                    vertex.getProducedPartitions().keySet().stream()
+                            .map(
+                                    ((DefaultExecutionGraph) 
newScheduler.getExecutionGraph())
+                                            ::createResultPartitionId)
+                            .collect(Collectors.toList());
+            for (ResultPartitionID partitionID : resultPartitionIds) {
+                
assertThat(partitionTracker.isPartitionTracked(partitionID)).isTrue();
+            }
+
+            // check partitions are recovered
+            IntermediateResultPartition partition =
+                    vertex.getProducedPartitions().values().iterator().next();
+            assertThat(partition.getNumberOfSubpartitions())
+                    
.isEqualTo(subpartitionNums.get(partition.getPartitionId()));
+        }
+
+        // check middle vertices state were not recovered.
+        for (ExecutionVertex vertex :
+                getExecutionVertices(MIDDLE_ID, 
newScheduler.getExecutionGraph())) {
+            assertThat(middleExecutions)
+                    
.doesNotContain(vertex.getCurrentExecutionAttempt().getAttemptId());
+            
assertThat(vertex.getExecutionState()).isEqualTo(ExecutionState.DEPLOYING);
+        }
+    }
+
+    // This case will use job graph with the following topology:
+    // Source (p=5) -- POINTWISE --> Middle (p=5) -- ALLTOALL --> Sink (p=2)
+    //
+    // This case will undergo the following stages:
+    // 1. All source tasks are finished, as well as middle task 0, while other 
middle tasks are
+    // still running.
+    // The middle vertex contains an operator coordinator that does not 
support batch snapshot.
+    // 2. The partition belonging to source task0 is released because middle 
task0 has finished.
+    // 3. The JobMaster undergoes a failover.
+    // 4. After failover, middle task0 and source task0 are expected be reset. 
Other source tasks
+    // should be restored to finished and their produced partitions should 
also be restored.
+    @TestTemplate
+    void testOpCoordUnsupportedBatchSnapshotWithJobVertexUnFinished() throws 
Exception {

Review Comment:
   The name needs to be refined.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptivebatch/BatchJobRecoveryTest.java:
##########
@@ -0,0 +1,1173 @@
+/*
+ * 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.JobID;
+import org.apache.flink.api.common.eventtime.WatermarkAlignmentParams;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.mocks.MockSource;
+import org.apache.flink.api.connector.source.mocks.MockSourceSplit;
+import org.apache.flink.api.connector.source.mocks.MockSplitEnumerator;
+import org.apache.flink.configuration.BatchExecutionOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+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.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.IntermediateResultPartition;
+import org.apache.flink.runtime.executiongraph.InternalExecutionGraphAccessor;
+import org.apache.flink.runtime.executiongraph.ResultPartitionBytes;
+import 
org.apache.flink.runtime.executiongraph.TestingComponentMainThreadExecutor;
+import 
org.apache.flink.runtime.executiongraph.failover.FixedDelayRestartBackoffTimeStrategy;
+import org.apache.flink.runtime.io.network.partition.JobMasterPartitionTracker;
+import 
org.apache.flink.runtime.io.network.partition.JobMasterPartitionTrackerImpl;
+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.IntermediateResultPartitionID;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.jobmaster.event.ExecutionVertexFinishedEvent;
+import org.apache.flink.runtime.jobmaster.event.FileSystemJobEventStore;
+import org.apache.flink.runtime.jobmaster.event.JobEvent;
+import org.apache.flink.runtime.jobmaster.event.JobEventManager;
+import org.apache.flink.runtime.jobmaster.event.JobEventStore;
+import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGateway;
+import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGatewayBuilder;
+import org.apache.flink.runtime.operators.coordination.EventReceivingTasks;
+import 
org.apache.flink.runtime.operators.coordination.OperatorCoordinatorHolder;
+import 
org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator;
+import 
org.apache.flink.runtime.operators.coordination.TestingOperatorCoordinator;
+import org.apache.flink.runtime.scheduler.DefaultSchedulerBuilder;
+import org.apache.flink.runtime.scheduler.SchedulerBase;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.scheduler.strategy.SchedulingResultPartition;
+import org.apache.flink.runtime.scheduler.strategy.SchedulingTopology;
+import org.apache.flink.runtime.shuffle.DefaultShuffleMetrics;
+import org.apache.flink.runtime.shuffle.JobShuffleContextImpl;
+import org.apache.flink.runtime.shuffle.NettyShuffleDescriptor;
+import org.apache.flink.runtime.shuffle.NettyShuffleMaster;
+import org.apache.flink.runtime.shuffle.PartitionWithMetrics;
+import org.apache.flink.runtime.shuffle.ShuffleDescriptor;
+import org.apache.flink.runtime.shuffle.ShuffleMaster;
+import org.apache.flink.runtime.shuffle.ShuffleMetrics;
+import org.apache.flink.runtime.source.coordinator.SourceCoordinator;
+import org.apache.flink.runtime.source.coordinator.SourceCoordinatorProvider;
+import org.apache.flink.runtime.source.event.ReaderRegistrationEvent;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.runtime.testutils.CommonTestUtils;
+import org.apache.flink.testutils.TestingUtils;
+import org.apache.flink.testutils.executor.TestExecutorExtension;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameter;
+import 
org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameters;
+import org.apache.flink.testutils.junit.utils.TempDirUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.SerializedValue;
+import org.apache.flink.util.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.util.concurrent.ScheduledExecutor;
+import org.apache.flink.util.concurrent.ScheduledExecutorServiceAdapter;
+import org.apache.flink.util.function.SupplierWithException;
+import org.apache.flink.util.function.ThrowingRunnable;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.junit.jupiter.api.io.TempDir;
+
+import javax.annotation.Nonnull;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+import static 
org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.waitUntilExecutionVertexState;
+import static 
org.apache.flink.runtime.scheduler.DefaultSchedulerBuilder.createCustomParallelismDecider;
+import static org.apache.flink.util.Preconditions.checkState;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.fail;
+
+/** Test for batch job recovery. */
+@ExtendWith(ParameterizedTestExtension.class)
+public class BatchJobRecoveryTest {
+
+    private final Duration previousWorkerRecoveryTimeout = 
Duration.ofSeconds(1);
+
+    @TempDir private java.nio.file.Path temporaryFolder;
+
+    // ---- Mocks for the underlying Operator Coordinator Context ---
+    protected EventReceivingTasks receivingTasks;
+
+    @RegisterExtension
+    static final TestExecutorExtension<ScheduledExecutorService> 
EXECUTOR_RESOURCE =
+            TestingUtils.defaultExecutorExtension();
+
+    @RegisterExtension
+    static final TestingComponentMainThreadExecutor.Extension 
MAIN_EXECUTOR_RESOURCE =
+            new TestingComponentMainThreadExecutor.Extension();
+
+    private final TestingComponentMainThreadExecutor mainThreadExecutor =
+            MAIN_EXECUTOR_RESOURCE.getComponentMainThreadTestExecutor();
+
+    private ScheduledExecutor delayedExecutor =
+            new 
ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor());
+
+    private static final OperatorID OPERATOR_ID = new OperatorID(1234L, 5678L);
+    private static final int NUM_SPLITS = 10;
+    private static final int SOURCE_PARALLELISM = 5;
+    private static final int MIDDLE_PARALLELISM = 5;
+    private static final int DECIDED_SINK_PARALLELISM = 2;
+    private static final JobVertexID SOURCE_ID = new JobVertexID();
+    private static final JobVertexID MIDDLE_ID = new JobVertexID();
+    private static final JobVertexID SINK_ID = new JobVertexID();
+    private static final JobID JOB_ID = new JobID();
+
+    private SourceCoordinatorProvider<MockSourceSplit> provider;
+    private FileSystemJobEventStore jobEventStore;
+    private List<JobEvent> persistedJobEventList;
+
+    private byte[] serializedJobGraph;
+
+    private final Collection<PartitionWithMetrics> allPartitionWithMetrics = 
new ArrayList<>();
+
+    @Parameter public boolean enableSpeculativeExecution;
+
+    @Parameters(name = "enableSpeculativeExecution={0}")
+    public static Collection<Boolean> parameters() {
+        return Arrays.asList(false, true);
+    }
+
+    @BeforeEach
+    void setUp() throws IOException {
+        final Path rootPath = new 
Path(TempDirUtils.newFolder(temporaryFolder).getAbsolutePath());
+        delayedExecutor = new 
ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor());
+        receivingTasks = EventReceivingTasks.createForRunningTasks();
+        persistedJobEventList = new ArrayList<>();
+        jobEventStore =
+                new TestingFileSystemJobEventStore(
+                        rootPath, new Configuration(), persistedJobEventList);
+
+        provider =
+                new SourceCoordinatorProvider<>(
+                        "AdaptiveBatchSchedulerTest",
+                        OPERATOR_ID,
+                        new MockSource(Boundedness.BOUNDED, NUM_SPLITS),
+                        1,
+                        WatermarkAlignmentParams.WATERMARK_ALIGNMENT_DISABLED,
+                        null);
+
+        this.serializedJobGraph = serializeJobGraph(createDefaultJobGraph());
+        allPartitionWithMetrics.clear();
+    }
+
+    @AfterEach
+    void after() {
+        jobEventStore.stop(true);
+    }
+
+    // This case will use job graph with the following topology:
+    // Source (p=5) -- POINTWISE --> Middle (p=5) -- ALLTOALL --> Sink (p=2)
+    //
+    // This case will experience the following stages:
+    // 1. All source tasks are finished and all middle tasks are running
+    // 2. JM failover
+    // 3. After the failover, all source tasks are expected to be recovered to
+    //  finished, and their produced partitions should also be restored. And 
middle vertex is
+    // redeployed.
+    @TestTemplate
+    void testRecoverFromJMFailover() throws Exception {
+        AdaptiveBatchScheduler scheduler = 
createScheduler(deserializeJobGraph(serializedJobGraph));
+
+        // start scheduling.
+        runInMainThread(scheduler::startScheduling);
+
+        runInMainThread(
+                () -> {
+                    // trigger all source finished
+                    transitionExecutionsState(scheduler, 
ExecutionState.FINISHED, SOURCE_ID);
+                });
+        runInMainThread(
+                () -> {
+                    // trigger all middle tasks to RUNNING state
+                    transitionExecutionsState(scheduler, 
ExecutionState.INITIALIZING, MIDDLE_ID);
+                    transitionExecutionsState(scheduler, 
ExecutionState.RUNNING, MIDDLE_ID);
+                });
+        List<ExecutionAttemptID> sourceExecutions =
+                
getCurrentAttemptIds(scheduler.getExecutionJobVertex(SOURCE_ID));
+        List<ExecutionAttemptID> middleExecutions =
+                
getCurrentAttemptIds(scheduler.getExecutionJobVertex(MIDDLE_ID));
+
+        Map<IntermediateResultPartitionID, Integer> subpartitionNums = new 
HashMap<>();
+        for (ExecutionVertex vertex :
+                getExecutionVertices(SOURCE_ID, 
scheduler.getExecutionGraph())) {
+
+            IntermediateResultPartition partition =
+                    vertex.getProducedPartitions().values().iterator().next();
+            subpartitionNums.put(partition.getPartitionId(), 
partition.getNumberOfSubpartitions());
+        }
+
+        waitWriteJobFinishedEventCompleted(5);
+        runInMainThread(
+                () -> {
+                    // flush events.
+                    jobEventStore.stop(false);
+                });
+
+        // register partitions
+        registerPartitions(scheduler);
+
+        // start a new scheduler and try to recover.
+        AdaptiveBatchScheduler newScheduler =
+                createScheduler(deserializeJobGraph(serializedJobGraph));
+        startSchedulingAndWaitRecoverFinish(newScheduler);
+
+        // check source vertices state were recovered.
+        for (ExecutionVertex vertex :
+                getExecutionVertices(SOURCE_ID, 
newScheduler.getExecutionGraph())) {
+            // check state.
+            assertThat(sourceExecutions)
+                    
.contains(vertex.getCurrentExecutionAttempt().getAttemptId());
+            
assertThat(vertex.getExecutionState()).isEqualTo(ExecutionState.FINISHED);
+
+            // check partition tracker was rebuild.
+            JobMasterPartitionTracker partitionTracker =
+                    ((InternalExecutionGraphAccessor) 
newScheduler.getExecutionGraph())
+                            .getPartitionTracker();
+            List<ResultPartitionID> resultPartitionIds =
+                    vertex.getProducedPartitions().keySet().stream()
+                            .map(
+                                    ((DefaultExecutionGraph) 
newScheduler.getExecutionGraph())
+                                            ::createResultPartitionId)
+                            .collect(Collectors.toList());
+            for (ResultPartitionID partitionID : resultPartitionIds) {
+                
assertThat(partitionTracker.isPartitionTracked(partitionID)).isTrue();
+            }
+
+            // check partitions are recovered
+            IntermediateResultPartition partition =
+                    vertex.getProducedPartitions().values().iterator().next();
+            assertThat(partition.getNumberOfSubpartitions())
+                    
.isEqualTo(subpartitionNums.get(partition.getPartitionId()));
+        }
+
+        // check middle vertices state were not recovered.
+        for (ExecutionVertex vertex :
+                getExecutionVertices(MIDDLE_ID, 
newScheduler.getExecutionGraph())) {
+            assertThat(middleExecutions)
+                    
.doesNotContain(vertex.getCurrentExecutionAttempt().getAttemptId());
+            
assertThat(vertex.getExecutionState()).isEqualTo(ExecutionState.DEPLOYING);
+        }
+    }
+
+    // This case will use job graph with the following topology:
+    // Source (p=5) -- POINTWISE --> Middle (p=5) -- ALLTOALL --> Sink (p=2)
+    //
+    // This case will undergo the following stages:
+    // 1. All source tasks are finished, as well as middle task 0, while other 
middle tasks are
+    // still running.
+    // The middle vertex contains an operator coordinator that does not 
support batch snapshot.
+    // 2. The partition belonging to source task0 is released because middle 
task0 has finished.
+    // 3. The JobMaster undergoes a failover.
+    // 4. After failover, middle task0 and source task0 are expected be reset. 
Other source tasks
+    // should be restored to finished and their produced partitions should 
also be restored.
+    @TestTemplate
+    void testOpCoordUnsupportedBatchSnapshotWithJobVertexUnFinished() throws 
Exception {
+        JobGraph jobGraph = deserializeJobGraph(serializedJobGraph);
+        JobVertex jobVertex = jobGraph.findVertexByID(MIDDLE_ID);
+        jobVertex.addOperatorCoordinator(
+                new SerializedValue<>(
+                        new TestingOperatorCoordinator.Provider(
+                                
jobVertex.getOperatorIDs().get(0).getGeneratedOperatorID())));
+        AdaptiveBatchScheduler scheduler =
+                createScheduler(
+                        jobGraph,
+                        Duration.ZERO /* make sure every finished event can 
flush on time.*/);
+
+        // start scheduling.
+        runInMainThread(scheduler::startScheduling);
+
+        runInMainThread(
+                () -> {
+                    // trigger all source finished
+                    transitionExecutionsState(scheduler, 
ExecutionState.FINISHED, SOURCE_ID);
+                });
+        runInMainThread(
+                () -> {
+                    // trigger first middle finished.
+                    ExecutionVertex firstMiddle =
+                            getExecutionVertex(MIDDLE_ID, 0, 
scheduler.getExecutionGraph());
+                    AdaptiveBatchSchedulerTest.transitionExecutionsState(
+                            scheduler,
+                            ExecutionState.FINISHED,
+                            
Collections.singletonList(firstMiddle.getCurrentExecutionAttempt()),
+                            null);
+                });
+        List<ExecutionAttemptID> sourceExecutions =
+                
getCurrentAttemptIds(scheduler.getExecutionJobVertex(SOURCE_ID));
+        List<ExecutionAttemptID> middleExecutions =
+                
getCurrentAttemptIds(scheduler.getExecutionJobVertex(MIDDLE_ID));
+
+        waitWriteJobFinishedEventCompleted(6);
+        runInMainThread(
+                () -> {
+                    // flush events.
+                    jobEventStore.stop(false);
+                });
+
+        // register partitions, the partition of source task 0 is lost, and it 
will be restarted
+        // if middle task 0 need be restarted.
+        int subtaskIndex = 0;
+        registerPartitions(
+                scheduler,
+                Collections.emptySet(),
+                Collections.singleton(
+                        scheduler
+                                .getExecutionJobVertex(SOURCE_ID)
+                                .getTaskVertices()[subtaskIndex]
+                                .getID()));
+
+        // start a new scheduler and try to recover.
+        AdaptiveBatchScheduler newScheduler = createScheduler(jobGraph);
+        startSchedulingAndWaitRecoverFinish(newScheduler);
+
+        // check source vertices state were recovered.
+        for (ExecutionVertex vertex :
+                getExecutionVertices(SOURCE_ID, 
newScheduler.getExecutionGraph())) {
+            if (vertex.getParallelSubtaskIndex() == subtaskIndex) {
+                
assertThat(vertex.getExecutionState()).isEqualTo(ExecutionState.DEPLOYING);
+                continue;
+            }
+
+            // check state.
+            assertThat(sourceExecutions)
+                    
.contains(vertex.getCurrentExecutionAttempt().getAttemptId());
+            
assertThat(vertex.getExecutionState()).isEqualTo(ExecutionState.FINISHED);
+
+            // check partition tracker was rebuild.
+            JobMasterPartitionTracker partitionTracker =
+                    ((InternalExecutionGraphAccessor) 
newScheduler.getExecutionGraph())
+                            .getPartitionTracker();
+            List<ResultPartitionID> resultPartitionIds =
+                    vertex.getProducedPartitions().keySet().stream()
+                            .map(
+                                    ((DefaultExecutionGraph) 
newScheduler.getExecutionGraph())
+                                            ::createResultPartitionId)
+                            .collect(Collectors.toList());
+            for (ResultPartitionID partitionID : resultPartitionIds) {
+                
assertThat(partitionTracker.isPartitionTracked(partitionID)).isTrue();
+            }
+        }
+
+        // check middle vertices state were not recovered.
+        for (ExecutionVertex vertex :
+                getExecutionVertices(MIDDLE_ID, 
newScheduler.getExecutionGraph())) {
+            if (vertex.getParallelSubtaskIndex() == subtaskIndex) {
+                
assertThat(vertex.getExecutionState()).isEqualTo(ExecutionState.CREATED);
+                continue;
+            }
+
+            assertThat(middleExecutions)
+                    
.doesNotContain(vertex.getCurrentExecutionAttempt().getAttemptId());
+            
assertThat(vertex.getExecutionState()).isEqualTo(ExecutionState.DEPLOYING);
+        }
+    }
+
+    // This case will use job graph with the following topology:
+    // Source (p=5) -- POINTWISE --> Middle (p=5) -- ALLTOALL --> Sink (p=2)
+    //
+    // This case will undergo the following stages:
+    // 1. All source tasks are finished.
+    // The source vertex contains an operator coordinator that does not 
support batch snapshot.
+    // 2. The jobMaster undergoes a failover.
+    // 3. After the failover, all source tasks are expected to be recovered to 
finished, and their
+    // produced partitions should also be restored.
+    // 4. Trigger all middle task running
+    // 5. All source task should be restarted.
+    @TestTemplate
+    void 
testOpCoordUnsupportedBatchSnapshotWithJobVertexFinishedAndPartitionNotFound()
+            throws Exception {
+        JobGraph jobGraph = deserializeJobGraph(serializedJobGraph);
+        JobVertex jobVertex = jobGraph.findVertexByID(SOURCE_ID);
+        jobVertex.addOperatorCoordinator(
+                new SerializedValue<>(
+                        new TestingOperatorCoordinator.Provider(
+                                
jobVertex.getOperatorIDs().get(0).getGeneratedOperatorID())));
+        AdaptiveBatchScheduler scheduler = createScheduler(jobGraph);
+
+        // start scheduling.
+        runInMainThread(scheduler::startScheduling);
+
+        runInMainThread(
+                () -> {
+                    // trigger all source finished
+                    transitionExecutionsState(scheduler, 
ExecutionState.FINISHED, SOURCE_ID);
+                });
+        List<ExecutionAttemptID> sourceExecutions =
+                
getCurrentAttemptIds(scheduler.getExecutionJobVertex(SOURCE_ID));
+
+        waitWriteJobFinishedEventCompleted(5);
+        runInMainThread(
+                () -> {
+                    // flush events.
+                    jobEventStore.stop(false);
+                });
+
+        // register partitions
+        registerPartitions(scheduler);
+
+        // start a new scheduler and try to recover.
+        AdaptiveBatchScheduler newScheduler = createScheduler(jobGraph);
+        startSchedulingAndWaitRecoverFinish(newScheduler);
+
+        // check source vertices state were recovered.
+        for (ExecutionVertex vertex :
+                getExecutionVertices(SOURCE_ID, 
newScheduler.getExecutionGraph())) {
+            // check state.
+            assertThat(sourceExecutions)
+                    
.contains(vertex.getCurrentExecutionAttempt().getAttemptId());
+            
assertThat(vertex.getExecutionState()).isEqualTo(ExecutionState.FINISHED);
+
+            // check partition tracker was rebuild.
+            JobMasterPartitionTracker partitionTracker =
+                    ((InternalExecutionGraphAccessor) 
newScheduler.getExecutionGraph())
+                            .getPartitionTracker();
+            List<ResultPartitionID> resultPartitionIds =
+                    vertex.getProducedPartitions().keySet().stream()
+                            .map(
+                                    ((DefaultExecutionGraph) 
newScheduler.getExecutionGraph())
+                                            ::createResultPartitionId)
+                            .collect(Collectors.toList());
+            for (ResultPartitionID partitionID : resultPartitionIds) {
+                
assertThat(partitionTracker.isPartitionTracked(partitionID)).isTrue();
+            }
+        }
+
+        runInMainThread(
+                () -> {
+                    // trigger all middle running
+                    transitionExecutionsState(scheduler, 
ExecutionState.RUNNING, SOURCE_ID);
+                });
+
+        // trigger partition not found
+        ExecutionVertex firstTask =
+                getExecutionVertex(MIDDLE_ID, 0, 
newScheduler.getExecutionGraph());
+        triggerFailedByDataConsumptionException(newScheduler, firstTask);
+
+        waitUntilExecutionVertexState(firstTask, ExecutionState.CREATED, 
5000L);
+
+        // verify all middle executionVertices were restarted

Review Comment:
   the comment does not seem to be consistent with the logic below.
   It also does not meet the expectation described in the method comment.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptivebatch/BatchJobRecoveryTest.java:
##########
@@ -0,0 +1,1173 @@
+/*
+ * 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.JobID;
+import org.apache.flink.api.common.eventtime.WatermarkAlignmentParams;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.mocks.MockSource;
+import org.apache.flink.api.connector.source.mocks.MockSourceSplit;
+import org.apache.flink.api.connector.source.mocks.MockSplitEnumerator;
+import org.apache.flink.configuration.BatchExecutionOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+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.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.IntermediateResultPartition;
+import org.apache.flink.runtime.executiongraph.InternalExecutionGraphAccessor;
+import org.apache.flink.runtime.executiongraph.ResultPartitionBytes;
+import 
org.apache.flink.runtime.executiongraph.TestingComponentMainThreadExecutor;
+import 
org.apache.flink.runtime.executiongraph.failover.FixedDelayRestartBackoffTimeStrategy;
+import org.apache.flink.runtime.io.network.partition.JobMasterPartitionTracker;
+import 
org.apache.flink.runtime.io.network.partition.JobMasterPartitionTrackerImpl;
+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.IntermediateResultPartitionID;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.jobmaster.event.ExecutionVertexFinishedEvent;
+import org.apache.flink.runtime.jobmaster.event.FileSystemJobEventStore;
+import org.apache.flink.runtime.jobmaster.event.JobEvent;
+import org.apache.flink.runtime.jobmaster.event.JobEventManager;
+import org.apache.flink.runtime.jobmaster.event.JobEventStore;
+import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGateway;
+import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGatewayBuilder;
+import org.apache.flink.runtime.operators.coordination.EventReceivingTasks;
+import 
org.apache.flink.runtime.operators.coordination.OperatorCoordinatorHolder;
+import 
org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator;
+import 
org.apache.flink.runtime.operators.coordination.TestingOperatorCoordinator;
+import org.apache.flink.runtime.scheduler.DefaultSchedulerBuilder;
+import org.apache.flink.runtime.scheduler.SchedulerBase;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.scheduler.strategy.SchedulingResultPartition;
+import org.apache.flink.runtime.scheduler.strategy.SchedulingTopology;
+import org.apache.flink.runtime.shuffle.DefaultShuffleMetrics;
+import org.apache.flink.runtime.shuffle.JobShuffleContextImpl;
+import org.apache.flink.runtime.shuffle.NettyShuffleDescriptor;
+import org.apache.flink.runtime.shuffle.NettyShuffleMaster;
+import org.apache.flink.runtime.shuffle.PartitionWithMetrics;
+import org.apache.flink.runtime.shuffle.ShuffleDescriptor;
+import org.apache.flink.runtime.shuffle.ShuffleMaster;
+import org.apache.flink.runtime.shuffle.ShuffleMetrics;
+import org.apache.flink.runtime.source.coordinator.SourceCoordinator;
+import org.apache.flink.runtime.source.coordinator.SourceCoordinatorProvider;
+import org.apache.flink.runtime.source.event.ReaderRegistrationEvent;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.runtime.testutils.CommonTestUtils;
+import org.apache.flink.testutils.TestingUtils;
+import org.apache.flink.testutils.executor.TestExecutorExtension;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameter;
+import 
org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameters;
+import org.apache.flink.testutils.junit.utils.TempDirUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.SerializedValue;
+import org.apache.flink.util.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.util.concurrent.ScheduledExecutor;
+import org.apache.flink.util.concurrent.ScheduledExecutorServiceAdapter;
+import org.apache.flink.util.function.SupplierWithException;
+import org.apache.flink.util.function.ThrowingRunnable;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.junit.jupiter.api.io.TempDir;
+
+import javax.annotation.Nonnull;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+import static 
org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.waitUntilExecutionVertexState;
+import static 
org.apache.flink.runtime.scheduler.DefaultSchedulerBuilder.createCustomParallelismDecider;
+import static org.apache.flink.util.Preconditions.checkState;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.fail;
+
+/** Test for batch job recovery. */
+@ExtendWith(ParameterizedTestExtension.class)
+public class BatchJobRecoveryTest {
+
+    private final Duration previousWorkerRecoveryTimeout = 
Duration.ofSeconds(1);
+
+    @TempDir private java.nio.file.Path temporaryFolder;
+
+    // ---- Mocks for the underlying Operator Coordinator Context ---
+    protected EventReceivingTasks receivingTasks;
+
+    @RegisterExtension
+    static final TestExecutorExtension<ScheduledExecutorService> 
EXECUTOR_RESOURCE =
+            TestingUtils.defaultExecutorExtension();
+
+    @RegisterExtension
+    static final TestingComponentMainThreadExecutor.Extension 
MAIN_EXECUTOR_RESOURCE =
+            new TestingComponentMainThreadExecutor.Extension();
+
+    private final TestingComponentMainThreadExecutor mainThreadExecutor =
+            MAIN_EXECUTOR_RESOURCE.getComponentMainThreadTestExecutor();
+
+    private ScheduledExecutor delayedExecutor =
+            new 
ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor());
+
+    private static final OperatorID OPERATOR_ID = new OperatorID(1234L, 5678L);
+    private static final int NUM_SPLITS = 10;
+    private static final int SOURCE_PARALLELISM = 5;
+    private static final int MIDDLE_PARALLELISM = 5;
+    private static final int DECIDED_SINK_PARALLELISM = 2;
+    private static final JobVertexID SOURCE_ID = new JobVertexID();
+    private static final JobVertexID MIDDLE_ID = new JobVertexID();
+    private static final JobVertexID SINK_ID = new JobVertexID();
+    private static final JobID JOB_ID = new JobID();
+
+    private SourceCoordinatorProvider<MockSourceSplit> provider;
+    private FileSystemJobEventStore jobEventStore;
+    private List<JobEvent> persistedJobEventList;
+
+    private byte[] serializedJobGraph;
+
+    private final Collection<PartitionWithMetrics> allPartitionWithMetrics = 
new ArrayList<>();
+
+    @Parameter public boolean enableSpeculativeExecution;
+
+    @Parameters(name = "enableSpeculativeExecution={0}")
+    public static Collection<Boolean> parameters() {
+        return Arrays.asList(false, true);
+    }
+
+    @BeforeEach
+    void setUp() throws IOException {
+        final Path rootPath = new 
Path(TempDirUtils.newFolder(temporaryFolder).getAbsolutePath());
+        delayedExecutor = new 
ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor());
+        receivingTasks = EventReceivingTasks.createForRunningTasks();
+        persistedJobEventList = new ArrayList<>();
+        jobEventStore =
+                new TestingFileSystemJobEventStore(
+                        rootPath, new Configuration(), persistedJobEventList);
+
+        provider =
+                new SourceCoordinatorProvider<>(
+                        "AdaptiveBatchSchedulerTest",
+                        OPERATOR_ID,
+                        new MockSource(Boundedness.BOUNDED, NUM_SPLITS),
+                        1,
+                        WatermarkAlignmentParams.WATERMARK_ALIGNMENT_DISABLED,
+                        null);
+
+        this.serializedJobGraph = serializeJobGraph(createDefaultJobGraph());
+        allPartitionWithMetrics.clear();
+    }
+
+    @AfterEach
+    void after() {
+        jobEventStore.stop(true);
+    }
+
+    // This case will use job graph with the following topology:
+    // Source (p=5) -- POINTWISE --> Middle (p=5) -- ALLTOALL --> Sink (p=2)
+    //
+    // This case will experience the following stages:
+    // 1. All source tasks are finished and all middle tasks are running
+    // 2. JM failover
+    // 3. After the failover, all source tasks are expected to be recovered to
+    //  finished, and their produced partitions should also be restored. And 
middle vertex is
+    // redeployed.
+    @TestTemplate
+    void testRecoverFromJMFailover() throws Exception {
+        AdaptiveBatchScheduler scheduler = 
createScheduler(deserializeJobGraph(serializedJobGraph));
+
+        // start scheduling.
+        runInMainThread(scheduler::startScheduling);
+
+        runInMainThread(
+                () -> {
+                    // trigger all source finished
+                    transitionExecutionsState(scheduler, 
ExecutionState.FINISHED, SOURCE_ID);
+                });
+        runInMainThread(
+                () -> {
+                    // trigger all middle tasks to RUNNING state
+                    transitionExecutionsState(scheduler, 
ExecutionState.INITIALIZING, MIDDLE_ID);
+                    transitionExecutionsState(scheduler, 
ExecutionState.RUNNING, MIDDLE_ID);
+                });
+        List<ExecutionAttemptID> sourceExecutions =
+                
getCurrentAttemptIds(scheduler.getExecutionJobVertex(SOURCE_ID));
+        List<ExecutionAttemptID> middleExecutions =
+                
getCurrentAttemptIds(scheduler.getExecutionJobVertex(MIDDLE_ID));
+
+        Map<IntermediateResultPartitionID, Integer> subpartitionNums = new 
HashMap<>();
+        for (ExecutionVertex vertex :
+                getExecutionVertices(SOURCE_ID, 
scheduler.getExecutionGraph())) {
+
+            IntermediateResultPartition partition =
+                    vertex.getProducedPartitions().values().iterator().next();
+            subpartitionNums.put(partition.getPartitionId(), 
partition.getNumberOfSubpartitions());
+        }
+
+        waitWriteJobFinishedEventCompleted(5);
+        runInMainThread(
+                () -> {
+                    // flush events.
+                    jobEventStore.stop(false);
+                });
+
+        // register partitions
+        registerPartitions(scheduler);
+
+        // start a new scheduler and try to recover.
+        AdaptiveBatchScheduler newScheduler =
+                createScheduler(deserializeJobGraph(serializedJobGraph));
+        startSchedulingAndWaitRecoverFinish(newScheduler);
+
+        // check source vertices state were recovered.
+        for (ExecutionVertex vertex :
+                getExecutionVertices(SOURCE_ID, 
newScheduler.getExecutionGraph())) {
+            // check state.
+            assertThat(sourceExecutions)
+                    
.contains(vertex.getCurrentExecutionAttempt().getAttemptId());
+            
assertThat(vertex.getExecutionState()).isEqualTo(ExecutionState.FINISHED);
+
+            // check partition tracker was rebuild.
+            JobMasterPartitionTracker partitionTracker =
+                    ((InternalExecutionGraphAccessor) 
newScheduler.getExecutionGraph())
+                            .getPartitionTracker();
+            List<ResultPartitionID> resultPartitionIds =
+                    vertex.getProducedPartitions().keySet().stream()
+                            .map(
+                                    ((DefaultExecutionGraph) 
newScheduler.getExecutionGraph())
+                                            ::createResultPartitionId)
+                            .collect(Collectors.toList());
+            for (ResultPartitionID partitionID : resultPartitionIds) {
+                
assertThat(partitionTracker.isPartitionTracked(partitionID)).isTrue();
+            }
+
+            // check partitions are recovered
+            IntermediateResultPartition partition =
+                    vertex.getProducedPartitions().values().iterator().next();
+            assertThat(partition.getNumberOfSubpartitions())
+                    
.isEqualTo(subpartitionNums.get(partition.getPartitionId()));
+        }
+
+        // check middle vertices state were not recovered.
+        for (ExecutionVertex vertex :
+                getExecutionVertices(MIDDLE_ID, 
newScheduler.getExecutionGraph())) {
+            assertThat(middleExecutions)
+                    
.doesNotContain(vertex.getCurrentExecutionAttempt().getAttemptId());
+            
assertThat(vertex.getExecutionState()).isEqualTo(ExecutionState.DEPLOYING);
+        }
+    }
+
+    // This case will use job graph with the following topology:
+    // Source (p=5) -- POINTWISE --> Middle (p=5) -- ALLTOALL --> Sink (p=2)
+    //
+    // This case will undergo the following stages:
+    // 1. All source tasks are finished, as well as middle task 0, while other 
middle tasks are
+    // still running.
+    // The middle vertex contains an operator coordinator that does not 
support batch snapshot.
+    // 2. The partition belonging to source task0 is released because middle 
task0 has finished.
+    // 3. The JobMaster undergoes a failover.
+    // 4. After failover, middle task0 and source task0 are expected be reset. 
Other source tasks
+    // should be restored to finished and their produced partitions should 
also be restored.
+    @TestTemplate
+    void testOpCoordUnsupportedBatchSnapshotWithJobVertexUnFinished() throws 
Exception {
+        JobGraph jobGraph = deserializeJobGraph(serializedJobGraph);
+        JobVertex jobVertex = jobGraph.findVertexByID(MIDDLE_ID);
+        jobVertex.addOperatorCoordinator(
+                new SerializedValue<>(
+                        new TestingOperatorCoordinator.Provider(
+                                
jobVertex.getOperatorIDs().get(0).getGeneratedOperatorID())));
+        AdaptiveBatchScheduler scheduler =
+                createScheduler(
+                        jobGraph,
+                        Duration.ZERO /* make sure every finished event can 
flush on time.*/);
+
+        // start scheduling.
+        runInMainThread(scheduler::startScheduling);
+
+        runInMainThread(
+                () -> {
+                    // trigger all source finished
+                    transitionExecutionsState(scheduler, 
ExecutionState.FINISHED, SOURCE_ID);
+                });
+        runInMainThread(
+                () -> {
+                    // trigger first middle finished.
+                    ExecutionVertex firstMiddle =
+                            getExecutionVertex(MIDDLE_ID, 0, 
scheduler.getExecutionGraph());
+                    AdaptiveBatchSchedulerTest.transitionExecutionsState(
+                            scheduler,
+                            ExecutionState.FINISHED,
+                            
Collections.singletonList(firstMiddle.getCurrentExecutionAttempt()),
+                            null);
+                });
+        List<ExecutionAttemptID> sourceExecutions =
+                
getCurrentAttemptIds(scheduler.getExecutionJobVertex(SOURCE_ID));
+        List<ExecutionAttemptID> middleExecutions =
+                
getCurrentAttemptIds(scheduler.getExecutionJobVertex(MIDDLE_ID));
+
+        waitWriteJobFinishedEventCompleted(6);
+        runInMainThread(
+                () -> {
+                    // flush events.
+                    jobEventStore.stop(false);
+                });
+
+        // register partitions, the partition of source task 0 is lost, and it 
will be restarted
+        // if middle task 0 need be restarted.
+        int subtaskIndex = 0;
+        registerPartitions(
+                scheduler,
+                Collections.emptySet(),
+                Collections.singleton(
+                        scheduler
+                                .getExecutionJobVertex(SOURCE_ID)
+                                .getTaskVertices()[subtaskIndex]
+                                .getID()));
+
+        // start a new scheduler and try to recover.
+        AdaptiveBatchScheduler newScheduler = createScheduler(jobGraph);
+        startSchedulingAndWaitRecoverFinish(newScheduler);
+
+        // check source vertices state were recovered.
+        for (ExecutionVertex vertex :
+                getExecutionVertices(SOURCE_ID, 
newScheduler.getExecutionGraph())) {
+            if (vertex.getParallelSubtaskIndex() == subtaskIndex) {
+                
assertThat(vertex.getExecutionState()).isEqualTo(ExecutionState.DEPLOYING);
+                continue;
+            }
+
+            // check state.
+            assertThat(sourceExecutions)
+                    
.contains(vertex.getCurrentExecutionAttempt().getAttemptId());
+            
assertThat(vertex.getExecutionState()).isEqualTo(ExecutionState.FINISHED);
+
+            // check partition tracker was rebuild.
+            JobMasterPartitionTracker partitionTracker =
+                    ((InternalExecutionGraphAccessor) 
newScheduler.getExecutionGraph())
+                            .getPartitionTracker();
+            List<ResultPartitionID> resultPartitionIds =
+                    vertex.getProducedPartitions().keySet().stream()
+                            .map(
+                                    ((DefaultExecutionGraph) 
newScheduler.getExecutionGraph())
+                                            ::createResultPartitionId)
+                            .collect(Collectors.toList());
+            for (ResultPartitionID partitionID : resultPartitionIds) {
+                
assertThat(partitionTracker.isPartitionTracked(partitionID)).isTrue();
+            }
+        }
+
+        // check middle vertices state were not recovered.
+        for (ExecutionVertex vertex :
+                getExecutionVertices(MIDDLE_ID, 
newScheduler.getExecutionGraph())) {
+            if (vertex.getParallelSubtaskIndex() == subtaskIndex) {
+                
assertThat(vertex.getExecutionState()).isEqualTo(ExecutionState.CREATED);
+                continue;
+            }
+
+            assertThat(middleExecutions)
+                    
.doesNotContain(vertex.getCurrentExecutionAttempt().getAttemptId());
+            
assertThat(vertex.getExecutionState()).isEqualTo(ExecutionState.DEPLOYING);
+        }
+    }
+
+    // This case will use job graph with the following topology:
+    // Source (p=5) -- POINTWISE --> Middle (p=5) -- ALLTOALL --> Sink (p=2)
+    //
+    // This case will undergo the following stages:
+    // 1. All source tasks are finished.
+    // The source vertex contains an operator coordinator that does not 
support batch snapshot.
+    // 2. The jobMaster undergoes a failover.
+    // 3. After the failover, all source tasks are expected to be recovered to 
finished, and their
+    // produced partitions should also be restored.
+    // 4. Trigger all middle task running
+    // 5. All source task should be restarted.
+    @TestTemplate
+    void 
testOpCoordUnsupportedBatchSnapshotWithJobVertexFinishedAndPartitionNotFound()
+            throws Exception {
+        JobGraph jobGraph = deserializeJobGraph(serializedJobGraph);
+        JobVertex jobVertex = jobGraph.findVertexByID(SOURCE_ID);
+        jobVertex.addOperatorCoordinator(
+                new SerializedValue<>(
+                        new TestingOperatorCoordinator.Provider(
+                                
jobVertex.getOperatorIDs().get(0).getGeneratedOperatorID())));
+        AdaptiveBatchScheduler scheduler = createScheduler(jobGraph);
+
+        // start scheduling.
+        runInMainThread(scheduler::startScheduling);
+
+        runInMainThread(
+                () -> {
+                    // trigger all source finished
+                    transitionExecutionsState(scheduler, 
ExecutionState.FINISHED, SOURCE_ID);
+                });
+        List<ExecutionAttemptID> sourceExecutions =
+                
getCurrentAttemptIds(scheduler.getExecutionJobVertex(SOURCE_ID));
+
+        waitWriteJobFinishedEventCompleted(5);
+        runInMainThread(
+                () -> {
+                    // flush events.
+                    jobEventStore.stop(false);
+                });
+
+        // register partitions
+        registerPartitions(scheduler);
+
+        // start a new scheduler and try to recover.
+        AdaptiveBatchScheduler newScheduler = createScheduler(jobGraph);
+        startSchedulingAndWaitRecoverFinish(newScheduler);
+
+        // check source vertices state were recovered.
+        for (ExecutionVertex vertex :
+                getExecutionVertices(SOURCE_ID, 
newScheduler.getExecutionGraph())) {
+            // check state.
+            assertThat(sourceExecutions)
+                    
.contains(vertex.getCurrentExecutionAttempt().getAttemptId());
+            
assertThat(vertex.getExecutionState()).isEqualTo(ExecutionState.FINISHED);
+
+            // check partition tracker was rebuild.
+            JobMasterPartitionTracker partitionTracker =
+                    ((InternalExecutionGraphAccessor) 
newScheduler.getExecutionGraph())
+                            .getPartitionTracker();
+            List<ResultPartitionID> resultPartitionIds =
+                    vertex.getProducedPartitions().keySet().stream()
+                            .map(
+                                    ((DefaultExecutionGraph) 
newScheduler.getExecutionGraph())
+                                            ::createResultPartitionId)
+                            .collect(Collectors.toList());
+            for (ResultPartitionID partitionID : resultPartitionIds) {
+                
assertThat(partitionTracker.isPartitionTracked(partitionID)).isTrue();
+            }
+        }
+
+        runInMainThread(
+                () -> {
+                    // trigger all middle running
+                    transitionExecutionsState(scheduler, 
ExecutionState.RUNNING, SOURCE_ID);

Review Comment:
   Mismatched comments and logic.



-- 
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-unsubscr...@flink.apache.org

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

Reply via email to