zhuzhurk commented on code in PR #25822:
URL: https://github.com/apache/flink/pull/25822#discussion_r1907086225


##########
flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/util/StreamEdgeUpdateRequestInfo.java:
##########
@@ -29,18 +29,27 @@ public class StreamEdgeUpdateRequestInfo {
     private final Integer targetId;
 
     private StreamPartitioner<?> outputPartitioner;
+    // The type number for the input of co-tasks.

Review Comment:
   It's better to add an empty new line in ahead.



##########
flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/util/StreamEdgeUpdateRequestInfo.java:
##########
@@ -29,6 +29,7 @@ public class StreamEdgeUpdateRequestInfo {
     private final Integer targetId;
 
     private StreamPartitioner<?> outputPartitioner;
+    private int typeNumber;

Review Comment:
   I thought that 0 means the request will not change the typeNumber.



##########
flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/util/StreamNodeUpdateRequestInfo.java:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.api.common.typeutils.TypeSerializer;
+
+/** Helper class carries the data required to updates a stream edge. */
+@Internal
+public class StreamNodeUpdateRequestInfo {
+    private final Integer nodeId;
+
+    private TypeSerializer<?>[] typeSerializersIn;
+
+    public StreamNodeUpdateRequestInfo(Integer nodeId) {
+        this.nodeId = nodeId;
+    }
+
+    public StreamNodeUpdateRequestInfo withTypeSerializersIn(
+            TypeSerializer<?>[] typeSerializersIn) {
+        this.typeSerializersIn = typeSerializersIn;
+        return this;
+    }
+
+    public Integer getNodeId() {
+        return nodeId;
+    }
+
+    public TypeSerializer<?>[] getTypeSerializersIn() {

Review Comment:
   `@Nullable`



##########
flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/util/StreamNodeUpdateRequestInfo.java:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.api.common.typeutils.TypeSerializer;
+
+/** Helper class carries the data required to updates a stream edge. */
+@Internal
+public class StreamNodeUpdateRequestInfo {
+    private final Integer nodeId;
+
+    private TypeSerializer<?>[] typeSerializersIn;

Review Comment:
   `@Nullable`



##########
flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/DefaultStreamGraphContext.java:
##########
@@ -161,21 +216,23 @@ private boolean 
validateStreamEdgeUpdateRequest(StreamEdgeUpdateRequestInfo requ
         }
 
         // Modification is not allowed when the subscribing output is reused.
-        Map<StreamEdge, NonChainedOutput> opIntermediateOutputs =
-                opIntermediateOutputsCaches.get(sourceNodeId);
-        NonChainedOutput output =
-                opIntermediateOutputs != null ? 
opIntermediateOutputs.get(targetEdge) : null;
-        if (output != null) {
-            Set<StreamEdge> consumerStreamEdges =
-                    opIntermediateOutputs.entrySet().stream()
-                            .filter(entry -> entry.getValue().equals(output))
-                            .map(Map.Entry::getKey)
-                            .collect(Collectors.toSet());
-            if (consumerStreamEdges.size() != 1) {
-                LOG.info(
-                        "Skip modifying edge {} because the subscribing output 
is reused.",
-                        targetEdge);
-                return false;
+        if (requestInfo.getOutputPartitioner() != null) {

Review Comment:
   I see. Could you update the earlier comment to explain it? e.g. 
'Modification to output partitioner is not allowed when ...'



##########
flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/util/StreamNodeUpdateRequestInfo.java:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.api.common.typeutils.TypeSerializer;
+
+/** Helper class carries the data required to updates a stream edge. */
+@Internal
+public class StreamNodeUpdateRequestInfo {
+    private final Integer nodeId;
+
+    private TypeSerializer<?>[] typeSerializersIn;

Review Comment:
   And comment that 'null means it does not request to change the 
typeSerializersIn'



-- 
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]

Reply via email to