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


##########
flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobIntermediateDatasetReuseTest.java:
##########
@@ -0,0 +1,284 @@
+/*
+ * 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.jobmaster;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.runtime.execution.Environment;
+import org.apache.flink.runtime.io.network.api.reader.RecordReader;
+import org.apache.flink.runtime.io.network.api.writer.RecordWriter;
+import org.apache.flink.runtime.io.network.api.writer.RecordWriterBuilder;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.minicluster.TestingMiniCluster;
+import org.apache.flink.runtime.minicluster.TestingMiniClusterConfiguration;
+import 
org.apache.flink.runtime.scheduler.CachedIntermediateDataSetCorruptedException;
+import org.apache.flink.types.IntValue;
+
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.CompletableFuture;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+/** Integration tests for reusing persisted intermediate dataset */
+public class JobIntermediateDatasetReuseTest {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(JobIntermediateDatasetReuseTest.class);
+
+    @Test
+    public void testClusterPartitionReuse() throws Exception {
+        internalTestClusterPartitionReuse(1);
+    }
+
+    @Test
+    public void testClusterPartitionReuseMultipleParallelism() throws 
Exception {
+        internalTestClusterPartitionReuse(64);
+    }
+
+    private void internalTestClusterPartitionReuse(int parallelism) throws 
Exception {
+        final TestingMiniClusterConfiguration miniClusterConfiguration =
+                TestingMiniClusterConfiguration.newBuilder().build();
+
+        try (TestingMiniCluster miniCluster =
+                
TestingMiniCluster.newBuilder(miniClusterConfiguration).build()) {
+            miniCluster.start();
+
+            IntermediateDataSetID intermediateDataSetID = new 
IntermediateDataSetID();
+            final JobGraph firstJobGraph = createFirstJobGraph(parallelism, 
intermediateDataSetID);
+            miniCluster.submitJob(firstJobGraph).get();
+            CompletableFuture<JobResult> jobResultFuture =
+                    miniCluster.requestJobResult(firstJobGraph.getJobID());
+            JobResult jobResult = jobResultFuture.get();
+            assertTrue(jobResult.isSuccess());
+
+            final JobGraph secondJobGraph =
+                    createSecondJobGraph(parallelism, intermediateDataSetID);
+            miniCluster.submitJob(secondJobGraph).get();
+            jobResultFuture = 
miniCluster.requestJobResult(secondJobGraph.getJobID());
+            jobResult = jobResultFuture.get();
+            assertTrue(jobResult.isSuccess());
+        }
+    }
+
+    @Test
+    public void 
testClusterPartitionReuseWithMoreConsumerParallelismThrowException()
+            throws Exception {
+        final TestingMiniClusterConfiguration miniClusterConfiguration =
+                TestingMiniClusterConfiguration.newBuilder().build();
+
+        try (TestingMiniCluster miniCluster =
+                
TestingMiniCluster.newBuilder(miniClusterConfiguration).build()) {
+            miniCluster.start();
+
+            IntermediateDataSetID intermediateDataSetID = new 
IntermediateDataSetID();
+            final JobGraph firstJobGraph = createFirstJobGraph(1, 
intermediateDataSetID);
+            miniCluster.submitJob(firstJobGraph).get();
+            CompletableFuture<JobResult> jobResultFuture =
+                    miniCluster.requestJobResult(firstJobGraph.getJobID());
+            JobResult jobResult = jobResultFuture.get();
+            assertTrue(jobResult.isSuccess());
+
+            final JobGraph secondJobGraph = createSecondJobGraph(2, 
intermediateDataSetID);
+            miniCluster.submitJob(secondJobGraph).get();
+            jobResultFuture = 
miniCluster.requestJobResult(secondJobGraph.getJobID());
+            jobResult = jobResultFuture.get();
+            assertFalse(jobResult.isSuccess());
+            assertTrue(jobResult.getSerializedThrowable().isPresent());

Review Comment:
   we can also make this verification a param `Consumer< JobResult >` of 
`internalTestClusterPartitionReuse`.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImpl.java:
##########
@@ -120,6 +129,31 @@ public Collection<ResultPartitionDeploymentDescriptor> 
getAllTrackedPartitions()
         return 
partitionInfos.values().stream().map(PartitionInfo::getMetaInfo).collect(toList());
     }
 
+    @Override
+    public void connectToResourceManager(ResourceManagerGateway 
resourceManagerGateway) {
+        this.resourceManagerGateway = resourceManagerGateway;
+    }
+
+    @Override
+    public List<ShuffleDescriptor> getClusterPartitionShuffleDescriptors(
+            IntermediateDataSetID intermediateDataSetID) {
+        return clusterPartitionShuffleDescriptors.computeIfAbsent(
+                intermediateDataSetID, 
this::requestShuffleDescriptorsFromResourceManager);
+    }
+
+    private List<ShuffleDescriptor> 
requestShuffleDescriptorsFromResourceManager(
+            IntermediateDataSetID intermediateDataSetID) {
+        Preconditions.checkNotNull(
+                resourceManagerGateway, "JobMaster is not connected to 
ResourceManager");
+        try {
+            return this.resourceManagerGateway
+                    
.getClusterPartitionsShuffleDescriptors(intermediateDataSetID)
+                    .get();
+        } catch (InterruptedException | ExecutionException e) {
+            throw new RuntimeException(e);

Review Comment:
   I would add a message to explain that this error happens when trying to find 
cached intermediate data sets, to better distinguish the cause of 
`CachedIntermediateDataSetCorruptedException` (currently there are 2 different 
causes).



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImpl.java:
##########
@@ -120,6 +129,31 @@ public Collection<ResultPartitionDeploymentDescriptor> 
getAllTrackedPartitions()
         return 
partitionInfos.values().stream().map(PartitionInfo::getMetaInfo).collect(toList());
     }
 
+    @Override
+    public void connectToResourceManager(ResourceManagerGateway 
resourceManagerGateway) {
+        this.resourceManagerGateway = resourceManagerGateway;
+    }
+
+    @Override
+    public List<ShuffleDescriptor> getClusterPartitionShuffleDescriptors(
+            IntermediateDataSetID intermediateDataSetID) {
+        return clusterPartitionShuffleDescriptors.computeIfAbsent(
+                intermediateDataSetID, 
this::requestShuffleDescriptorsFromResourceManager);
+    }
+
+    private List<ShuffleDescriptor> 
requestShuffleDescriptorsFromResourceManager(
+            IntermediateDataSetID intermediateDataSetID) {
+        Preconditions.checkNotNull(
+                resourceManagerGateway, "JobMaster is not connected to 
ResourceManager");
+        try {
+            return this.resourceManagerGateway
+                    
.getClusterPartitionsShuffleDescriptors(intermediateDataSetID)
+                    .get();
+        } catch (InterruptedException | ExecutionException e) {
+            throw new RuntimeException(e);

Review Comment:
   I would also catch all throwable here.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandler.java:
##########
@@ -115,6 +122,29 @@ private FailureHandlingResult handleFailure(
             final Set<ExecutionVertexID> verticesToRestart,
             final boolean globalFailure) {
 
+        if (!globalFailure) {

Review Comment:
   I would propose to do the translation of the partition exception of a cached 
result in `DefaultScheduler` and keep `ExecutionFailureHandler` as is. The 
benefits are:
   1. the `ExecutionFailureHandler` does not need to understand exceptions.
   2. no need to modify `DefaultExecutionVertex`. The scheduler can have a 
method to check if a partition is from a cached result.
   3. no need to distinguish if it is a global failure or not. Actually even if 
it is a global failure, `CachedIntermediateDataSetCorruptedException` should 
also fail the job in unrecoverable way.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to