zhuzhurk commented on a change in pull request #16688:
URL: https://github.com/apache/flink/pull/16688#discussion_r685785327



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/SchedulingPipelinedRegionComputeUtil.java
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.executiongraph.failover.flip1;
+
+import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
+import org.apache.flink.runtime.scheduler.strategy.ConsumedPartitionGroup;
+import org.apache.flink.runtime.scheduler.strategy.ConsumerVertexGroup;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.scheduler.strategy.SchedulingExecutionVertex;
+import org.apache.flink.runtime.scheduler.strategy.SchedulingPipelinedRegion;
+import org.apache.flink.runtime.scheduler.strategy.SchedulingResultPartition;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+
+import static 
org.apache.flink.runtime.executiongraph.failover.flip1.PipelinedRegionComputeUtil.mergeRegions;
+import static 
org.apache.flink.runtime.executiongraph.failover.flip1.PipelinedRegionComputeUtil.mergeRegionsOnCycles;
+
+/** Utils for computing {@link SchedulingPipelinedRegion}s. */
+public final class SchedulingPipelinedRegionComputeUtil {
+
+    public static Set<Set<SchedulingExecutionVertex>> computePipelinedRegions(
+            final Iterable<? extends SchedulingExecutionVertex> 
topologicallySortedVertexes,
+            final Function<ExecutionVertexID, ? extends 
SchedulingExecutionVertex>
+                    executionVertexRetriever,
+            final Function<IntermediateResultPartitionID, ? extends 
SchedulingResultPartition>
+                    resultPartitionRetriever) {
+
+        final Map<SchedulingExecutionVertex, Set<SchedulingExecutionVertex>> 
vertexToRegion =
+                buildRawRegions(topologicallySortedVertexes, 
resultPartitionRetriever);
+
+        return mergeRegionsOnCycles(
+                vertexToRegion,
+                (currentRegion, regionIndices) ->
+                        buildOutEdgesForRegion(
+                                currentRegion,
+                                regionIndices,
+                                vertexToRegion,
+                                executionVertexRetriever));
+    }
+
+    private static Map<SchedulingExecutionVertex, 
Set<SchedulingExecutionVertex>> buildRawRegions(
+            final Iterable<? extends SchedulingExecutionVertex> 
topologicallySortedVertexes,
+            final Function<IntermediateResultPartitionID, ? extends 
SchedulingResultPartition>
+                    resultPartitionRetriever) {
+
+        final Map<SchedulingExecutionVertex, Set<SchedulingExecutionVertex>> 
vertexToRegion =
+                new IdentityHashMap<>();
+
+        // iterate all the vertices which are topologically sorted
+        for (SchedulingExecutionVertex vertex : topologicallySortedVertexes) {
+            Set<SchedulingExecutionVertex> currentRegion = new HashSet<>();
+            currentRegion.add(vertex);
+            vertexToRegion.put(vertex, currentRegion);
+
+            for (ConsumedPartitionGroup consumedPartitionGroup :
+                    vertex.getConsumedPartitionGroups()) {
+                for (IntermediateResultPartitionID consumedPartitionId : 
consumedPartitionGroup) {
+                    SchedulingResultPartition consumedPartition =
+                            
resultPartitionRetriever.apply(consumedPartitionId);
+                    // Similar to the BLOCKING ResultPartitionType, each 
vertex connected through
+                    // PIPELINED_APPROXIMATE is also considered as a single 
region. This attribute
+                    // is called  "reconnectable". Reconnectable will be 
removed after FLINK-19895,
+                    // see also {@link  ResultPartitionType#isReconnectable}
+                    if (!consumedPartition.getResultType().isReconnectable()) {
+                        final SchedulingExecutionVertex producerVertex =
+                                consumedPartition.getProducer();
+                        final Set<SchedulingExecutionVertex> producerRegion =
+                                vertexToRegion.get(producerVertex);
+
+                        // check if it is the same as the producer region, if 
so skip the merge
+                        // this check can significantly reduce compute 
complexity in All-to-All
+                        // PIPELINED edge case
+                        if (producerRegion != null && currentRegion != 
producerRegion) {
+                            currentRegion =
+                                    mergeRegions(currentRegion, 
producerRegion, vertexToRegion);
+                        }
+                    } else {
+                        break;
+                    }
+                }
+            }
+        }
+
+        return vertexToRegion;
+    }
+
+    private static List<Integer> buildOutEdgesForRegion(
+            final Set<SchedulingExecutionVertex> currentRegion,
+            final Map<Set<SchedulingExecutionVertex>, Integer> regionIndices,
+            final Map<SchedulingExecutionVertex, 
Set<SchedulingExecutionVertex>> vertexToRegion,
+            final Function<ExecutionVertexID, ? extends 
SchedulingExecutionVertex>
+                    executionVertexRetriever) {
+
+        final List<Integer> currentRegionOutEdges = new ArrayList<>();
+        for (SchedulingExecutionVertex vertex : currentRegion) {
+            for (SchedulingResultPartition producedResult : 
vertex.getProducedResults()) {
+                if (producedResult.getResultType().isPipelined()) {

Review comment:
       yes of course




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