JunRuiLee commented on code in PR #25366: URL: https://github.com/apache/flink/pull/25366#discussion_r1772938079
########## flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/util/JobVertexBuildContext.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.streaming.api.graph.util; + +import org.apache.flink.annotation.Internal; +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.StreamConfig; +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; + +/** + * Helper class encapsulates all necessary information and configurations required during the + * construction of job vertices. + */ +@Internal +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 config for node chained with the current operator,for multi input operators containing + // sourceChain, the key will be the ids of all nodes in the chain, otherwise the key will only + // be the id of the head node in the chain + private final Map<Integer, Map<Integer, StreamConfig>> chainedConfigs; + + // 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; + + // We need to use AtomicBoolean to sense whether HybridResultPartition exists during the process Review Comment: // We use AtomicBoolean to track the existence of HybridResultPartition // during the incremental JobGraph generation process introduced by // AdaptiveGraphManager. It is essential to globally monitor changes // to this variable, thus necessitating the use of a Boolean object // instead of a primitive boolean. ########## flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/util/JobVertexBuildContext.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.streaming.api.graph.util; + +import org.apache.flink.annotation.Internal; +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.StreamConfig; +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; + +/** + * Helper class encapsulates all necessary information and configurations required during the + * construction of job vertices. + */ +@Internal +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 config for node chained with the current operator,for multi input operators containing + // sourceChain, the key will be the ids of all nodes in the chain, otherwise the key will only + // be the id of the head node in the chain + private final Map<Integer, Map<Integer, StreamConfig>> chainedConfigs; + + // 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; + + // We need to use AtomicBoolean to sense whether HybridResultPartition exists during the process + // of JobVertex building. + 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<>(); + this.chainedConfigs = 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 void addOperatorInfo(Integer nodeId, OperatorInfo operatorInfo) { + operatorInfosInorder.put(nodeId, operatorInfo); + } + + public Map<Integer, OperatorInfo> getOperatorInfos() { + return operatorInfosInorder; + } + + public StreamGraph getStreamGraph() { + return streamGraph; + } + + public boolean hasHybridResultPartition() { + return hasHybridResultPartition.get(); + } + + public void setHasHybridResultPartition(boolean hasHybridResultPartition) { + this.hasHybridResultPartition.set(hasHybridResultPartition); + } + + public void addPhysicalEdgesInOrder(StreamEdge edge) { + physicalEdgesInOrder.add(edge); + } + + public List<StreamEdge> getPhysicalEdgesInOrder() { + return physicalEdgesInOrder; + } + + public void addJobVertex(Integer startNodeId, JobVertex jobVertex) { + jobVerticesInorder.put(startNodeId, jobVertex); + } + + public Map<Integer, JobVertex> getJobVertices() { + return jobVerticesInorder; + } + + public JobVertex getJobVertex(Integer startNodeId) { + return jobVerticesInorder.get(startNodeId); + } + + public void putCoordinatorSerializationFutures( + JobVertexID vertexID, Review Comment: vertexId ########## flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/util/ChainedOperatorHashInfo.java: ########## @@ -0,0 +1,51 @@ +/* + * 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.annotation.Internal; +import org.apache.flink.streaming.api.graph.StreamNode; + +/** Helper class to help maintain the hash info of an operator chain. */ +@Internal +public final class ChainedOperatorHashInfo { + private final byte[] generatedOperatorID; + private final byte[] userDefinedOperatorID; + private final StreamNode streamNode; + + ChainedOperatorHashInfo( + final byte[] generatedOperatorID, Review Comment: generatedOperatorId ########## flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java: ########## @@ -355,32 +337,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 = + jobVertexBuildContext.getJobVertices().values().stream() + .map(JobVertex::getID) + .collect(Collectors.toSet()); jobGraph.getVerticesSortedTopologicallyFromSources() .forEach( - vertex -> + vertex -> { + if (jobVertexIds.contains(vertex.getID())) { vertex.setName( String.format( "[vertex-%d]%s", - vertexIndexId.getAndIncrement(), - vertex.getName()))); + vertexIndexId.getAndIncrement(), vertex.getName())); + } + }); } Review Comment: It seems that this method does not need to consider the concept of newly added vertices; you can simply retrieve the vertices from the context and set them directly. However, in the case of incremental graph, the context only contains the incremental vertices, while the setName method needs to set based on the entire topology. Therefore, it has to rely on the job graph to calculate a difference set. I suggest updating the Javadoc for this method. ########## flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java: ########## @@ -593,15 +616,22 @@ private Map<Integer, OperatorChainInfo> buildChainedInputsAndGetHeadInputs( if (targetChainingStrategy == ChainingStrategy.HEAD_WITH_SOURCES && isChainableInput(sourceOutEdge, streamGraph)) { + // we cache the non-chainable outputs here, and set the non-chained config later + OperatorInfo operatorInfo = new OperatorInfo(); + operatorInfo.setChainableOutputs(Collections.emptyList()); Review Comment: maybe nonChainable? ``` OperatorInfo operatorInfo = new OperatorInfo(); jobVertexBuildContext.addOperatorInfo(sourceNodeId, operatorInfo); final OperatorID opId = new OperatorID(hashes.get(sourceNodeId)); final StreamConfig.SourceInputConfig inputConfig = new StreamConfig.SourceInputConfig(sourceOutEdge); final StreamConfig operatorConfig = new StreamConfig(new Configuration()); setOperatorConfig( sourceNodeId, operatorConfig, Collections.emptyMap(), jobVertexBuildContext); setOperatorChainedOutputsConfig( operatorConfig, Collections.emptyList(), jobVertexBuildContext); // we cache the non-chainable outputs here, and set the non-chained config later operatorInfo.setNonChainableOutputs(Collections.emptyList()); ``` And a unit test case is needed ########## flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/util/ChainedOperatorHashInfo.java: ########## @@ -0,0 +1,51 @@ +/* + * 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.annotation.Internal; +import org.apache.flink.streaming.api.graph.StreamNode; + +/** Helper class to help maintain the hash info of an operator chain. */ +@Internal +public final class ChainedOperatorHashInfo { + private final byte[] generatedOperatorID; + private final byte[] userDefinedOperatorID; + private final StreamNode streamNode; + + ChainedOperatorHashInfo( + final byte[] generatedOperatorID, + final byte[] userDefinedOperatorID, Review Comment: userDefinedOperatorId ########## 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. The other chainedXXX fields are similar. ########## flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/util/JobVertexBuildContext.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.streaming.api.graph.util; + +import org.apache.flink.annotation.Internal; +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.StreamConfig; +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; + +/** + * Helper class encapsulates all necessary information and configurations required during the + * construction of job vertices. + */ +@Internal +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 config for node chained with the current operator,for multi input operators containing + // sourceChain, the key will be the ids of all nodes in the chain, otherwise the key will only + // be the id of the head node in the chain Review Comment: // This map's key represents the starting node id of each chain. Note that this includes not // only the usual head node of the chain but also the ids of chain sources which are used by // multi-input. -- 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