Thesharing commented on a change in pull request #14868:
URL: https://github.com/apache/flink/pull/14868#discussion_r581001291
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java
##########
@@ -32,25 +32,24 @@
private final IntermediateResultPartitionID partitionId;
- private List<List<ExecutionEdge>> consumers;
-
/** Whether this partition has produced some data. */
private boolean hasDataProduced = false;
public IntermediateResultPartition(
IntermediateResult totalResult, ExecutionVertex producer, int
partitionNumber) {
this.totalResult = totalResult;
this.producer = producer;
- this.consumers = new ArrayList<List<ExecutionEdge>>(0);
this.partitionId = new
IntermediateResultPartitionID(totalResult.getId(), partitionNumber);
+
+ producer.getExecutionGraph().registerResultPartition(partitionId,
this);
}
public ExecutionVertex getProducer() {
return producer;
}
public int getPartitionNumber() {
- return partitionId.getPartitionNum();
+ return partitionId.getPartitionNumber();
Review comment:
Yes, I'll rearrange it once the PR is ready.
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/EdgeManagerBuildUtil.java
##########
@@ -0,0 +1,161 @@
+/*
+ * 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;
+
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+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 java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/** Utilities for building {@link EdgeManager}. */
+public class EdgeManagerBuildUtil {
+
+ public static void connectVertexToResult(
+ ExecutionVertex[] taskVertices,
+ IntermediateResult ires,
+ int inputNumber,
+ DistributionPattern distributionPattern) {
+
+ switch (distributionPattern) {
+ case POINTWISE:
+ connectPointwise(taskVertices, ires, inputNumber);
+ break;
+ case ALL_TO_ALL:
+ connectAllToAll(taskVertices, ires, inputNumber);
+ break;
+ default:
+ throw new RuntimeException("Unrecognized distribution
pattern.");
+ }
+ }
+
+ private static void connectAllToAll(
+ ExecutionVertex[] taskVertices, IntermediateResult ires, int
inputNumber) {
+
+ ConsumedPartitionGroup consumedPartitions =
+ new ConsumedPartitionGroup(
+ Arrays.stream(ires.getPartitions())
+
.map(IntermediateResultPartition::getPartitionId)
+ .collect(Collectors.toList()));
+ for (ExecutionVertex ev : taskVertices) {
+ ev.addConsumedPartitions(consumedPartitions, inputNumber);
+ }
+
+ ConsumerVertexGroup vertices =
+ new ConsumerVertexGroup(
+ Arrays.stream(taskVertices)
+ .map(ExecutionVertex::getID)
+ .collect(Collectors.toList()));
+ for (IntermediateResultPartition partition : ires.getPartitions()) {
+ partition.addConsumers(vertices);
+ }
+ }
+
+ private static void connectPointwise(
+ ExecutionVertex[] taskVertices, IntermediateResult ires, int
inputNumber) {
+
+ final int sourceCount = ires.getPartitions().length;
+ final int targetCount = taskVertices.length;
+
+ if (sourceCount == targetCount) {
+ for (int i = 0; i < sourceCount; i++) {
+ ExecutionVertex executionVertex = taskVertices[i];
+ IntermediateResultPartition partition =
ires.getPartitions()[i];
+
+ ConsumerVertexGroup consumerVertexGroup =
+ new ConsumerVertexGroup(executionVertex.getID());
+ partition.addConsumers(consumerVertexGroup);
+
+ ConsumedPartitionGroup consumedPartitionGroup =
+ new ConsumedPartitionGroup(partition.getPartitionId());
+ executionVertex.addConsumedPartitions(consumedPartitionGroup,
inputNumber);
+ }
+ } else if (sourceCount > targetCount) {
+ for (int index = 0; index < targetCount; index++) {
+
+ ExecutionVertex executionVertex = taskVertices[index];
+ ConsumerVertexGroup consumerVertexGroup =
+ new ConsumerVertexGroup(executionVertex.getID());
+
+ int start = index * sourceCount / targetCount;
+ int end = (index + 1) * sourceCount / targetCount;
+
+ List<IntermediateResultPartitionID> consumedPartitions =
+ new ArrayList<>(end - start);
+
+ for (int i = start; i < end; i++) {
+ IntermediateResultPartition partition =
ires.getPartitions()[i];
+ partition.addConsumers(consumerVertexGroup);
+
+ consumedPartitions.add(partition.getPartitionId());
+ }
+
+ ConsumedPartitionGroup consumedPartitionGroup =
+ new ConsumedPartitionGroup(consumedPartitions);
+ executionVertex.addConsumedPartitions(consumedPartitionGroup,
inputNumber);
+ }
+ } else {
+ for (int partitionNum = 0; partitionNum < sourceCount;
partitionNum++) {
+
+ IntermediateResultPartition partition =
ires.getPartitions()[partitionNum];
+ ConsumedPartitionGroup consumerPartitionGroup =
+ new ConsumedPartitionGroup(partition.getPartitionId());
+
+ List<ExecutionVertexID> consumers = new
ArrayList<>(targetCount / sourceCount + 1);
+
+ if (targetCount % sourceCount == 0) {
+ int factor = targetCount / sourceCount;
Review comment:
Here, could we simplify the logic like this:
```java
int start = partitionNum * targetCount / sourceCount;
int end = (partitionNum + 1) * targetCount / sourceCount;
List<ExecutionVertexID> consumers = new ArrayList<>(end - start);
for (int i = start; i < end; i++) {
ExecutionVertex executionVertex = taskVertices[i];
executionVertex.addConsumedPartitions(consumerPartitionGroup,
inputNumber);
consumers.add(executionVertex.getID());
}
```
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/DistributionPattern.java
##########
@@ -18,29 +18,15 @@
package org.apache.flink.runtime.jobgraph;
-import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-
/**
* A distribution pattern determines, which sub tasks of a producing task are
connected to which
* consuming sub tasks.
Review comment:
Added. I'm wondering would it be better to say "how ExecutionVertex and
IntermediateResultPartition is connected in {@link EdgeManagerBuildUtil}"?
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/EdgeManagerBuildUtil.java
##########
@@ -0,0 +1,174 @@
+/*
+ * 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;
+
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+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 java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/** Utilities for building {@link EdgeManager}. */
+public class EdgeManagerBuildUtil {
+
+ public static void registerToExecutionEdgeManager(
+ ExecutionVertex[] taskVertices,
+ IntermediateResult ires,
+ int inputNumber,
+ DistributionPattern distributionPattern) {
+
+ switch (distributionPattern) {
+ case POINTWISE:
+ connectPointwise(taskVertices, ires, inputNumber);
+ break;
+ case ALL_TO_ALL:
+ connectAllToAll(taskVertices, ires, inputNumber);
+ break;
+ default:
+ throw new RuntimeException("Unrecognized distribution
pattern.");
+ }
+ }
+
+ private static void connectAllToAll(
+ ExecutionVertex[] taskVertices, IntermediateResult ires, int
inputNumber) {
+
+ ConsumedPartitionGroup consumedPartitions =
+ new ConsumedPartitionGroup(
+ Arrays.stream(ires.getPartitions())
+
.map(IntermediateResultPartition::getPartitionId)
+ .collect(Collectors.toList()));
+ for (ExecutionVertex ev : taskVertices) {
+ ev.setConsumedPartitions(consumedPartitions, inputNumber);
+ }
+
+ ConsumerVertexGroup vertices =
+ new ConsumerVertexGroup(
+ Arrays.stream(taskVertices)
+ .map(ExecutionVertex::getID)
+ .collect(Collectors.toList()));
+ for (IntermediateResultPartition partition : ires.getPartitions()) {
+ partition.setConsumers(vertices);
+ }
+ }
+
+ private static void connectPointwise(
+ ExecutionVertex[] taskVertices, IntermediateResult ires, int
inputNumber) {
+
+ final int sourceCount = ires.getPartitions().length;
+ final int targetCount = taskVertices.length;
+
+ if (sourceCount == targetCount) {
+ for (int i = 0; i < sourceCount; i++) {
+ ExecutionVertex executionVertex = taskVertices[i];
+ IntermediateResultPartition partition =
ires.getPartitions()[i];
+
+ ConsumerVertexGroup consumerVertexGroup =
+ new ConsumerVertexGroup(executionVertex.getID());
+ partition.setConsumers(consumerVertexGroup);
+
+ ConsumedPartitionGroup consumedPartitionGroup =
+ new ConsumedPartitionGroup(partition.getPartitionId());
+ executionVertex.setConsumedPartitions(consumedPartitionGroup,
inputNumber);
+ }
+ } else if (sourceCount > targetCount) {
+ for (int index = 0; index < targetCount; index++) {
+
+ ExecutionVertex executionVertex = taskVertices[index];
+ ConsumerVertexGroup consumerVertexGroup =
+ new ConsumerVertexGroup(executionVertex.getID());
+
+ List<IntermediateResultPartitionID> consumedPartitions =
+ new ArrayList<>(sourceCount / targetCount + 1);
+
+ if (sourceCount % targetCount == 0) {
Review comment:
I'm wondering could we replace the logic of `sourceCount < targetCount`
in this way?
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/ConsumerVertexGroup.java
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.strategy;
+
+import java.util.Collections;
+import java.util.List;
+
+/** Group of consumer {@link ExecutionVertexID}s. */
+public class ConsumerVertexGroup {
+ private final List<ExecutionVertexID> vertices;
+
+ public ConsumerVertexGroup(List<ExecutionVertexID> vertices) {
+ this.vertices = vertices;
+ }
+
+ public ConsumerVertexGroup(ExecutionVertexID vertex) {
+ this(Collections.singletonList(vertex));
+ }
+
+ public List<ExecutionVertexID> getVertices() {
+ return vertices;
Review comment:
Done.
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/ConsumedPartitionGroup.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.strategy;
+
+import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
+
+import java.util.Collections;
+import java.util.List;
+
+/** Group of consumed {@link IntermediateResultPartitionID}s. */
+public class ConsumedPartitionGroup {
+ private final List<IntermediateResultPartitionID> resultPartitions;
+
+ public ConsumedPartitionGroup(List<IntermediateResultPartitionID>
resultPartitions) {
+ this.resultPartitions = resultPartitions;
+ }
+
+ public ConsumedPartitionGroup(IntermediateResultPartitionID
resultPartition) {
+ this(Collections.singletonList(resultPartition));
+ }
+
+ public List<IntermediateResultPartitionID> getResultPartitions() {
+ return resultPartitions;
Review comment:
Agreed. Done.
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
##########
@@ -326,107 +328,11 @@ public ExecutionGraph getExecutionGraph() {
// Graph building
//
--------------------------------------------------------------------------------------------
- public void connectSource(
- int inputNumber, IntermediateResult source, JobEdge edge, int
consumerNumber) {
-
- final DistributionPattern pattern = edge.getDistributionPattern();
- final IntermediateResultPartition[] sourcePartitions =
source.getPartitions();
-
- ExecutionEdge[] edges;
-
- switch (pattern) {
- case POINTWISE:
- edges = connectPointwise(sourcePartitions, inputNumber);
- break;
-
- case ALL_TO_ALL:
- edges = connectAllToAll(sourcePartitions, inputNumber);
- break;
-
- default:
- throw new RuntimeException("Unrecognized distribution
pattern.");
- }
-
- inputEdges[inputNumber] = edges;
-
- // add the consumers to the source
- // for now (until the receiver initiated handshake is in place), we
need to register the
- // edges as the execution graph
- for (ExecutionEdge ee : edges) {
- ee.getSource().addConsumer(ee, consumerNumber);
- }
- }
-
- private ExecutionEdge[] connectAllToAll(
- IntermediateResultPartition[] sourcePartitions, int inputNumber) {
- ExecutionEdge[] edges = new ExecutionEdge[sourcePartitions.length];
-
- for (int i = 0; i < sourcePartitions.length; i++) {
- IntermediateResultPartition irp = sourcePartitions[i];
- edges[i] = new ExecutionEdge(irp, this, inputNumber);
- }
-
- return edges;
- }
-
- private ExecutionEdge[] connectPointwise(
- IntermediateResultPartition[] sourcePartitions, int inputNumber) {
- final int numSources = sourcePartitions.length;
- final int parallelism = getTotalNumberOfParallelSubtasks();
-
- // simple case same number of sources as targets
- if (numSources == parallelism) {
- return new ExecutionEdge[] {
- new ExecutionEdge(sourcePartitions[subTaskIndex], this,
inputNumber)
- };
- } else if (numSources < parallelism) {
-
- int sourcePartition;
-
- // check if the pattern is regular or irregular
- // we use int arithmetics for regular, and floating point with
rounding for irregular
- if (parallelism % numSources == 0) {
- // same number of targets per source
- int factor = parallelism / numSources;
- sourcePartition = subTaskIndex / factor;
- } else {
- // different number of targets per source
- float factor = ((float) parallelism) / numSources;
- sourcePartition = (int) (subTaskIndex / factor);
- }
-
- return new ExecutionEdge[] {
- new ExecutionEdge(sourcePartitions[sourcePartition], this,
inputNumber)
- };
- } else {
- if (numSources % parallelism == 0) {
- // same number of targets per source
- int factor = numSources / parallelism;
- int startIndex = subTaskIndex * factor;
-
- ExecutionEdge[] edges = new ExecutionEdge[factor];
- for (int i = 0; i < factor; i++) {
- edges[i] =
- new ExecutionEdge(sourcePartitions[startIndex +
i], this, inputNumber);
- }
- return edges;
- } else {
- float factor = ((float) numSources) / parallelism;
+ public void setConsumedPartitions(ConsumedPartitionGroup
consumedPartitions, int inputNum) {
Review comment:
Done.
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
##########
@@ -471,15 +475,15 @@ public void setInitialState(@Nullable
JobManagerTaskRestore taskRestore) {
});
}
- private static int getPartitionMaxParallelism(IntermediateResultPartition
partition) {
- final List<List<ExecutionEdge>> consumers = partition.getConsumers();
+ private static int getPartitionMaxParallelism(
Review comment:
I prefer using the `Function<>`, since there's no need to introduce the
entire `ExecutionGraph`. `ExecutionGraph#getIntermediateResultPartitionMapping`
and `ExecutionGraph#getExecutionVertexMapping` has been removed.
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/EdgeManagerBuildUtil.java
##########
@@ -0,0 +1,174 @@
+/*
+ * 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;
+
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+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 java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/** Utilities for building {@link EdgeManager}. */
+public class EdgeManagerBuildUtil {
+
+ public static void registerToExecutionEdgeManager(
Review comment:
Totally agreed. Done.
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java
##########
@@ -90,21 +89,12 @@ void resetForNewExecution() {
hasDataProduced = false;
}
- int addConsumerGroup() {
- int pos = consumers.size();
-
- // NOTE: currently we support only one consumer per result!!!
- if (pos != 0) {
- throw new RuntimeException(
- "Currently, each intermediate result can only have one
consumer.");
- }
-
- consumers.add(new ArrayList<ExecutionEdge>());
- return pos;
+ public void setConsumers(ConsumerVertexGroup consumers) {
+
producer.getExecutionGraph().getEdgeManager().addPartitionConsumers(partitionId,
consumers);
}
- void addConsumer(ExecutionEdge edge, int consumerNumber) {
- consumers.get(consumerNumber).add(edge);
+ EdgeManager getEdgeManager() {
Review comment:
Yes, it should be private. Thanks for point this out.
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java
##########
@@ -90,21 +89,12 @@ void resetForNewExecution() {
hasDataProduced = false;
}
- int addConsumerGroup() {
- int pos = consumers.size();
-
- // NOTE: currently we support only one consumer per result!!!
- if (pos != 0) {
- throw new RuntimeException(
- "Currently, each intermediate result can only have one
consumer.");
- }
-
- consumers.add(new ArrayList<ExecutionEdge>());
- return pos;
+ public void setConsumers(ConsumerVertexGroup consumers) {
Review comment:
Since there is `addConsumedPartitions` in `ExecutionVertex`, would it be
better to call this one `addConsumers`?
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
##########
@@ -668,6 +678,37 @@ public int getTotalNumberOfVertices() {
};
}
+ public EdgeManager getEdgeManager() {
+ return edgeManager;
+ }
+
+ public void registerExecutionVertex(ExecutionVertexID id, ExecutionVertex
vertex) {
+ executionVerticesById.put(id, vertex);
+ }
+
+ public void registerResultPartition(
+ IntermediateResultPartitionID id, IntermediateResultPartition
partition) {
+
+ resultPartitionsById.put(id, partition);
+ }
+
+ public ExecutionVertex getVertex(ExecutionVertexID id) {
+ return executionVerticesById.get(id);
+ }
+
+ public IntermediateResultPartition getResultPartition(final
IntermediateResultPartitionID id) {
Review comment:
Done.
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
##########
@@ -710,7 +714,9 @@ else if (current == RUNNING || current == DEPLOYING) {
return releaseFuture;
}
- private void updatePartitionConsumers(final List<List<ExecutionEdge>>
allConsumers) {
+ private void updatePartitionConsumers(
Review comment:
Yes, sorry for the carelessness. 😂
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java
##########
@@ -90,21 +89,12 @@ void resetForNewExecution() {
hasDataProduced = false;
}
- int addConsumerGroup() {
- int pos = consumers.size();
-
- // NOTE: currently we support only one consumer per result!!!
- if (pos != 0) {
- throw new RuntimeException(
- "Currently, each intermediate result can only have one
consumer.");
- }
-
- consumers.add(new ArrayList<ExecutionEdge>());
- return pos;
+ public void setConsumers(ConsumerVertexGroup consumers) {
+
producer.getExecutionGraph().getEdgeManager().addPartitionConsumers(partitionId,
consumers);
Review comment:
Sorry for being careless. Done.
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
##########
@@ -668,6 +678,37 @@ public int getTotalNumberOfVertices() {
};
}
+ public EdgeManager getEdgeManager() {
+ return edgeManager;
+ }
+
+ public void registerExecutionVertex(ExecutionVertexID id, ExecutionVertex
vertex) {
+ executionVerticesById.put(id, vertex);
+ }
+
+ public void registerResultPartition(
+ IntermediateResultPartitionID id, IntermediateResultPartition
partition) {
+
+ resultPartitionsById.put(id, partition);
+ }
+
+ public ExecutionVertex getVertex(ExecutionVertexID id) {
Review comment:
Agreed, I ignored the null value.
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorFactory.java
##########
@@ -72,15 +76,17 @@ private TaskDeploymentDescriptorFactory(
JobID jobID,
boolean allowUnknownPartitions,
int subtaskIndex,
- ExecutionEdge[][] inputEdges) {
+ List<ConsumedPartitionGroup> consumedPartitions,
+ Map<IntermediateResultPartitionID, IntermediateResultPartition>
resultPartitionsById) {
this.executionId = executionId;
this.attemptNumber = attemptNumber;
this.serializedJobInformation = serializedJobInformation;
this.taskInfo = taskInfo;
this.jobID = jobID;
this.allowUnknownPartitions = allowUnknownPartitions;
this.subtaskIndex = subtaskIndex;
- this.inputEdges = inputEdges;
+ this.consumedPartitions = consumedPartitions;
Review comment:
I agree. The parameter list seems to be complicated. But we need to
create `List<List<IntermediateResultPartition>>` in
`TaskDeploymentDescriptorFactory#fromExecutionVertex`, it may introduce extra
space costs. Should we use a `Function<>` here?
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/EdgeManagerBuildUtil.java
##########
@@ -0,0 +1,174 @@
+/*
+ * 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;
+
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+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 java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/** Utilities for building {@link EdgeManager}. */
+public class EdgeManagerBuildUtil {
+
+ public static void registerToExecutionEdgeManager(
+ ExecutionVertex[] taskVertices,
+ IntermediateResult ires,
+ int inputNumber,
+ DistributionPattern distributionPattern) {
+
+ switch (distributionPattern) {
+ case POINTWISE:
+ connectPointwise(taskVertices, ires, inputNumber);
+ break;
+ case ALL_TO_ALL:
+ connectAllToAll(taskVertices, ires, inputNumber);
+ break;
+ default:
+ throw new RuntimeException("Unrecognized distribution
pattern.");
+ }
+ }
+
+ private static void connectAllToAll(
+ ExecutionVertex[] taskVertices, IntermediateResult ires, int
inputNumber) {
+
+ ConsumedPartitionGroup consumedPartitions =
+ new ConsumedPartitionGroup(
+ Arrays.stream(ires.getPartitions())
+
.map(IntermediateResultPartition::getPartitionId)
+ .collect(Collectors.toList()));
+ for (ExecutionVertex ev : taskVertices) {
+ ev.setConsumedPartitions(consumedPartitions, inputNumber);
+ }
+
+ ConsumerVertexGroup vertices =
+ new ConsumerVertexGroup(
+ Arrays.stream(taskVertices)
+ .map(ExecutionVertex::getID)
+ .collect(Collectors.toList()));
+ for (IntermediateResultPartition partition : ires.getPartitions()) {
+ partition.setConsumers(vertices);
+ }
+ }
+
+ private static void connectPointwise(
+ ExecutionVertex[] taskVertices, IntermediateResult ires, int
inputNumber) {
+
+ final int sourceCount = ires.getPartitions().length;
+ final int targetCount = taskVertices.length;
+
+ if (sourceCount == targetCount) {
+ for (int i = 0; i < sourceCount; i++) {
+ ExecutionVertex executionVertex = taskVertices[i];
+ IntermediateResultPartition partition =
ires.getPartitions()[i];
+
+ ConsumerVertexGroup consumerVertexGroup =
+ new ConsumerVertexGroup(executionVertex.getID());
+ partition.setConsumers(consumerVertexGroup);
+
+ ConsumedPartitionGroup consumedPartitionGroup =
+ new ConsumedPartitionGroup(partition.getPartitionId());
+ executionVertex.setConsumedPartitions(consumedPartitionGroup,
inputNumber);
+ }
+ } else if (sourceCount > targetCount) {
+ for (int index = 0; index < targetCount; index++) {
+
+ ExecutionVertex executionVertex = taskVertices[index];
+ ConsumerVertexGroup consumerVertexGroup =
+ new ConsumerVertexGroup(executionVertex.getID());
+
+ List<IntermediateResultPartitionID> consumedPartitions =
+ new ArrayList<>(sourceCount / targetCount + 1);
+
+ if (sourceCount % targetCount == 0) {
Review comment:
Thank you for providing a better solution. I've replaced this part and
validate several common cases.
----------------------------------------------------------------
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.
For queries about this service, please contact Infrastructure at:
[email protected]