You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/07/28 07:48:25 UTC

[GitHub] [flink] Vancior commented on a diff in pull request #20375: [FLINK-28559][python] Support DataStream PythonKeyedProcessOperator in Thread Mode

Vancior commented on code in PR #20375:
URL: https://github.com/apache/flink/pull/20375#discussion_r931879127


##########
flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/embedded/EmbeddedPythonKeyedProcessOperator.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * 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.operators.python.embedded;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.fnexecution.v1.FlinkFnApi;
+import org.apache.flink.python.util.ProtoUtils;
+import org.apache.flink.runtime.state.VoidNamespace;
+import org.apache.flink.runtime.state.VoidNamespaceSerializer;
+import org.apache.flink.streaming.api.SimpleTimerService;
+import org.apache.flink.streaming.api.TimeDomain;
+import org.apache.flink.streaming.api.TimerService;
+import org.apache.flink.streaming.api.functions.python.DataStreamPythonFunctionInfo;
+import org.apache.flink.streaming.api.operators.InternalTimer;
+import org.apache.flink.streaming.api.operators.InternalTimerService;
+import org.apache.flink.streaming.api.operators.Triggerable;
+import org.apache.flink.types.Row;
+
+import pemja.core.object.PyIterator;
+
+import java.util.List;
+
+import static org.apache.flink.python.Constants.STATEFUL_FUNCTION_URN;
+import static org.apache.flink.python.PythonOptions.MAP_STATE_READ_CACHE_SIZE;
+import static org.apache.flink.python.PythonOptions.MAP_STATE_WRITE_CACHE_SIZE;
+import static org.apache.flink.python.PythonOptions.PYTHON_METRIC_ENABLED;
+import static org.apache.flink.python.PythonOptions.PYTHON_PROFILE_ENABLED;
+import static org.apache.flink.python.PythonOptions.STATE_CACHE_SIZE;
+import static org.apache.flink.streaming.api.utils.PythonOperatorUtils.inBatchExecutionMode;
+
+/**
+ * {@link EmbeddedPythonKeyedProcessOperator} is responsible for executing user defined python
+ * KeyedProcessFunction in embedded Python environment. It is also able to handle the timer and
+ * state request from the python stateful user defined function.
+ */
+@Internal
+public class EmbeddedPythonKeyedProcessOperator<K, IN, OUT>
+        extends AbstractOneInputEmbeddedPythonFunctionOperator<IN, OUT>
+        implements Triggerable<K, VoidNamespace> {
+
+    private static final long serialVersionUID = 1L;
+
+    /** The TypeInformation of the key. */
+    private transient TypeInformation<Row> keyTypeInfo;
+
+    private transient ContextImpl context;
+
+    private transient OnTimerContextImpl onTimerContext;
+
+    public EmbeddedPythonKeyedProcessOperator(
+            Configuration config,
+            DataStreamPythonFunctionInfo pythonFunctionInfo,
+            TypeInformation<IN> inputTypeInfo,
+            TypeInformation<OUT> outputTypeInfo) {
+        super(STATEFUL_FUNCTION_URN, config, pythonFunctionInfo, inputTypeInfo, outputTypeInfo);
+    }
+
+    @Override
+    public void open() throws Exception {
+        keyTypeInfo = ((RowTypeInfo) this.getInputTypeInfo()).getTypeAt(0);
+
+        InternalTimerService<VoidNamespace> internalTimerService =
+                getInternalTimerService("user-timers", VoidNamespaceSerializer.INSTANCE, this);
+
+        TimerService timerService = new SimpleTimerService(internalTimerService);
+
+        context = new ContextImpl(timerService);
+
+        onTimerContext = new OnTimerContextImpl(timerService);
+
+        super.open();
+    }
+
+    @Override
+    public List<FlinkFnApi.UserDefinedDataStreamFunction> createUserDefinedFunctionsProto() {
+        return ProtoUtils.createUserDefinedDataStreamStatefulFunctionProtos(
+                getPythonFunctionInfo(),
+                getRuntimeContext(),
+                getJobParameters(),
+                keyTypeInfo,
+                inBatchExecutionMode(getKeyedStateBackend()),
+                config.get(PYTHON_METRIC_ENABLED),
+                config.get(PYTHON_PROFILE_ENABLED),
+                false,
+                config.get(STATE_CACHE_SIZE),
+                config.get(MAP_STATE_READ_CACHE_SIZE),
+                config.get(MAP_STATE_WRITE_CACHE_SIZE));
+    }
+
+    @Override
+    public void onEventTime(InternalTimer<K, VoidNamespace> timer) throws Exception {
+        collector.setAbsoluteTimestamp(timer.getTimestamp());
+        invokeUserFunction(TimeDomain.EVENT_TIME, timer);
+    }
+
+    @Override
+    public void onProcessingTime(InternalTimer<K, VoidNamespace> timer) throws Exception {
+        collector.eraseTimestamp();
+        invokeUserFunction(TimeDomain.PROCESSING_TIME, timer);
+    }
+
+    @Override
+    public Object getFunctionContext() {
+        return context;
+    }
+
+    @Override
+    public Object getTimerContext() {
+        return onTimerContext;
+    }
+
+    @Override
+    public <T> AbstractEmbeddedDataStreamPythonFunctionOperator<T> copy(
+            DataStreamPythonFunctionInfo pythonFunctionInfo, TypeInformation<T> outputTypeInfo) {
+        return new EmbeddedPythonKeyedProcessOperator<>(
+                config, pythonFunctionInfo, getInputTypeInfo(), outputTypeInfo);
+    }
+
+    private void invokeUserFunction(TimeDomain timeDomain, InternalTimer<K, VoidNamespace> timer)
+            throws Exception {
+        onTimerContext.timeDomain = timeDomain;
+        onTimerContext.timer = timer;
+        PyIterator results =
+                (PyIterator)
+                        interpreter.invokeMethod("operation", "on_timer", timer.getTimestamp());
+
+        while (results.hasNext()) {
+            OUT result = outputDataConverter.toInternal(results.next());
+            collector.collect(result);
+        }
+        results.close();
+
+        onTimerContext.timeDomain = null;
+        onTimerContext.timer = null;
+    }
+
+    private class ContextImpl {
+
+        private final TimerService timerService;
+
+        ContextImpl(TimerService timerService) {
+            this.timerService = timerService;
+        }
+
+        public long timestamp() {
+            return timestamp;
+        }
+
+        public TimerService timerService() {
+            return timerService;
+        }
+
+        @SuppressWarnings("unchecked")
+        public K getCurrentKey() {
+            return (K) ((Row) EmbeddedPythonKeyedProcessOperator.this.getCurrentKey()).getField(0);

Review Comment:
   Key should be translated to external data for Python?



##########
flink-python/pyflink/datastream/data_stream.py:
##########
@@ -1205,40 +1211,47 @@ def __init__(self, reduce_function):
                     self._open_func = None
                     self._close_func = None
                     self._reduce_function = reduce_function
-                self._reduce_value_state = None  # type: ValueState
+                self._reduce_state = None  # type: ReducingState
+                self._in_batch_execution_mode = True
+                self._has_started_key_set = set()
 
             def open(self, runtime_context: RuntimeContext):
                 if self._open_func:
                     self._open_func(runtime_context)
 
-                self._reduce_value_state = runtime_context.get_state(
-                    ValueStateDescriptor("_reduce_state" + str(uuid.uuid4()), output_type))
-                from pyflink.fn_execution.datastream.process.runtime_context import (
-                    StreamingRuntimeContext)
-                self._in_batch_execution_mode = \
-                    cast(StreamingRuntimeContext, runtime_context)._in_batch_execution_mode
+                self._reduce_state = runtime_context.get_reducing_state(
+                    ReducingStateDescriptor(
+                        "_reduce_state" + str(uuid.uuid4()),
+                        self._reduce_function,
+                        output_type))
+
+                if python_execution_mode == "process":
+                    from pyflink.fn_execution.datastream.process.runtime_context import (
+                        StreamingRuntimeContext)
+                    self._in_batch_execution_mode = (
+                        cast(StreamingRuntimeContext, runtime_context)._in_batch_execution_mode)
+                else:
+                    self._in_batch_execution_mode = runtime_context.get_job_parameter(
+                        "inBatchExecutionMode", "false") == "true"
 
             def close(self):
                 if self._close_func:
                     self._close_func()
 
             def process_element(self, value, ctx: 'KeyedProcessFunction.Context'):
-                reduce_value = self._reduce_value_state.value()
-                if reduce_value is not None:
-                    reduce_value = self._reduce_function(reduce_value, value)
-                else:
-                    # register a timer for emitting the result at the end when this is the
-                    # first input for this key
-                    if self._in_batch_execution_mode:
+                self._reduce_state.add(value)
+                if self._in_batch_execution_mode:
+                    key = ctx.get_current_key()
+                    if isinstance(key, list):
+                        key = tuple(key)
+                    if key not in self._has_started_key_set:
                         ctx.timer_service().register_event_time_timer(0x7fffffffffffffff)
-                    reduce_value = value
-                self._reduce_value_state.update(reduce_value)
-                if not self._in_batch_execution_mode:
-                    # only emitting the result when all the data for a key is received
-                    yield reduce_value
+                        self._has_started_key_set.add(key)

Review Comment:
   This might grow infinitely



##########
flink-python/pyflink/fn_execution/embedded/converters.py:
##########
@@ -214,9 +228,23 @@ def from_field_type_proto(field_type):
             [from_field_type_proto(f.type) for f in field_type.row_schema.fields],
             [f.name for f in field_type.row_schema.fields])
     elif type_name == schema_type_name.BASIC_ARRAY:
-        return ListDataConverter(from_field_type_proto(field_type.collection_element_type))
+        return ArrayDataConverter(from_field_type_proto(field_type.collection_element_type))
     elif type_name == schema_type_name.MAP:
         return DictDataConverter(from_field_type_proto(field_type.map_info.key_type),
                                  from_field_type_proto(field_type.map_info.value_type))
 
     return IdentityDataConverter()
+
+
+def from_type_info(type_info: TypeInformation):
+    if isinstance(type_info, (PickledBytesTypeInfo, RowTypeInfo, TupleTypeInfo)):

Review Comment:
   Why `RowTypeInfo` and `TupleTypeInfo` also use pickling here?



##########
flink-python/pyflink/fn_execution/embedded/operations.py:
##########
@@ -45,13 +50,15 @@ def __init__(self,
                  output_data_converter: DataConverter,
                  runtime_context,
                  function_context,
+                 timer_context,
                  job_parameters):
         operations = (
             [OneInputOperation(
                 function_urn,

Review Comment:
   This would make chained operations like (keyed, non-keyed, non-keyed) to produce a chain of three keyed operations.



##########
flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/embedded/EmbeddedPythonKeyedProcessOperator.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * 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.operators.python.embedded;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.fnexecution.v1.FlinkFnApi;
+import org.apache.flink.python.util.ProtoUtils;
+import org.apache.flink.runtime.state.VoidNamespace;
+import org.apache.flink.runtime.state.VoidNamespaceSerializer;
+import org.apache.flink.streaming.api.SimpleTimerService;
+import org.apache.flink.streaming.api.TimeDomain;
+import org.apache.flink.streaming.api.TimerService;
+import org.apache.flink.streaming.api.functions.python.DataStreamPythonFunctionInfo;
+import org.apache.flink.streaming.api.operators.InternalTimer;
+import org.apache.flink.streaming.api.operators.InternalTimerService;
+import org.apache.flink.streaming.api.operators.Triggerable;
+import org.apache.flink.types.Row;
+
+import pemja.core.object.PyIterator;
+
+import java.util.List;
+
+import static org.apache.flink.python.Constants.STATEFUL_FUNCTION_URN;
+import static org.apache.flink.python.PythonOptions.MAP_STATE_READ_CACHE_SIZE;
+import static org.apache.flink.python.PythonOptions.MAP_STATE_WRITE_CACHE_SIZE;
+import static org.apache.flink.python.PythonOptions.PYTHON_METRIC_ENABLED;
+import static org.apache.flink.python.PythonOptions.PYTHON_PROFILE_ENABLED;
+import static org.apache.flink.python.PythonOptions.STATE_CACHE_SIZE;
+import static org.apache.flink.streaming.api.utils.PythonOperatorUtils.inBatchExecutionMode;
+
+/**
+ * {@link EmbeddedPythonKeyedProcessOperator} is responsible for executing user defined python
+ * KeyedProcessFunction in embedded Python environment. It is also able to handle the timer and
+ * state request from the python stateful user defined function.
+ */
+@Internal
+public class EmbeddedPythonKeyedProcessOperator<K, IN, OUT>
+        extends AbstractOneInputEmbeddedPythonFunctionOperator<IN, OUT>
+        implements Triggerable<K, VoidNamespace> {
+
+    private static final long serialVersionUID = 1L;
+
+    /** The TypeInformation of the key. */
+    private transient TypeInformation<Row> keyTypeInfo;
+
+    private transient ContextImpl context;
+
+    private transient OnTimerContextImpl onTimerContext;
+
+    public EmbeddedPythonKeyedProcessOperator(
+            Configuration config,
+            DataStreamPythonFunctionInfo pythonFunctionInfo,
+            TypeInformation<IN> inputTypeInfo,
+            TypeInformation<OUT> outputTypeInfo) {
+        super(STATEFUL_FUNCTION_URN, config, pythonFunctionInfo, inputTypeInfo, outputTypeInfo);
+    }
+
+    @Override
+    public void open() throws Exception {
+        keyTypeInfo = ((RowTypeInfo) this.getInputTypeInfo()).getTypeAt(0);
+
+        InternalTimerService<VoidNamespace> internalTimerService =
+                getInternalTimerService("user-timers", VoidNamespaceSerializer.INSTANCE, this);
+
+        TimerService timerService = new SimpleTimerService(internalTimerService);
+
+        context = new ContextImpl(timerService);
+
+        onTimerContext = new OnTimerContextImpl(timerService);
+
+        super.open();
+    }
+
+    @Override
+    public List<FlinkFnApi.UserDefinedDataStreamFunction> createUserDefinedFunctionsProto() {
+        return ProtoUtils.createUserDefinedDataStreamStatefulFunctionProtos(
+                getPythonFunctionInfo(),
+                getRuntimeContext(),
+                getJobParameters(),
+                keyTypeInfo,
+                inBatchExecutionMode(getKeyedStateBackend()),
+                config.get(PYTHON_METRIC_ENABLED),
+                config.get(PYTHON_PROFILE_ENABLED),
+                false,
+                config.get(STATE_CACHE_SIZE),
+                config.get(MAP_STATE_READ_CACHE_SIZE),
+                config.get(MAP_STATE_WRITE_CACHE_SIZE));
+    }
+
+    @Override
+    public void onEventTime(InternalTimer<K, VoidNamespace> timer) throws Exception {
+        collector.setAbsoluteTimestamp(timer.getTimestamp());
+        invokeUserFunction(TimeDomain.EVENT_TIME, timer);
+    }
+
+    @Override
+    public void onProcessingTime(InternalTimer<K, VoidNamespace> timer) throws Exception {
+        collector.eraseTimestamp();
+        invokeUserFunction(TimeDomain.PROCESSING_TIME, timer);
+    }
+
+    @Override
+    public Object getFunctionContext() {
+        return context;
+    }
+
+    @Override
+    public Object getTimerContext() {
+        return onTimerContext;
+    }
+
+    @Override
+    public <T> AbstractEmbeddedDataStreamPythonFunctionOperator<T> copy(
+            DataStreamPythonFunctionInfo pythonFunctionInfo, TypeInformation<T> outputTypeInfo) {
+        return new EmbeddedPythonKeyedProcessOperator<>(
+                config, pythonFunctionInfo, getInputTypeInfo(), outputTypeInfo);
+    }
+
+    private void invokeUserFunction(TimeDomain timeDomain, InternalTimer<K, VoidNamespace> timer)
+            throws Exception {
+        onTimerContext.timeDomain = timeDomain;
+        onTimerContext.timer = timer;
+        PyIterator results =
+                (PyIterator)
+                        interpreter.invokeMethod("operation", "on_timer", timer.getTimestamp());
+
+        while (results.hasNext()) {
+            OUT result = outputDataConverter.toInternal(results.next());
+            collector.collect(result);
+        }
+        results.close();
+
+        onTimerContext.timeDomain = null;
+        onTimerContext.timer = null;
+    }
+
+    private class ContextImpl {
+
+        private final TimerService timerService;
+
+        ContextImpl(TimerService timerService) {
+            this.timerService = timerService;
+        }
+
+        public long timestamp() {
+            return timestamp;
+        }
+
+        public TimerService timerService() {
+            return timerService;
+        }
+
+        @SuppressWarnings("unchecked")
+        public K getCurrentKey() {
+            return (K) ((Row) EmbeddedPythonKeyedProcessOperator.this.getCurrentKey()).getField(0);
+        }
+    }
+
+    private class OnTimerContextImpl {
+
+        private final TimerService timerService;
+
+        private TimeDomain timeDomain;
+
+        private InternalTimer<K, VoidNamespace> timer;
+
+        OnTimerContextImpl(TimerService timerService) {
+            this.timerService = timerService;
+        }
+
+        public long timestamp() {
+            return timer.getTimestamp();
+        }
+
+        public TimerService timerService() {
+            return timerService;
+        }
+
+        public int timeDomain() {
+            return timeDomain.ordinal();
+        }
+
+        @SuppressWarnings("unchecked")
+        public K getCurrentKey() {

Review Comment:
   ditto



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org