Re: [PR] [FLINK-33892][runtime] Support Job Recovery from JobMaster Failures for Batch Jobs. [flink]

2024-05-27 Thread via GitHub


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


##
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptivebatch/BatchJobRecoveryTest.java:
##
@@ -406,16 +405,16 @@ void 
testJobVertexUnFinishedAndOperatorCoordinatorNotSupportBatchSnapshot() thro
 }
 
 // This case will use job graph with the following topology:
-// Source (p=5) -- POINTWISE --> Middle (p=5) -- ALLTOALL --> Sink (p=2)
+// Source (p=5) -- POINTWISE --> Middle (p=5) -- ALLTOALL --> Sink (p=-1)
 //
 // 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. JM 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. Trigger the partition consumed by middle task0 to go missing.
+// 4. Transition all middle task to running
+// 5. Transition the partition consumed by middle task0 to go missing.

Review Comment:
   Mark the partition consumed by middle task0 as missing.



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



Re: [PR] [FLINK-33892][runtime] Support Job Recovery from JobMaster Failures for Batch Jobs. [flink]

2024-05-24 Thread via GitHub


JunRuiLee commented on PR #24771:
URL: https://github.com/apache/flink/pull/24771#issuecomment-2130666752

   Thanks @zhuzhurk for the thorough review. I have refactored the 
BatchJobRecoveryTest and JMFailoverITCase based on your comments. PTAL.
   
   


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



Re: [PR] [FLINK-33892][runtime] Support Job Recovery from JobMaster Failures for Batch Jobs. [flink]

2024-05-24 Thread via GitHub


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 indices, boolean going, Map 
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 

Re: [PR] [FLINK-33892][runtime] Support Job Recovery from JobMaster Failures for Batch Jobs. [flink]

2024-05-24 Thread via GitHub


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


