zhuzhurk commented on a change in pull request #18673:
URL: https://github.com/apache/flink/pull/18673#discussion_r802323336



##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java
##########
@@ -879,6 +882,28 @@ private void setVertexConfig(
         vertexConfigs.put(vertexID, config);
     }
 
+    private void tryConvertPartitionerForDynamicGraph(
+            List<StreamEdge> chainableOutputs, List<StreamEdge> 
nonChainableOutputs) {
+
+        if (!streamGraph.getExecutionConfig().isDynamicGraph()) {
+            return;
+        }
+
+        for (StreamEdge edge : chainableOutputs) {
+            StreamPartitioner<?> partitioner = edge.getPartitioner();
+            if (partitioner instanceof ForwardForLocalKeyByPartitioner) {

Review comment:
       If there is a `ForwardForLocalKeyByPartitioner` while it is not a 
dynamic graph, it cannot work and I think an exception should be thrown.

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/partitioner/ForwardForLocalKeyByPartitioner.java
##########
@@ -0,0 +1,79 @@
+/*
+ * 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.runtime.partitioner;
+
+import org.apache.flink.runtime.io.network.api.writer.SubtaskStateMapper;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+
+/**
+ * {@link ForwardForLocalKeyByPartitioner} is a intermediate partitioner in 
optimization phase,
+ * should be converted to following partitioners after the operator chain 
creation:
+ *
+ * <p>1. Be converted to {@link ForwardPartitioner} if this partitioner is 
intra-chain.
+ *
+ * <p>2. Be converted to {@link 
ForwardForLocalKeyByPartitioner#hashPartitioner} if this partitioner
+ * is inter-chain.
+ *
+ * @param <T> Type of the elements in the Stream
+ */
+public class ForwardForLocalKeyByPartitioner<T> extends ForwardPartitioner<T> {

Review comment:
       better to mark it as `@Internal`

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/partitioner/ForwardForLocalKeyByPartitioner.java
##########
@@ -0,0 +1,79 @@
+/*
+ * 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.runtime.partitioner;
+
+import org.apache.flink.runtime.io.network.api.writer.SubtaskStateMapper;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+
+/**
+ * {@link ForwardForLocalKeyByPartitioner} is a intermediate partitioner in 
optimization phase,
+ * should be converted to following partitioners after the operator chain 
creation:
+ *
+ * <p>1. Be converted to {@link ForwardPartitioner} if this partitioner is 
intra-chain.

Review comment:
       I'd like to confirm why do we need to convert the partitioner if the 
operators are already chained? Is it a safetynet?

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/partitioner/ForwardForLocalKeyByPartitioner.java
##########
@@ -0,0 +1,79 @@
+/*
+ * 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.runtime.partitioner;
+
+import org.apache.flink.runtime.io.network.api.writer.SubtaskStateMapper;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+
+/**
+ * {@link ForwardForLocalKeyByPartitioner} is a intermediate partitioner in 
optimization phase,

Review comment:
       I'm thinking to rename it to `ForwardForConsecutiveHashPartitioner` for 
easier understanding. And add explanation that it can be used only if all the 
records from subtask `#i` will also be sent to the same channel `#i` when using 
the underlying `hashPartitioner`. WDYT?

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/partitioner/ForwardForLocalKeyByPartitioner.java
##########
@@ -0,0 +1,79 @@
+/*
+ * 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.runtime.partitioner;
+
+import org.apache.flink.runtime.io.network.api.writer.SubtaskStateMapper;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+
+/**
+ * {@link ForwardForLocalKeyByPartitioner} is a intermediate partitioner in 
optimization phase,
+ * should be converted to following partitioners after the operator chain 
creation:
+ *
+ * <p>1. Be converted to {@link ForwardPartitioner} if this partitioner is 
intra-chain.
+ *
+ * <p>2. Be converted to {@link 
ForwardForLocalKeyByPartitioner#hashPartitioner} if this partitioner
+ * is inter-chain.
+ *
+ * @param <T> Type of the elements in the Stream
+ */
+public class ForwardForLocalKeyByPartitioner<T> extends ForwardPartitioner<T> {
+
+    private final StreamPartitioner<T> hashPartitioner;
+
+    /**
+     * Create a new ForwardForLocalKeyPartitioner.
+     *
+     * @param hashPartitioner the HashPartitioner
+     */
+    public ForwardForLocalKeyByPartitioner(StreamPartitioner<T> 
hashPartitioner) {
+        this.hashPartitioner = hashPartitioner;
+    }
+
+    @Override
+    public StreamPartitioner<T> copy() {
+        throw new RuntimeException(
+                "ForwardForLocalKeyPartitioner is a intermediate partitioner 
in optimization phase, "
+                        + "should be converted to 
ForwardPartitioner/HashPartitioner finally.");

Review comment:
       There is no `HashPartitioner` in Flink. Maybe throw an 
UnsupportedOperationException and say "... should be converted to a 
ForwardPartitioner and its underlying hashPartitioner at runtime."




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