Thesharing commented on a change in pull request #16688: URL: https://github.com/apache/flink/pull/16688#discussion_r685313909
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/LogicalEdge.java ########## @@ -0,0 +1,34 @@ +/* + * 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.jobgraph.topology; + +import org.apache.flink.runtime.jobgraph.DistributionPattern; +import org.apache.flink.runtime.jobgraph.IntermediateDataSet; +import org.apache.flink.runtime.jobgraph.JobEdge; +import org.apache.flink.runtime.jobgraph.JobVertex; + +/** Represents a vertex in {@link LogicalTopology}, i.e. {@link JobEdge}. */ Review comment: Thank you for pointing this out. Resolved. ########## 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: Should we make this `!isReconnectable()`? ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/LogicalEdge.java ########## @@ -0,0 +1,34 @@ +/* + * 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.jobgraph.topology; + +import org.apache.flink.runtime.jobgraph.DistributionPattern; +import org.apache.flink.runtime.jobgraph.IntermediateDataSet; +import org.apache.flink.runtime.jobgraph.JobEdge; +import org.apache.flink.runtime.jobgraph.JobVertex; + +/** Represents a vertex in {@link LogicalTopology}, i.e. {@link JobEdge}. */ +public interface LogicalEdge { + + DistributionPattern getDistributionPattern(); + + JobVertex getTarget(); Review comment: I'm wondering that if we introduce `getTarget` and `getSource` method for `LogicalEdge` interface, we can make sure `DefaultLogicalResult` and `DefaultLogicalVertex` doesn't have full access to `JobEdge`, would it be better? ########## 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, Review comment: Resolved. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/DefaultExecutionTopology.java ########## @@ -243,13 +289,57 @@ private static DefaultExecutionVertex generateSchedulingExecutionVertex( } private static IndexedPipelinedRegions computePipelinedRegions( - Iterable<DefaultExecutionVertex> topologicallySortedVertexes, + Iterable<DefaultLogicalPipelinedRegion> logicalPipelinedRegions, + Function<DefaultLogicalPipelinedRegion, List<DefaultExecutionVertex>> + sortedExecutionVerticesInPipelinedRegion, + Function<ExecutionVertexID, DefaultExecutionVertex> executionVertexRetriever, Function<IntermediateResultPartitionID, DefaultResultPartition> resultPartitionRetriever) { + long buildRegionsStartTime = System.nanoTime(); Set<Set<SchedulingExecutionVertex>> rawPipelinedRegions = - PipelinedRegionComputeUtil.computePipelinedRegions(topologicallySortedVertexes); + Collections.newSetFromMap(new IdentityHashMap<>()); + + // A SchedulingPipelinedRegion can be derived from only one LogicalPipelinedRegion. + // Thus, we can traverse all LogicalPipelinedRegions and convert them into + // SchedulingPipelinedRegions one by one. The LogicalPipelinedRegions and + // SchedulingPipelinedRegions are both connected with inter-region blocking edges. + for (DefaultLogicalPipelinedRegion logicalPipelinedRegion : logicalPipelinedRegions) { + + List<DefaultExecutionVertex> schedulingExecutionVertices = + sortedExecutionVerticesInPipelinedRegion.apply(logicalPipelinedRegion); + + if (containsIntraRegionAllToAllEdge(logicalPipelinedRegion)) { + // For edges inside one LogicalPipelinedRegion, if there is any all-to-all edge, it + // could be under two circumstances: + // + // 1. Pipelined all-to-all edge: + // Pipelined all-to-all edge will connect all vertices connected by the edge, + // and merges all the SchedulingPipelinedRegions derived from this + // LogicalPipelinedRegion into one sole region. Review comment: Thank you for proposing a better description. Resolved. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/DefaultExecutionTopology.java ########## @@ -243,13 +289,57 @@ private static DefaultExecutionVertex generateSchedulingExecutionVertex( } private static IndexedPipelinedRegions computePipelinedRegions( - Iterable<DefaultExecutionVertex> topologicallySortedVertexes, + Iterable<DefaultLogicalPipelinedRegion> logicalPipelinedRegions, + Function<DefaultLogicalPipelinedRegion, List<DefaultExecutionVertex>> + sortedExecutionVerticesInPipelinedRegion, + Function<ExecutionVertexID, DefaultExecutionVertex> executionVertexRetriever, Function<IntermediateResultPartitionID, DefaultResultPartition> resultPartitionRetriever) { + long buildRegionsStartTime = System.nanoTime(); Set<Set<SchedulingExecutionVertex>> rawPipelinedRegions = - PipelinedRegionComputeUtil.computePipelinedRegions(topologicallySortedVertexes); + Collections.newSetFromMap(new IdentityHashMap<>()); + + // A SchedulingPipelinedRegion can be derived from only one LogicalPipelinedRegion. Review comment: Resolved. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/LogicalEdge.java ########## @@ -0,0 +1,34 @@ +/* + * 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.jobgraph.topology; + +import org.apache.flink.runtime.jobgraph.DistributionPattern; +import org.apache.flink.runtime.jobgraph.IntermediateDataSet; +import org.apache.flink.runtime.jobgraph.JobEdge; +import org.apache.flink.runtime.jobgraph.JobVertex; + +/** Represents a vertex in {@link LogicalTopology}, i.e. {@link JobEdge}. */ +public interface LogicalEdge { + + DistributionPattern getDistributionPattern(); Review comment: Added. I forgot to push the last fix-up commit 😆 ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/LogicalPipelinedRegionComputeUtil.java ########## @@ -0,0 +1,113 @@ +/* + * 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.topology.LogicalPipelinedRegion; +import org.apache.flink.runtime.jobgraph.topology.LogicalResult; +import org.apache.flink.runtime.jobgraph.topology.LogicalVertex; + +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 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 LogicalPipelinedRegion}s. */ +public final class LogicalPipelinedRegionComputeUtil { + public static Set<Set<LogicalVertex>> computePipelinedRegions( + final Iterable<? extends LogicalVertex> topologicallySortedVertexes) { Review comment: Sorry for the typo. It's weird IntelliJ doesn't find this out. ########## 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, Review comment: Resolved. ########## 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; Review comment: In `SchedulingPipelinedRegionComputeUtil`, `producerRegion` can be null, while in `LogicalPipelinedRegionComputUtil`, `producerRegion` cannot be null. Should we merge it? ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/DefaultExecutionTopology.java ########## @@ -243,13 +289,57 @@ private static DefaultExecutionVertex generateSchedulingExecutionVertex( } private static IndexedPipelinedRegions computePipelinedRegions( - Iterable<DefaultExecutionVertex> topologicallySortedVertexes, + Iterable<DefaultLogicalPipelinedRegion> logicalPipelinedRegions, + Function<DefaultLogicalPipelinedRegion, List<DefaultExecutionVertex>> + sortedExecutionVerticesInPipelinedRegion, + Function<ExecutionVertexID, DefaultExecutionVertex> executionVertexRetriever, Function<IntermediateResultPartitionID, DefaultResultPartition> resultPartitionRetriever) { + long buildRegionsStartTime = System.nanoTime(); Set<Set<SchedulingExecutionVertex>> rawPipelinedRegions = - PipelinedRegionComputeUtil.computePipelinedRegions(topologicallySortedVertexes); + Collections.newSetFromMap(new IdentityHashMap<>()); + + // A SchedulingPipelinedRegion can be derived from only one LogicalPipelinedRegion. + // Thus, we can traverse all LogicalPipelinedRegions and convert them into + // SchedulingPipelinedRegions one by one. The LogicalPipelinedRegions and + // SchedulingPipelinedRegions are both connected with inter-region blocking edges. + for (DefaultLogicalPipelinedRegion logicalPipelinedRegion : logicalPipelinedRegions) { + + List<DefaultExecutionVertex> schedulingExecutionVertices = + sortedExecutionVerticesInPipelinedRegion.apply(logicalPipelinedRegion); + + if (containsIntraRegionAllToAllEdge(logicalPipelinedRegion)) { + // For edges inside one LogicalPipelinedRegion, if there is any all-to-all edge, it + // could be under two circumstances: + // + // 1. Pipelined all-to-all edge: + // Pipelined all-to-all edge will connect all vertices connected by the edge, + // and merges all the SchedulingPipelinedRegions derived from this + // LogicalPipelinedRegion into one sole region. + // + // 2. Blocking all-to-all edge: + // For intra-region blocking all-to-all edge, we must make sure all the vertices + // are inside one SchedulingPipelinedRegions, so that there will be no deadlock Review comment: Sorry for the typo. Fixed. ########## 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()) { + continue; + } + for (ConsumerVertexGroup consumerVertexGroup : + producedResult.getConsumerVertexGroups()) { + for (ExecutionVertexID consumerVertexId : consumerVertexGroup) { + SchedulingExecutionVertex consumerVertex = + executionVertexRetriever.apply(consumerVertexId); + if (vertexToRegion.containsKey(consumerVertex) Review comment: Yes, can't agree more. Resolved. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/LogicalPipelinedRegionComputeUtil.java ########## @@ -0,0 +1,113 @@ +/* + * 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.topology.LogicalPipelinedRegion; +import org.apache.flink.runtime.jobgraph.topology.LogicalResult; +import org.apache.flink.runtime.jobgraph.topology.LogicalVertex; + +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 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 LogicalPipelinedRegion}s. */ +public final class LogicalPipelinedRegionComputeUtil { + public static Set<Set<LogicalVertex>> computePipelinedRegions( + final Iterable<? extends LogicalVertex> topologicallySortedVertexes) { + final Map<LogicalVertex, Set<LogicalVertex>> vertexToRegion = + buildRawRegions(topologicallySortedVertexes); + return mergeRegionsOnCycles( + vertexToRegion, + (currentRegion, regionIndices) -> + buildOutEdgesForRegion(currentRegion, regionIndices, vertexToRegion)); + } + + private static Map<LogicalVertex, Set<LogicalVertex>> buildRawRegions( + final Iterable<? extends LogicalVertex> topologicallySortedVertexes) { Review comment: Resolved. -- 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]