##
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptivebatch/BatchJobRecoveryTest.java:
##
@@ -0,0 +1,1217 @@
+/*
+ * 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 

Re: [PR] [FLINK-33892][runtime] Support Job Recovery from JobMaster Failures for Batch Jobs. [flink]

2024-05-24 Thread via GitHub


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


##
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:
   This case utilizes the TestExecutorResource, which is compatible with JUnit 
4.



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



Re: [PR] [FLINK-33892][runtime] Support Job Recovery from JobMaster Failures for Batch Jobs. [flink]

2024-05-24 Thread via GitHub


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


##
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:
   Junit5 should be used.



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



Re: [PR] [FLINK-33892][runtime] Support Job Recovery from JobMaster Failures for Batch Jobs. [flink]

2024-05-24 Thread via GitHub


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


##
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptivebatch/BatchJobRecoveryTest.java:
##
@@ -0,0 +1,1217 @@
+/*
+ * 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 

Re: [PR] [FLINK-33892][runtime] Support Job Recovery from JobMaster Failures for Batch Jobs. [flink]

2024-05-24 Thread via GitHub


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


##
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java:
##
@@ -1012,12 +1013,12 @@ public 
CompletableFuture> getPartitionWithMetri
 timeout.toMillis(),
 expectedPartitions);
 
-return 
fetchedPartitionsWithMetrics.values();
+return new 
ArrayList<>(fetchedPartitionsWithMetrics.values());

Review Comment:
   Because the values() method of HashMap returns a collection that is not 
serializable.



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



Re: [PR] [FLINK-33892][runtime] Support Job Recovery from JobMaster Failures for Batch Jobs. [flink]

2024-05-23 Thread via GitHub


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 needToSnapshotJobVertices = new HashSet<>();
+
+private static final ResourceID UNKNOWN_PRODUCER = ResourceID.generate();
+
+private 

Re: [PR] [FLINK-33892][runtime] Support Job Recovery from JobMaster Failures for Batch Jobs. [flink]

2024-05-23 Thread via GitHub


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


##
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java:
##
@@ -229,11 +231,15 @@ public class JobMaster extends 
FencedRpcEndpoint
 
 private final BlocklistHandler blocklistHandler;
 
-private final List>>
-partitionWithMetricsOnTaskManagerFutures = new ArrayList<>();
+private final Map 
partitionWithMetricsOnTaskManagers =
+new HashMap<>();
 
 private boolean fetchAndRetainPartitions = false;

Review Comment:
   Comments are needed to explain in what scenario it will be used?



##
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java:
##
@@ -1001,17 +1037,54 @@ private void 
fetchAndRetainPartitionWithMetricsOnTaskManager(ResourceID resource
 TaskManagerRegistration taskManager = 
registeredTaskManagers.get(resourceId);
 checkNotNull(taskManager);
 
-partitionWithMetricsOnTaskManagerFutures.add(
-taskManager
-.getTaskExecutorGateway()
-
.getAndRetainPartitionWithMetrics(jobGraph.getJobID()));
+taskManager
+.getTaskExecutorGateway()
+.getAndRetainPartitionWithMetrics(jobGraph.getJobID())
+.thenAccept(
+partitionWithMetrics -> {
+if (fetchAndRetainPartitions) {
+for (PartitionWithMetrics partitionWithMetric :
+partitionWithMetrics) {
+partitionWithMetricsOnTaskManagers.put(
+partitionWithMetric
+.getPartition()
+.getResultPartitionID(),
+partitionWithMetric);
+}
+checkPartitionOnTaskManagerReportFinished();
+} else {
+log.info(
+"Received partition metrics from {} 
later. Releasing it.",

Review Comment:
   Received partition metrics from {} later. Releasing it. -> Received late 
report of partition metrics from {}. Release the partitions.



##
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java:
##
@@ -1001,17 +1037,54 @@ private void 
fetchAndRetainPartitionWithMetricsOnTaskManager(ResourceID resource
 TaskManagerRegistration taskManager = 
registeredTaskManagers.get(resourceId);
 checkNotNull(taskManager);
 
-partitionWithMetricsOnTaskManagerFutures.add(
-taskManager
-.getTaskExecutorGateway()
-
.getAndRetainPartitionWithMetrics(jobGraph.getJobID()));
+taskManager
+.getTaskExecutorGateway()
+.getAndRetainPartitionWithMetrics(jobGraph.getJobID())
+.thenAccept(
+partitionWithMetrics -> {
+if (fetchAndRetainPartitions) {
+for (PartitionWithMetrics partitionWithMetric :
+partitionWithMetrics) {
+partitionWithMetricsOnTaskManagers.put(
+partitionWithMetric
+.getPartition()
+.getResultPartitionID(),
+partitionWithMetric);
+}
+checkPartitionOnTaskManagerReportFinished();
+} else {
+log.info(
+"Received partition metrics from {} 
later. Releasing it.",
+resourceId);
+
+taskManager
+.getTaskExecutorGateway()
+.releasePartitions(
+jobGraph.getJobID(),
+partitionWithMetrics.stream()
+
.map(PartitionWithMetrics::getPartition)
+.map(
+
ShuffleDescriptor
+
::getResultPartitionID)
+
.collect(Collectors.toSet()));
+}
+});
 }
 
-@Override
-

Re: [PR] [FLINK-33892][runtime] Support Job Recovery from JobMaster Failures for Batch Jobs. [flink]

2024-05-23 Thread via GitHub


JunRuiLee commented on PR #24771:
URL: https://github.com/apache/flink/pull/24771#issuecomment-2126498507

   Thanks @zhuzhurk for reviewing, I've updated this pr accordingly. PTAL.


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



Re: [PR] [FLINK-33892][runtime] Support Job Recovery from JobMaster Failures for Batch Jobs. [flink]

2024-05-23 Thread via GitHub


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


##
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/DefaultBatchJobRecoveryHandler.java:
##
@@ -0,0 +1,840 @@
+/*
+ * 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.configuration.BatchExecutionOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.JobException;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
+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.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.JobVertexInputInfo;
+import org.apache.flink.runtime.executiongraph.ResultPartitionBytes;
+import org.apache.flink.runtime.executiongraph.TaskExecutionStateTransition;
+import org.apache.flink.runtime.executiongraph.failover.FailoverStrategy;
+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.ShuffleMaster;
+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.util.function.ConsumerWithException;
+import org.apache.flink.util.function.QuadConsumerWithException;
+import org.apache.flink.util.function.TriConsumer;
+
+import org.apache.flink.shaded.guava31.com.google.common.collect.Sets;
+
+import org.slf4j.Logger;
+
+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.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 Logger log;
+
+private final 

Re: [PR] [FLINK-33892][runtime] Support Job Recovery from JobMaster Failures for Batch Jobs. [flink]

2024-05-23 Thread via GitHub


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


##
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/AdaptiveBatchScheduler.java:
##
@@ -224,18 +250,153 @@ 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);
+
+if (jobRecoveryHandler.needRecover()) {
+getMainThreadExecutor()
+.schedule(
+() ->
+jobRecoveryHandler.startRecovering(
+this::onRecoveringFinished, 
this::onRecoveringFailed),
+previousWorkerRecoveryTimeout.toMillis(),
+TimeUnit.MILLISECONDS);
+} else {
+tryComputeSourceParallelismThenRunAsync(
+(Void value, Throwable throwable) -> {
+if (getExecutionGraph().getState() == 
JobStatus.CREATED) {
+initializeVerticesIfPossible();
+super.startSchedulingInternal();
+}
+});
+}
+}
+
+@Override
+protected void maybeRestartTasks(final FailureHandlingResult 
failureHandlingResult) {
+FailureHandlingResult wrappedResult = failureHandlingResult;
+if (failureHandlingResult.canRestart()) {
+Set originalNeedToRestartVertices =
+failureHandlingResult.getVerticesToRestart();
+
+Set extraNeedToRestartJobVertices =
+originalNeedToRestartVertices.stream()
+.map(ExecutionVertexID::getJobVertexId)
+.filter(requiredRestartJobVertices::contains)
+.collect(Collectors.toSet());
+
+
requiredRestartJobVertices.removeAll(extraNeedToRestartJobVertices);
+
+Set needToRestartVertices =
+extraNeedToRestartJobVertices.stream()
+.flatMap(
+jobVertexId -> {
+ExecutionJobVertex jobVertex =
+
getExecutionJobVertex(jobVertexId);
+return 
Arrays.stream(jobVertex.getTaskVertices())
+.map(ExecutionVertex::getID);
+})
+.collect(Collectors.toSet());
+needToRestartVertices.addAll(originalNeedToRestartVertices);
+
+wrappedResult =
+FailureHandlingResult.restartable(
+
failureHandlingResult.getFailedExecution().orElse(null),
+failureHandlingResult.getError(),
+failureHandlingResult.getTimestamp(),
+failureHandlingResult.getFailureLabels(),
+needToRestartVertices,
+failureHandlingResult.getRestartDelayMS(),
+failureHandlingResult.isGlobalFailure(),
+failureHandlingResult.isRootCause());
+}
+
+super.maybeRestartTasks(wrappedResult);
+}
+
+@VisibleForTesting
+boolean isRecovering() {
+return jobRecoveryHandler.isRecovering();
+}
+
+@Override
+public boolean updateTaskExecutionState(final TaskExecutionStateTransition 
taskExecutionState) {
+boolean success = super.updateTaskExecutionState(taskExecutionState);
+
+if (success
+&& taskExecutionState.getExecutionState() == 
ExecutionState.FINISHED
+&& !isRecovering()) {
+final ExecutionVertexID executionVertexId =
+taskExecutionState.getID().getExecutionVertexId();
+jobRecoveryHandler.notifyExecutionFinished(executionVertexId, 
taskExecutionState);
+}
+return success;
+}
+
+@Override
+protected void resetForNewExecutions(Collection 
vertices) {
+super.resetForNewExecutions(vertices);
+if (!isRecovering()) {
+jobRecoveryHandler.notifyExecutionVertexReset(vertices);
+}
+}
+
+private void initializeJobVertex(
+  

Re: [PR] [FLINK-33892][runtime] Support Job Recovery from JobMaster Failures for Batch Jobs. [flink]

2024-05-22 Thread via GitHub


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


##
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/DefaultBatchJobRecoveryHandler.java:
##
@@ -0,0 +1,840 @@
+/*
+ * 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.configuration.BatchExecutionOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.JobException;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
+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.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.JobVertexInputInfo;
+import org.apache.flink.runtime.executiongraph.ResultPartitionBytes;
+import org.apache.flink.runtime.executiongraph.TaskExecutionStateTransition;
+import org.apache.flink.runtime.executiongraph.failover.FailoverStrategy;
+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.ShuffleMaster;
+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.util.function.ConsumerWithException;
+import org.apache.flink.util.function.QuadConsumerWithException;
+import org.apache.flink.util.function.TriConsumer;
+
+import org.apache.flink.shaded.guava31.com.google.common.collect.Sets;
+
+import org.slf4j.Logger;
+
+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.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 Logger log;
+
+private final 

Re: [PR] [FLINK-33892][runtime] Support Job Recovery from JobMaster Failures for Batch Jobs. [flink]

2024-05-22 Thread via GitHub


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


##
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/AdaptiveBatchScheduler.java:
##
@@ -224,18 +250,153 @@ 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);
+
+if (jobRecoveryHandler.needRecover()) {
+getMainThreadExecutor()
+.schedule(
+() ->
+jobRecoveryHandler.startRecovering(
+this::onRecoveringFinished, 
this::onRecoveringFailed),
+previousWorkerRecoveryTimeout.toMillis(),

Review Comment:
   Is it possible to start the recovering immediately when all recorded 
finished partitions are recovered?



##
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/AdaptiveBatchScheduler.java:
##
@@ -224,18 +250,153 @@ 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);
+
+if (jobRecoveryHandler.needRecover()) {
+getMainThreadExecutor()
+.schedule(
+() ->
+jobRecoveryHandler.startRecovering(
+this::onRecoveringFinished, 
this::onRecoveringFailed),
+previousWorkerRecoveryTimeout.toMillis(),
+TimeUnit.MILLISECONDS);
+} else {
+tryComputeSourceParallelismThenRunAsync(
+(Void value, Throwable throwable) -> {
+if (getExecutionGraph().getState() == 
JobStatus.CREATED) {
+initializeVerticesIfPossible();
+super.startSchedulingInternal();
+}
+});
+}
+}
+
+@Override
+protected void maybeRestartTasks(final FailureHandlingResult 
failureHandlingResult) {
+FailureHandlingResult wrappedResult = failureHandlingResult;
+if (failureHandlingResult.canRestart()) {
+Set originalNeedToRestartVertices =
+failureHandlingResult.getVerticesToRestart();
+
+Set extraNeedToRestartJobVertices =
+originalNeedToRestartVertices.stream()
+.map(ExecutionVertexID::getJobVertexId)
+.filter(requiredRestartJobVertices::contains)
+.collect(Collectors.toSet());
+
+
requiredRestartJobVertices.removeAll(extraNeedToRestartJobVertices);
+
+Set needToRestartVertices =
+extraNeedToRestartJobVertices.stream()
+.flatMap(
+jobVertexId -> {
+ExecutionJobVertex jobVertex =
+
getExecutionJobVertex(jobVertexId);
+return 
Arrays.stream(jobVertex.getTaskVertices())
+.map(ExecutionVertex::getID);
+})
+.collect(Collectors.toSet());
+needToRestartVertices.addAll(originalNeedToRestartVertices);
+
+wrappedResult =
+FailureHandlingResult.restartable(
+
failureHandlingResult.getFailedExecution().orElse(null),
+failureHandlingResult.getError(),
+failureHandlingResult.getTimestamp(),
+

Re: [PR] [FLINK-33892][runtime] Support Job Recovery from JobMaster Failures for Batch Jobs. [flink]

2024-05-21 Thread via GitHub


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


##
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IOMetrics.java:
##
@@ -145,6 +143,8 @@ public long getAccumulateIdleTime() {
 }
 
 public Map 
getResultPartitionBytes() {
-return Collections.unmodifiableMap(checkNotNull(resultPartitionBytes));
+return resultPartitionBytes == null

Review Comment:
   In which case it can be `null` when this method is used?



##
flink-core-api/src/main/java/org/apache/flink/util/function/ConsumerWithException.java:
##
@@ -0,0 +1,56 @@
+/*
+ * 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.util.function;
+
+import java.util.function.Consumer;
+
+/**
+ * A checked extension of the {@link Consumer} interface.
+ *
+ * @param  type of the argument
+ * @param  type of the thrown exception
+ */
+@FunctionalInterface
+public interface ConsumerWithException {
+
+/**
+ * Performs this operation on the given arguments.
+ *
+ * @param t the first input argument

Review Comment:
   arguments -> argument
   
   the first input argument -> the input argument



##
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java:
##
@@ -499,8 +516,12 @@ public CompletableFuture 
registerProducedPartitions(TaskManagerLocation lo
 });
 }
 
-private static int getPartitionMaxParallelism(IntermediateResultPartition 
partition) {
-return partition.getIntermediateResult().getConsumersMaxParallelism();
+public static ResultPartitionDeploymentDescriptor 
createResultPartitionDeploymentDescriptor(
+IntermediateResultPartition partition, ShuffleDescriptor 
shuffleDescriptor) {

Review Comment:
   I prefer to reuse the `partitionDescriptor` which was already created.
   Looks to me that the creation of `partitionDescriptor` is not very light 
weighted(e.g. `isNumberOfPartitionConsumersUndefined()`) so I would avoid doing 
it twice if possible.



##
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java:
##
@@ -453,6 +453,26 @@ public CompletableFuture 
registerProducedPartitions(TaskManagerLocation lo
 });
 }
 
+private void recoverAttempt(ExecutionAttemptID newId) {
+if (!this.attemptId.equals(newId)) {
+getVertex().getExecutionGraphAccessor().deregisterExecution(this);
+this.attemptId = newId;
+getVertex().getExecutionGraphAccessor().registerExecution(this);
+}
+}
+
+/** Recover the execution vertex status after JM failover. */
+public void recoverExecution(ExecutionAttemptID attemptID, 
TaskManagerLocation location) {

Review Comment:
   execution vertex -> execution attempt
   
   attemptID -> attemptId



##
flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContext.java:
##
@@ -229,6 +229,10 @@ void sendEventToSourceOperator(int subtaskId, 
OperatorEvent event) {
 String.format("Failed to send event %s to subtask %d", event, 
subtaskId));
 }
 
+ScheduledExecutorService getCoordinatorExecutor() {

Review Comment:
   better to annotate it with `@VisibleForTesting`.



##
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/BatchJobRecoveryHandler.java:
##
@@ -0,0 +1,143 @@
+/*
+ * 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.
+ * 

Re: [PR] [FLINK-33892][runtime] Support Job Recovery from JobMaster Failures for Batch Jobs. [flink]

2024-05-21 Thread via GitHub


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


##
flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java:
##
@@ -1931,53 +1931,133 @@ void testGetAllPartitionWithMetrics() throws Exception 
{
 
 NettyShuffleDescriptor shuffleDescriptor =
 NettyShuffleDescriptorBuilder.newBuilder().buildLocal();
-DefaultShuffleMetrics shuffleMetrics =
+DefaultShuffleMetrics shuffleMetrics1 =
 new DefaultShuffleMetrics(new ResultPartitionBytes(new 
long[] {1, 2, 3}));
-Collection defaultPartitionWithMetrics =
+Collection defaultPartitionWithMetrics1 =
 Collections.singletonList(
-new DefaultPartitionWithMetrics(shuffleDescriptor, 
shuffleMetrics));
-final TestingTaskExecutorGateway taskExecutorGateway =
+new DefaultPartitionWithMetrics(shuffleDescriptor, 
shuffleMetrics1));
+DefaultShuffleMetrics shuffleMetrics2 =
+new DefaultShuffleMetrics(new ResultPartitionBytes(new 
long[] {4, 5, 6}));
+Collection defaultPartitionWithMetrics2 =
+Collections.singletonList(
+new DefaultPartitionWithMetrics(shuffleDescriptor, 
shuffleMetrics2));
+DefaultShuffleMetrics shuffleMetrics3 =
+new DefaultShuffleMetrics(new ResultPartitionBytes(new 
long[] {7, 8, 9}));
+Collection defaultPartitionWithMetrics3 =
+Collections.singletonList(
+new DefaultPartitionWithMetrics(shuffleDescriptor, 
shuffleMetrics3));
+
+// start fetch and retain partitions and then register tm1
+final TestingTaskExecutorGateway taskExecutorGateway1 =
 new TestingTaskExecutorGatewayBuilder()
 .setRequestPartitionWithMetricsFunction(
 ignored ->
 CompletableFuture.completedFuture(
-
defaultPartitionWithMetrics))
+
defaultPartitionWithMetrics1))
+.setAddress("tm1")
 .createTestingTaskExecutorGateway();
 
-final LocalUnresolvedTaskManagerLocation 
taskManagerUnresolvedLocation =
-new LocalUnresolvedTaskManagerLocation();
-final Collection slotOffers =
-registerSlotsAtJobMaster(
-1,
-jobMasterGateway,
-jobGraph.getJobID(),
-taskExecutorGateway,
-taskManagerUnresolvedLocation);
-assertThat(slotOffers).hasSize(1);
+registerSlotsAtJobMaster(
+1,
+jobMasterGateway,
+jobGraph.getJobID(),
+taskExecutorGateway1,
+new LocalUnresolvedTaskManagerLocation());
 
-waitUntilAllExecutionsAreScheduledOrDeployed(jobMasterGateway);
+jobMaster.startFetchAndRetainPartitionWithMetricsOnTaskManager();
 
-PartitionWithMetrics metrics =
-jobMasterGateway
-.getAllPartitionWithMetricsOnTaskManagers()
-.get()
-.iterator()
-.next();
-PartitionWithMetrics expectedMetrics = 
defaultPartitionWithMetrics.iterator().next();
-
-
assertThat(metrics.getPartitionMetrics().getPartitionBytes().getSubpartitionBytes())
-.isEqualTo(
-expectedMetrics
-.getPartitionMetrics()
-.getPartitionBytes()
-.getSubpartitionBytes());
-assertThat(metrics.getPartition().getResultPartitionID())
-
.isEqualTo(expectedMetrics.getPartition().getResultPartitionID());
-assertThat(metrics.getPartition().isUnknown())
-.isEqualTo(expectedMetrics.getPartition().isUnknown());
-assertThat(metrics.getPartition().storesLocalResourcesOn())
-
.isEqualTo(expectedMetrics.getPartition().storesLocalResourcesOn());
-}
+verifyPartitionMetrics(jobMasterGateway, 
defaultPartitionWithMetrics1);
+
+// register tm2
+TestingTaskExecutorGateway taskExecutorGateway2 =
+new TestingTaskExecutorGatewayBuilder()
+.setRequestPartitionWithMetricsFunction(
+ignored ->
+   

Re: [PR] [FLINK-33892][runtime] Support Job Recovery from JobMaster Failures for Batch Jobs. [flink]

2024-05-14 Thread via GitHub


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


##
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/AdaptiveBatchScheduler.java:
##
@@ -225,17 +308,831 @@ protected void startSchedulingInternal() {
 speculativeExecutionHandler.init(
 getExecutionGraph(), getMainThreadExecutor(), 
jobManagerJobMetricGroup);
 
+final boolean needRecover;
+if (isJobRecoveryEnabled) {
+try {
+needRecover = !jobEventManager.isEmpty();
+jobEventManager.start();
+} catch (Throwable throwable) {
+this.handleGlobalFailure(new 
SuppressRestartsException(throwable));
+return;
+}
+} else {
+needRecover = false;
+}
+
 tryComputeSourceParallelismThenRunAsync(
 (Void value, Throwable throwable) -> {
 if (getExecutionGraph().getState() == JobStatus.CREATED) {
-initializeVerticesIfPossible();
-super.startSchedulingInternal();
+if (needRecover) {
+
shuffleMaster.notifyPartitionRecoveryStarted(getJobId());
+getMainThreadExecutor()
+.schedule(
+this::recoverAndStartScheduling,
+
previousWorkerRecoveryTimeout.toMillis(),
+TimeUnit.MILLISECONDS);
+} else {
+initializeVerticesIfPossible();
+super.startSchedulingInternal();
+}
 }
 });
 }
 
+private void recoverAndStartScheduling() {

Review Comment:
   Is it possible to extract the job recovery related logic to a separate class?



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



Re: [PR] [FLINK-33892][runtime] Support Job Recovery from JobMaster Failures for Batch Jobs. [flink]

2024-05-11 Thread via GitHub


flinkbot commented on PR #24771:
URL: https://github.com/apache/flink/pull/24771#issuecomment-2105619645

   
   ## CI report:
   
   * 77f15330f3056aaa256565f7fc800787598a6c6a UNKNOWN
   
   
   Bot commands
 The @flinkbot bot supports the following commands:
   
- `@flinkbot run azure` re-run the last Azure build
   


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