JunRuiLee commented on code in PR #25366:
URL: https://github.com/apache/flink/pull/25366#discussion_r1770890500
##########
flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java:
##########
@@ -517,7 +537,8 @@ private void preValidate() {
}
if (checkpointConfig.isUnalignedCheckpointsEnabled()
&& !checkpointConfig.isForceUnalignedCheckpoints()
- &&
streamGraph.getStreamNodes().stream().anyMatch(this::hasCustomPartitioner)) {
+ && streamGraph.getStreamNodes().stream()
+
.anyMatch(StreamingJobGraphGenerator::hasCustomPartitioner)) {
Review Comment:
why we need change this line?
##########
flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java:
##########
@@ -220,24 +199,16 @@ private StreamingJobGraphGenerator(
this.defaultStreamGraphHasher = new StreamGraphHasherV2();
this.legacyStreamGraphHashers = Arrays.asList(new
StreamGraphUserHashHasher());
- this.jobVertices = new LinkedHashMap<>();
this.builtVertices = new HashSet<>();
- this.chainedConfigs = new HashMap<>();
- this.vertexConfigs = new HashMap<>();
- this.chainedNames = new HashMap<>();
- this.chainedMinResources = new HashMap<>();
- this.chainedPreferredResources = new HashMap<>();
- this.chainedInputOutputFormats = new HashMap<>();
- this.physicalEdgesInOrder = new ArrayList<>();
this.serializationExecutor =
Preconditions.checkNotNull(serializationExecutor);
- this.chainInfos = new HashMap<>();
- this.opNonChainableOutputsCache = new LinkedHashMap<>();
-
+ this.hasHybridResultPartition = new AtomicBoolean(false);
Review Comment:
It seems that this field is only used for JobVertexBuildContext. I don't
think it's useful for this class anymore.
##########
flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java:
##########
@@ -172,43 +176,18 @@ public static JobGraph createJobGraph(
private final ClassLoader userClassloader;
private final StreamGraph streamGraph;
- private final Map<Integer, JobVertex> jobVertices;
private final JobGraph jobGraph;
private final Collection<Integer> builtVertices;
- private final List<StreamEdge> physicalEdgesInOrder;
-
- private final Map<Integer, Map<Integer, StreamConfig>> chainedConfigs;
-
- private final Map<Integer, StreamConfig> vertexConfigs;
- private final Map<Integer, String> chainedNames;
-
- private final Map<Integer, ResourceSpec> chainedMinResources;
- private final Map<Integer, ResourceSpec> chainedPreferredResources;
-
- private final Map<Integer, InputOutputFormatContainer>
chainedInputOutputFormats;
-
private final StreamGraphHasher defaultStreamGraphHasher;
private final List<StreamGraphHasher> legacyStreamGraphHashers;
- private boolean hasHybridResultPartition = false;
-
private final Executor serializationExecutor;
- // Futures for the serialization of operator coordinators
- private final Map<
- JobVertexID,
-
List<CompletableFuture<SerializedValue<OperatorCoordinator.Provider>>>>
- coordinatorSerializationFuturesPerJobVertex = new HashMap<>();
-
- /** The {@link OperatorChainInfo}s, key is the start node id of the chain.
*/
- private final Map<Integer, OperatorChainInfo> chainInfos;
+ /** We save all the context needed to create the JobVertex in this
structure */
+ private final JobVertexBuildContext jobVertexBuildContext;
- /**
- * This is used to cache the non-chainable outputs, to set the
non-chainable outputs config
- * after all job vertices are created.
- */
- private final Map<Integer, List<StreamEdge>> opNonChainableOutputsCache;
+ private final AtomicBoolean hasHybridResultPartition;
Review Comment:
why we need change this field to Atomic?
##########
flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java:
##########
@@ -1221,52 +1304,68 @@ private void setVertexNonChainedOutputsConfig(
for (StreamEdge edge : transitiveOutEdges) {
NonChainedOutput output =
opIntermediateOutputs.get(edge.getSourceId()).get(edge);
transitiveOutputs.add(output);
- connect(startNodeId, edge, output);
+ connect(
+ startNodeId,
+ edge,
+ output,
+ jobVertexBuildContext.getJobVertices(),
+ jobVertexBuildContext);
}
config.setVertexNonChainedOutputs(new ArrayList<>(transitiveOutputs));
}
- private void setAllOperatorNonChainedOutputsConfigs(
- final Map<Integer, Map<StreamEdge, NonChainedOutput>>
opIntermediateOutputs) {
+ public static void setAllOperatorNonChainedOutputsConfigs(
+ final Map<Integer, Map<StreamEdge, NonChainedOutput>>
opIntermediateOutputs,
+ JobVertexBuildContext jobVertexBuildContext) {
// set non chainable output config
- opNonChainableOutputsCache.forEach(
- (vertexId, nonChainableOutputs) -> {
- Map<StreamEdge, NonChainedOutput> outputsConsumedByEdge =
- opIntermediateOutputs.computeIfAbsent(
- vertexId, ignored -> new HashMap<>());
- setOperatorNonChainedOutputsConfig(
- vertexId,
- vertexConfigs.get(vertexId),
- nonChainableOutputs,
- outputsConsumedByEdge);
- });
+ jobVertexBuildContext
+ .getOperatorInfos()
Review Comment:
After this change, all operator info will be looped and set
setOperatorNonChainedOutputsConfig, while the original behavior only looped
opNonChainableOutputsCache.
Is this approach safe because nonChainableOutputs is guaranteed to be empty,
right ?
##########
flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/util/JobVertexBuildContext.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.streaming.api.graph.util;
+
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.streaming.api.graph.StreamEdge;
+import org.apache.flink.streaming.api.graph.StreamGraph;
+import org.apache.flink.util.SerializedValue;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class JobVertexBuildContext {
+
+ private final StreamGraph streamGraph;
+
+ /** The {@link OperatorChainInfo}s, key is the start node id of the chain.
*/
+ private final Map<Integer, OperatorChainInfo> chainInfos;
+
+ /** The {@link OperatorInfo}s, key is the id of the stream node. */
+ private final Map<Integer, OperatorInfo> operatorInfosInorder;
+
+ // The created JobVertex, the key is start node id
+ private final Map<Integer, JobVertex> jobVerticesInorder;
+
+ // Futures for the serialization of operator coordinators
+ private final Map<
+ JobVertexID,
+
List<CompletableFuture<SerializedValue<OperatorCoordinator.Provider>>>>
+ coordinatorSerializationFuturesPerJobVertex;
+
+ // The order of StreamEdge connected to other vertices should be
consistent with the order in
+ // which JobEdge was created
+ private final List<StreamEdge> physicalEdgesInOrder;
+
+ private final AtomicBoolean hasHybridResultPartition;
Review Comment:
private boolean
##########
flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/util/OperatorInfo.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.streaming.api.graph.util;
+
+import org.apache.flink.api.common.operators.ResourceSpec;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.graph.StreamEdge;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/** A public class to help maintain the information of an operator */
+public class OperatorInfo {
+
+ private String chainedName;
+
+ private ResourceSpec chainedMinResource;
+
+ private ResourceSpec chainedPreferredResources;
+
+ private StreamConfig vertexConfig;
+
+ /**
+ * The config for node chained with the current operator, the key is the
ID of the stream node
+ */
+ private final Map<Integer, StreamConfig> chainedConfigs;
Review Comment:
Would it be better to move this field to OperatorChainInfo? The other
chainedXXX fields are similar, include chainedName, chainedMinResource and
chainedPreferredResources.
##########
flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java:
##########
@@ -355,32 +340,44 @@ private void
waitForSerializationFuturesAndUpdateJobVertices()
}
}
- private void addVertexIndexPrefixInVertexName() {
- if (!streamGraph.isVertexNameIncludeIndexPrefix()) {
+ // Set this property for the newly generated JobVertex
+ public static void addVertexIndexPrefixInVertexName(
+ JobVertexBuildContext jobVertexBuildContext,
+ AtomicInteger vertexIndexId,
+ JobGraph jobGraph) {
+ if
(!jobVertexBuildContext.getStreamGraph().isVertexNameIncludeIndexPrefix()) {
return;
}
- final AtomicInteger vertexIndexId = new AtomicInteger(0);
+ Set<JobVertexID> jobVertexIDS =
Review Comment:
jobVertexIds
##########
flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/util/JobVertexBuildContext.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.streaming.api.graph.util;
+
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.streaming.api.graph.StreamEdge;
+import org.apache.flink.streaming.api.graph.StreamGraph;
+import org.apache.flink.util.SerializedValue;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class JobVertexBuildContext {
+
+ private final StreamGraph streamGraph;
+
+ /** The {@link OperatorChainInfo}s, key is the start node id of the chain.
*/
+ private final Map<Integer, OperatorChainInfo> chainInfos;
+
+ /** The {@link OperatorInfo}s, key is the id of the stream node. */
+ private final Map<Integer, OperatorInfo> operatorInfosInorder;
+
+ // The created JobVertex, the key is start node id
+ private final Map<Integer, JobVertex> jobVerticesInorder;
+
+ // Futures for the serialization of operator coordinators
+ private final Map<
+ JobVertexID,
+
List<CompletableFuture<SerializedValue<OperatorCoordinator.Provider>>>>
+ coordinatorSerializationFuturesPerJobVertex;
+
+ // The order of StreamEdge connected to other vertices should be
consistent with the order in
+ // which JobEdge was created
+ private final List<StreamEdge> physicalEdgesInOrder;
+
+ private final AtomicBoolean hasHybridResultPartition;
+
+ public JobVertexBuildContext(StreamGraph streamGraph, AtomicBoolean
hasHybridResultPartition) {
+ this.streamGraph = streamGraph;
+ this.chainInfos = new LinkedHashMap<>();
+ this.operatorInfosInorder = new LinkedHashMap<>();
+ this.jobVerticesInorder = new LinkedHashMap<>();
+ this.physicalEdgesInOrder = new ArrayList<>();
+ this.hasHybridResultPartition = hasHybridResultPartition;
+ this.coordinatorSerializationFuturesPerJobVertex = new HashMap<>();
+ }
+
+ public void addChainInfo(Integer startNodeId, OperatorChainInfo chainInfo)
{
+ chainInfos.put(startNodeId, chainInfo);
+ }
+
+ public OperatorChainInfo getChainInfo(Integer startNodeId) {
+ return chainInfos.get(startNodeId);
+ }
+
+ public Map<Integer, OperatorChainInfo> getChainInfos() {
+ return chainInfos;
+ }
+
+ public OperatorInfo getOperatorInfo(Integer nodeId) {
+ return operatorInfosInorder.get(nodeId);
+ }
+
+ public OperatorInfo getOrCreateOperatorInfo(Integer nodeId) {
+ return operatorInfosInorder.computeIfAbsent(nodeId, key -> new
OperatorInfo());
+ }
Review Comment:
Could we split this method into two methods: getOperatorInfo and
createOperatorInfo?
--
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]