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


##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/DefaultBatchJobRecoveryHandler.java:
##########
@@ -0,0 +1,850 @@
+/*
+ * 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.JobStatus;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.BatchExecutionOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+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.JobVertexInputInfo;
+import org.apache.flink.runtime.executiongraph.ResultPartitionBytes;
+import org.apache.flink.runtime.failure.FailureEnricherUtils;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
+import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import 
org.apache.flink.runtime.jobmaster.event.ExecutionJobVertexInitializedEvent;
+import org.apache.flink.runtime.jobmaster.event.ExecutionVertexFinishedEvent;
+import org.apache.flink.runtime.jobmaster.event.ExecutionVertexResetEvent;
+import org.apache.flink.runtime.jobmaster.event.JobEvent;
+import org.apache.flink.runtime.jobmaster.event.JobEventManager;
+import org.apache.flink.runtime.jobmaster.event.JobEventReplayHandler;
+import 
org.apache.flink.runtime.operators.coordination.OperatorCoordinatorHolder;
+import org.apache.flink.runtime.scheduler.strategy.ConsumerVertexGroup;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.shuffle.DefaultShuffleMasterSnapshotContext;
+import org.apache.flink.runtime.shuffle.PartitionWithMetrics;
+import org.apache.flink.runtime.shuffle.ShuffleDescriptor;
+import org.apache.flink.runtime.shuffle.ShuffleMasterSnapshot;
+import org.apache.flink.util.clock.Clock;
+import org.apache.flink.util.clock.SystemClock;
+
+import org.apache.flink.shaded.guava31.com.google.common.collect.Sets;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import static 
org.apache.flink.runtime.operators.coordination.OperatorCoordinator.NO_CHECKPOINT;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** Default implementation of {@link BatchJobRecoveryHandler} and {@link 
JobEventReplayHandler}. */
+public class DefaultBatchJobRecoveryHandler
+        implements BatchJobRecoveryHandler, JobEventReplayHandler {
+
+    private final Logger log = LoggerFactory.getLogger(getClass());
+
+    private final JobEventManager jobEventManager;
+
+    private BatchJobRecoveryContext context;
+
+    /** The timestamp (via {@link Clock#relativeTimeMillis()}) of the last 
snapshot. */
+    private long lastSnapshotRelativeTime;
+
+    private final Set<JobVertexID> needToSnapshotJobVertices = new HashSet<>();
+
+    private static final ResourceID UNKNOWN_PRODUCER = ResourceID.generate();
+
+    private final long snapshotMinPauseMills;
+
+    private Clock clock;
+
+    private final Map<ExecutionVertexID, ExecutionVertexFinishedEvent>
+            executionVertexFinishedEventMap = new LinkedHashMap<>();
+
+    private final List<ExecutionJobVertexInitializedEvent> 
jobVertexInitializedEvents =
+            new ArrayList<>();
+
+    /**
+     * A set of JobVertex Ids associated with JobVertices whose 
operatorCoordinators did not
+     * successfully recover. And if any execution within these job vertices 
needs to be restarted in
+     * the future, all other executions within the same vertex must also be 
restarted to ensure the
+     * consistency and correctness of the state.
+     */
+    private final Set<JobVertexID> jobVerticesWithUnRecoveredCoordinators = 
new HashSet<>();
+
+    private final Duration previousWorkerRecoveryTimeout;
+
+    public DefaultBatchJobRecoveryHandler(
+            JobEventManager jobEventManager, Configuration 
jobMasterConfiguration) {
+        this.jobEventManager = jobEventManager;
+
+        this.previousWorkerRecoveryTimeout =
+                jobMasterConfiguration.get(
+                        
BatchExecutionOptions.JOB_RECOVERY_PREVIOUS_WORKER_RECOVERY_TIMEOUT);
+        this.snapshotMinPauseMills =
+                jobMasterConfiguration
+                        
.get(BatchExecutionOptions.JOB_RECOVERY_SNAPSHOT_MIN_PAUSE)
+                        .toMillis();
+    }
+
+    @Override
+    public void initialize(BatchJobRecoveryContext context) {
+

Review Comment:
   Useless empty line.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/DefaultBatchJobRecoveryHandler.java:
##########
@@ -791,18 +785,34 @@ private boolean 
isPartitionShouldBeReconciled(IntermediateResultPartition partit
                                 jobVertexId -> 
getExecutionJobVertex(jobVertexId).isInitialized());
 
         if (!allConsumersInitialized) {
-            return true;
+            return PartitionReservationStatus.RESERVE;
         }
 
-        // 3. If all downstream vertices are initialized, we need to check if 
the all partition
-        // consumers finished.
+        // 3. If all downstream vertices are finished, we need reserve the 
partitions. Otherwise, we

Review Comment:
   all downstream vertices -> not all the consumer vertices
   
   downstream vertices include vertices which do not directly consume the 
partition.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/DefaultBatchJobRecoveryHandler.java:
##########
@@ -813,28 +823,28 @@ private List<ExecutionVertex> 
getConsumers(IntermediateResultPartitionID partiti
     }
 
     private ExecutionVertex getExecutionVertex(final ExecutionVertexID 
executionVertexId) {
-        return executionGraph
+        return context.getExecutionGraph()
                 .getAllVertices()
                 .get(executionVertexId.getJobVertexId())
                 .getTaskVertices()[executionVertexId.getSubtaskIndex()];
     }
 
     private ExecutionJobVertex getExecutionJobVertex(final JobVertexID 
jobVertexId) {
-        return executionGraph.getAllVertices().get(jobVertexId);
+        return context.getExecutionGraph().getAllVertices().get(jobVertexId);
     }
 
     private static class ReconcileResult {
-        private final Set<ResultPartitionID> unknownPartitions;
-        private final Set<ResultPartitionID> missingPartitions;
-        private final Set<ResultPartitionID> availablePartitions;
+        private final Set<ResultPartitionID> partitionsToBeReleased;

Review Comment:
   partitionsToBeReleased -> partitionsToRelease



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/DefaultBatchJobRecoveryHandler.java:
##########
@@ -813,28 +823,28 @@ private List<ExecutionVertex> 
getConsumers(IntermediateResultPartitionID partiti
     }
 
     private ExecutionVertex getExecutionVertex(final ExecutionVertexID 
executionVertexId) {
-        return executionGraph
+        return context.getExecutionGraph()
                 .getAllVertices()
                 .get(executionVertexId.getJobVertexId())
                 .getTaskVertices()[executionVertexId.getSubtaskIndex()];
     }
 
     private ExecutionJobVertex getExecutionJobVertex(final JobVertexID 
jobVertexId) {
-        return executionGraph.getAllVertices().get(jobVertexId);
+        return context.getExecutionGraph().getAllVertices().get(jobVertexId);
     }
 
     private static class ReconcileResult {
-        private final Set<ResultPartitionID> unknownPartitions;
-        private final Set<ResultPartitionID> missingPartitions;
-        private final Set<ResultPartitionID> availablePartitions;
+        private final Set<ResultPartitionID> partitionsToBeReleased;
+        private final Set<ResultPartitionID> partitionsMissed;

Review Comment:
   partitionsMissed -> partitionsMissing



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/AdaptiveBatchScheduler.java:
##########
@@ -281,6 +261,11 @@ protected void startSchedulingInternal() {
         }
     }
 
+    /**
+     * Modifies the vertices which need to be restarted. If any task need 
restarting belong to job
+     * vertices with unRecovered operator coordinators, all tasks within those 
job vertices are need

Review Comment:
   any task need restarting belong to -> any task needing restarting belongs
   
   unRecovered -> unrecovered



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/AdaptiveBatchScheduler.java:
##########
@@ -250,26 +245,11 @@ private SpeculativeExecutionHandler 
createSpeculativeExecutionHandler(
     protected void startSchedulingInternal() {
         speculativeExecutionHandler.init(
                 getExecutionGraph(), getMainThreadExecutor(), 
jobManagerJobMetricGroup);
-        jobRecoveryHandler.initialize(
-                log,
-                getExecutionGraph(),
-                shuffleMaster,
-                getMainThreadExecutor(),
-                failoverStrategy,
-                this::failJob,
-                this::resetVerticesInRecovering,
-                this::updateResultPartitionBytesMetrics,
-                this::initializeJobVertex,
-                this::updateTopology);
+        jobRecoveryHandler.initialize(new DefaultBatchJobRecoveryContext());
 
         if (jobRecoveryHandler.needRecover()) {
-            getMainThreadExecutor()
-                    .schedule(
-                            () ->
-                                    jobRecoveryHandler.startRecovering(
-                                            this::onRecoveringFinished, 
this::onRecoveringFailed),
-                            previousWorkerRecoveryTimeout.toMillis(),
-                            TimeUnit.MILLISECONDS);
+            jobRecoveryHandler.startRecovering(
+                    this::onRecoveringFinished, this::onRecoveringFailed);

Review Comment:
   `onRecoveringFinished` and `onRecoveringFailed` can be part of the context.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/DefaultBatchJobRecoveryHandler.java:
##########
@@ -813,28 +823,28 @@ private List<ExecutionVertex> 
getConsumers(IntermediateResultPartitionID partiti
     }
 
     private ExecutionVertex getExecutionVertex(final ExecutionVertexID 
executionVertexId) {
-        return executionGraph
+        return context.getExecutionGraph()
                 .getAllVertices()
                 .get(executionVertexId.getJobVertexId())
                 .getTaskVertices()[executionVertexId.getSubtaskIndex()];
     }
 
     private ExecutionJobVertex getExecutionJobVertex(final JobVertexID 
jobVertexId) {
-        return executionGraph.getAllVertices().get(jobVertexId);
+        return context.getExecutionGraph().getAllVertices().get(jobVertexId);
     }
 
     private static class ReconcileResult {
-        private final Set<ResultPartitionID> unknownPartitions;
-        private final Set<ResultPartitionID> missingPartitions;
-        private final Set<ResultPartitionID> availablePartitions;
+        private final Set<ResultPartitionID> partitionsToBeReleased;
+        private final Set<ResultPartitionID> partitionsMissed;
+        private final Set<ResultPartitionID> partitionsToBeReserved;

Review Comment:
   partitionsToBeReserved -> partitionsToReserve



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