zhuzhurk commented on code in PR #25414: URL: https://github.com/apache/flink/pull/25414#discussion_r1818916132
########## flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/util/ImmutableStreamGraph.java: ########## @@ -0,0 +1,47 @@ +/* + * 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.StreamGraph; + +import java.util.HashMap; +import java.util.Map; + +/** Helper class that provides read-only StreamGraph. */ +@Internal +public class ImmutableStreamGraph { + private final StreamGraph streamGraph; + + private final Map<Integer, ImmutableStreamNode> immutableStreamNodes; + + public ImmutableStreamGraph(StreamGraph streamGraph) { + this.streamGraph = streamGraph; + this.immutableStreamNodes = new HashMap<>(); + } + + public ImmutableStreamNode getStreamNode(Integer vertexId) { + if (!immutableStreamNodes.containsKey(vertexId) + && streamGraph.getStreamNode(vertexId) != null) { Review Comment: Looks to me it will return `null` via `immutableStreamNodes.get(vertexId)` if `streamGraph.getStreamNode(vertexId) == null`. How about to directly return `null` in this case? And maybe avoid the unnecessary `put` and `get` if the immutable version does not exist. ########## flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/util/ImmutableStreamNode.java: ########## @@ -0,0 +1,70 @@ +/* + * 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.StreamEdge; +import org.apache.flink.streaming.api.graph.StreamNode; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +/** Helper class that provides read-only StreamNode. */ +@Internal +public class ImmutableStreamNode { + private final StreamNode streamNode; + private List<ImmutableStreamEdge> immutableOutEdges = null; + private List<ImmutableStreamEdge> immutableInEdges = null; + + public ImmutableStreamNode(StreamNode streamNode) { + this.streamNode = streamNode; + } + + public List<ImmutableStreamEdge> getOutEdges() { + if (immutableOutEdges == null) { + immutableOutEdges = new ArrayList<>(); + for (StreamEdge edge : streamNode.getOutEdges()) { + immutableOutEdges.add(new ImmutableStreamEdge(edge)); + } + } + return Collections.unmodifiableList(immutableOutEdges); + } + + public List<ImmutableStreamEdge> getInEdges() { + if (immutableInEdges == null) { + immutableInEdges = new ArrayList<>(); + for (StreamEdge edge : streamNode.getInEdges()) { + immutableInEdges.add(new ImmutableStreamEdge(edge)); + } + } + return Collections.unmodifiableList(immutableInEdges); + } + + public int getId() { + return streamNode.getId(); + } + + public @Nullable StreamOperatorFactory<?> getOperatorFactory() { + return streamNode.getOperatorFactory(); Review Comment: In what case it can be null? ########## flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/util/ImmutableStreamGraph.java: ########## @@ -0,0 +1,47 @@ +/* + * 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.StreamGraph; + +import java.util.HashMap; +import java.util.Map; + +/** Helper class that provides read-only StreamGraph. */ +@Internal Review Comment: No need to mark it as `@Internal`. `StreamGraph` was annotated as `@Internal` because it was in an API package(flink-streaming-java) in previous version. ########## flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/util/ImmutableStreamGraphTest.java: ########## @@ -0,0 +1,74 @@ +/* + * 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.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.graph.StreamEdge; +import org.apache.flink.streaming.api.graph.StreamGraph; +import org.apache.flink.streaming.api.graph.StreamNode; + +import org.junit.jupiter.api.Test; + +import java.util.List; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Unit tests for {@link ImmutableStreamGraph}. */ +class ImmutableStreamGraphTest { + @Test + void testImmutableStreamGraphGraph() { Review Comment: testImmutableStreamGraphGraph -> testImmutableStreamGraphContent -- 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