dianfu commented on code in PR #22003:
URL: https://github.com/apache/flink/pull/22003#discussion_r1119985077


##########
flink-python/pyflink/datastream/tests/test_data_stream.py:
##########
@@ -589,6 +589,40 @@ def process_element2(self, value, ctx: 
'CoProcessFunction.Context'):
         side_expected = ['0', '0', '1', '1', '2', '3']
         self.assert_equals_sorted(side_expected, side_sink.get_results())
 
+    def test_co_broadcast_side_output(self):
+        tag = OutputTag("side", Types.INT())
+
+        class MyBroadcastProcessFunction(BroadcastProcessFunction):
+
+            def process_element(self, value, ctx):
+                yield value[0]
+                yield tag, value[1]
+
+            def process_broadcast_element(self, value, ctx):
+                yield value[1]
+                yield tag, value[0]
+
+        self.env.set_parallelism(2)

Review Comment:
   This is unnecessary as the parallelism is 2 by default. Refer to 
PyFlinkStreamingTestCase for more details.



##########
flink-python/src/main/java/org/apache/flink/streaming/api/transformations/python/DelegateOperatorTransformation.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.transformations.python;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.python.env.PythonEnvironmentManager;
+import 
org.apache.flink.streaming.api.functions.python.DataStreamPythonFunctionInfo;
+import org.apache.flink.streaming.api.operators.SimpleOperatorFactory;
+import 
org.apache.flink.streaming.api.operators.python.AbstractPythonFunctionOperator;
+import 
org.apache.flink.streaming.api.operators.python.DataStreamPythonFunctionOperator;
+import org.apache.flink.util.OutputTag;
+
+import javax.annotation.Nullable;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * For those {@link org.apache.flink.api.dag.Transformation} that don't have 
an operator entity,
+ * {@link DelegateOperatorTransformation} provides a {@link 
SimpleOperatorFactory} containing a
+ * {@link DelegateOperator} , which can hold special configurations during 
transformation
+ * preprocessing for Python jobs, and later be queried at translation stage. 
Currently, those
+ * configurations include {@link OutputTag}s, {@code numPartitions} and 
general {@link
+ * Configuration}.
+ */
+public interface DelegateOperatorTransformation<OUT> {
+
+    SimpleOperatorFactory<OUT> getOperatorFactory();
+
+    static void configureDelegatedOperator(
+            DelegateOperatorTransformation<?> transformation,
+            AbstractPythonFunctionOperator<?> operator) {
+        DelegateOperator<?> delegateOperator =
+                (DelegateOperator<?>) 
transformation.getOperatorFactory().getOperator();
+
+        
operator.getConfiguration().addAll(delegateOperator.getConfiguration());
+
+        if (operator instanceof DataStreamPythonFunctionOperator) {
+            DataStreamPythonFunctionOperator<?> dataStreamOperator =
+                    (DataStreamPythonFunctionOperator<?>) operator;
+            
dataStreamOperator.addSideOutputTags(delegateOperator.getSideOutputTags());
+            if (delegateOperator.getNumPartitions() != null) {
+                
dataStreamOperator.setNumPartitions(delegateOperator.getNumPartitions());
+            }
+        }
+    }
+
+    /**
+     * {@link DelegateOperator} holds configurations, e.g. {@link OutputTag}s, 
which will be applied
+     * to the actual python operator at translation stage.
+     */
+    class DelegateOperator<OUT> extends AbstractPythonFunctionOperator<OUT>
+            implements DataStreamPythonFunctionOperator<OUT> {
+
+        private final Map<String, OutputTag<?>> sideOutputTags = new 
HashMap<>();
+        private @Nullable Integer numPartitions = null;
+
+        public DelegateOperator() {
+            super(new Configuration());
+        }
+
+        @Override
+        public void addSideOutputTags(Collection<OutputTag<?>> outputTags) {
+            for (OutputTag<?> outputTag : outputTags) {
+                sideOutputTags.put(outputTag.getId(), outputTag);
+            }
+        }
+
+        @Override
+        public Collection<OutputTag<?>> getSideOutputTags() {
+            return sideOutputTags.values();
+        }
+
+        @Override
+        public void setNumPartitions(int numPartitions) {
+            this.numPartitions = numPartitions;
+        }
+
+        @Nullable
+        public Integer getNumPartitions() {
+            return numPartitions;
+        }
+
+        @Override
+        public TypeInformation<OUT> getProducedType() {
+            throw new RuntimeException("This should not be invoked on a 
DelegateOperator!");
+        }
+
+        @Override
+        public DataStreamPythonFunctionInfo getPythonFunctionInfo() {
+            throw new RuntimeException("This should not be invoked on a 
DelegateOperator!");
+        }
+
+        @Override
+        public <T> DataStreamPythonFunctionOperator<T> copy(
+                DataStreamPythonFunctionInfo pythonFunctionInfo,
+                TypeInformation<T> outputTypeInfo) {
+            throw new RuntimeException("This should not be invoked on a 
DelegateOperator!");

Review Comment:
   May be we should implement this. This method is used when performing 
operator chain optimization.



##########
flink-python/src/main/java/org/apache/flink/streaming/api/transformations/python/DelegateOperatorTransformation.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.transformations.python;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.python.env.PythonEnvironmentManager;
+import 
org.apache.flink.streaming.api.functions.python.DataStreamPythonFunctionInfo;
+import org.apache.flink.streaming.api.operators.SimpleOperatorFactory;
+import 
org.apache.flink.streaming.api.operators.python.AbstractPythonFunctionOperator;
+import 
org.apache.flink.streaming.api.operators.python.DataStreamPythonFunctionOperator;
+import org.apache.flink.util.OutputTag;
+
+import javax.annotation.Nullable;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * For those {@link org.apache.flink.api.dag.Transformation} that don't have 
an operator entity,
+ * {@link DelegateOperatorTransformation} provides a {@link 
SimpleOperatorFactory} containing a
+ * {@link DelegateOperator} , which can hold special configurations during 
transformation
+ * preprocessing for Python jobs, and later be queried at translation stage. 
Currently, those
+ * configurations include {@link OutputTag}s, {@code numPartitions} and 
general {@link
+ * Configuration}.
+ */
+public interface DelegateOperatorTransformation<OUT> {
+
+    SimpleOperatorFactory<OUT> getOperatorFactory();
+
+    static void configureDelegatedOperator(

Review Comment:
   Renaming it to `configure` or something else? As in this method, we are 
actually config `operator` instead of `delegateOperator`.



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

Reply via email to