Vancior commented on code in PR #20435: URL: https://github.com/apache/flink/pull/20435#discussion_r938583462
########## flink-python/pyflink/fn_execution/embedded/state_impl.py: ########## @@ -0,0 +1,80 @@ +################################################################################ +# 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. +################################################################################ + +from pemja import findClass + +from pyflink.datastream.state import (ValueStateDescriptor, ListStateDescriptor, MapStateDescriptor, + StateDescriptor, ReducingStateDescriptor, + AggregatingStateDescriptor) +from pyflink.fn_execution.datastream.embedded.state_impl import (ValueStateImpl, ListStateImpl, + MapStateImpl, ReducingStateImpl, + AggregatingStateImpl) +from pyflink.fn_execution.embedded.converters import from_type_info +from pyflink.fn_execution.embedded.java_utils import to_java_state_descriptor + +JVoidNamespace = findClass('org.apache.flink.runtime.state.VoidNamespace') +JVoidNamespace_INSTANCE = JVoidNamespace.INSTANCE + + +class KeyedStateBackend(object): + def __init__(self, function_context, keyed_state_backend, window_serializer, window_converter): + self._function_context = function_context + self._keyed_state_backend = keyed_state_backend + self._window_serializer = window_serializer + self._window_converter = window_converter + + def get_current_key(self): + return self._function_context.get_current_key() + + def get_value_state(self, state_descriptor: ValueStateDescriptor) -> ValueStateImpl: + return ValueStateImpl( + self._get_or_create_keyed_state(state_descriptor), + from_type_info(state_descriptor.type_info), + self._window_converter) + + def get_list_state(self, state_descriptor: ListStateDescriptor) -> ListStateImpl: + return ListStateImpl( + self._get_or_create_keyed_state(state_descriptor), + from_type_info(state_descriptor.type_info), + self._window_converter) + + def get_map_state(self, state_descriptor: MapStateDescriptor) -> MapStateImpl: + return MapStateImpl( + self._get_or_create_keyed_state(state_descriptor), + from_type_info(state_descriptor.type_info), + self._window_converter) + + def get_reducing_state(self, state_descriptor: ReducingStateDescriptor): + return ReducingStateImpl( + self._get_or_create_keyed_state(state_descriptor), + from_type_info(state_descriptor.type_info), + state_descriptor.get_reduce_function(), + self._window_converter) + + def get_aggregating_state(self, state_descriptor: AggregatingStateDescriptor): + return AggregatingStateImpl( + self._get_or_create_keyed_state(state_descriptor), + from_type_info(state_descriptor.type_info), + state_descriptor.get_agg_function(), + self._window_converter) + + def _get_or_create_keyed_state(self, state_descriptor: StateDescriptor): + return self._keyed_state_backend.getPartitionedState( Review Comment: Why not using getOrCreateKeyedState? ########## flink-python/pyflink/fn_execution/embedded/state_impl.py: ########## @@ -0,0 +1,80 @@ +################################################################################ +# 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. +################################################################################ + +from pemja import findClass + +from pyflink.datastream.state import (ValueStateDescriptor, ListStateDescriptor, MapStateDescriptor, + StateDescriptor, ReducingStateDescriptor, + AggregatingStateDescriptor) +from pyflink.fn_execution.datastream.embedded.state_impl import (ValueStateImpl, ListStateImpl, + MapStateImpl, ReducingStateImpl, + AggregatingStateImpl) +from pyflink.fn_execution.embedded.converters import from_type_info +from pyflink.fn_execution.embedded.java_utils import to_java_state_descriptor + +JVoidNamespace = findClass('org.apache.flink.runtime.state.VoidNamespace') +JVoidNamespace_INSTANCE = JVoidNamespace.INSTANCE + + +class KeyedStateBackend(object): Review Comment: How about using this implementation for all embeded operators? ########## flink-python/pyflink/fn_execution/datastream/embedded/state_impl.py: ########## @@ -88,13 +112,32 @@ def add(self, value: IN) -> None: self._state.update(self._value_converter.to_external(reduce_value)) + def merge_namespaces(self, target: N, sources: Collection[N]) -> None: + merged = None + for source in sources: + self.set_current_namespace(source) + source_state = self.get() + self.clear() + if merged and source_state: + if merged: + merged = self._reduce_function.reduce(merged, source_state) + else: + merged = source_state + elif merged is None: + merged = source_state + + if merged: Review Comment: `merged` might be `False` ########## flink-python/pyflink/fn_execution/datastream/embedded/state_impl.py: ########## @@ -88,13 +112,32 @@ def add(self, value: IN) -> None: self._state.update(self._value_converter.to_external(reduce_value)) + def merge_namespaces(self, target: N, sources: Collection[N]) -> None: + merged = None + for source in sources: + self.set_current_namespace(source) + source_state = self.get() + self.clear() + if merged and source_state: + if merged: + merged = self._reduce_function.reduce(merged, source_state) + else: Review Comment: This can never be reached. Maybe could be writed as early-skip: ```python if source_state is None: continue ``` ########## flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/embedded/EmbeddedPythonWindowOperator.java: ########## @@ -0,0 +1,216 @@ +/* + * 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.common.typeutils.TypeSerializer; +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.KeyedStateBackend; +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.streaming.api.operators.python.DataStreamPythonFunctionOperator; +import org.apache.flink.streaming.api.utils.PythonTypeUtils; +import org.apache.flink.table.runtime.operators.window.Window; +import org.apache.flink.types.Row; + +import pemja.core.object.PyIterator; + +import java.util.List; + +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; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * {@link EmbeddedPythonWindowOperator} is responsible for executing user defined python + * ProcessWindowFunction in embedded Python environment. + */ +@Internal +public class EmbeddedPythonWindowOperator<K, IN, OUT, W extends Window> + extends AbstractOneInputEmbeddedPythonFunctionOperator<IN, OUT> + implements Triggerable<K, W> { + + private static final long serialVersionUID = 1L; + + /** For serializing the window in checkpoints. */ + private final TypeSerializer<W> windowSerializer; + + /** The TypeInformation of the key. */ + private transient TypeInformation<K> keyTypeInfo; + + private transient PythonTypeUtils.DataConverter<K, Object> keyConverter; + + private transient WindowContextImpl windowContext; + + private transient WindowTimerContextImpl windowTimerContext; + + public EmbeddedPythonWindowOperator( + Configuration config, + DataStreamPythonFunctionInfo pythonFunctionInfo, + TypeInformation<IN> inputTypeInfo, + TypeInformation<OUT> outputTypeInfo, + TypeSerializer<W> windowSerializer) { + super(config, pythonFunctionInfo, inputTypeInfo, outputTypeInfo); + this.windowSerializer = checkNotNull(windowSerializer); + } + + @Override + public void open() throws Exception { + keyTypeInfo = ((RowTypeInfo) this.getInputTypeInfo()).getTypeAt(0); + + keyConverter = PythonTypeUtils.TypeInfoToDataConverter.typeInfoDataConverter(keyTypeInfo); + + InternalTimerService<W> internalTimerService = + getInternalTimerService("window-timers", windowSerializer, this); + + windowContext = new WindowContextImpl(internalTimerService); + + windowTimerContext = new WindowTimerContextImpl(internalTimerService); + + 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, W> timer) throws Exception { + collector.setAbsoluteTimestamp(timer.getTimestamp()); + invokeUserFunction(timer); + } + + @Override + public void onProcessingTime(InternalTimer<K, W> timer) throws Exception { + collector.eraseTimestamp(); + invokeUserFunction(timer); + } + + @Override + public Object getFunctionContext() { + return windowContext; + } + + @Override + public Object getTimerContext() { + return windowTimerContext; + } + + @Override + public <T> DataStreamPythonFunctionOperator<T> copy( + DataStreamPythonFunctionInfo pythonFunctionInfo, TypeInformation<T> outputTypeInfo) { + return null; + } + + private void invokeUserFunction(InternalTimer<K, W> timer) throws Exception { + windowTimerContext.timer = timer; + interpreter.invokeMethod("operation", "on_timer", timer.getTimestamp()); + + PyIterator results = + (PyIterator) + interpreter.invokeMethod("operation", "on_timer", timer.getTimestamp()); + + while (results.hasNext()) { + OUT result = outputDataConverter.toInternal(results.next()); + collector.collect(result); + } + results.close(); + + windowTimerContext.timer = null; + } + + private class WindowContextImpl { + private final InternalTimerService<W> timerService; + + WindowContextImpl(InternalTimerService<W> timerService) { + this.timerService = timerService; + } + + public TypeSerializer<W> getWindowSerializer() { + return windowSerializer; + } + + public KeyedStateBackend<K> getCurrentKeyedStateBackend() { + return getKeyedStateBackend(); + } + + public long timestamp() { + return timestamp; + } + + public InternalTimerService<W> timerService() { + return timerService; + } + + @SuppressWarnings("unchecked") + public Object getCurrentKey() { + return keyConverter.toExternal( + (K) ((Row) EmbeddedPythonWindowOperator.this.getCurrentKey()).getField(0)); + } + } + + private class WindowTimerContextImpl { Review Comment: This could be a static inner class? -- 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]
