diff --git a/flink-python/pyflink/fn_execution/beam/beam_coder_impl_fast.pxd b/flink-python/pyflink/fn_execution/beam/beam_coder_impl_fast.pxd index 3c5dac406770b..0afa27f9e068c 100644 --- a/flink-python/pyflink/fn_execution/beam/beam_coder_impl_fast.pxd +++ b/flink-python/pyflink/fn_execution/beam/beam_coder_impl_fast.pxd @@ -24,20 +24,16 @@ from apache_beam.coders.coder_impl cimport OutputStream as BOutputStream from apache_beam.coders.coder_impl cimport StreamCoderImpl from pyflink.fn_execution.coder_impl_fast cimport LengthPrefixBaseCoderImpl, FieldCoderImpl -from pyflink.fn_execution.stream_fast cimport LengthPrefixInputStream, OutputStream +from pyflink.fn_execution.stream_fast cimport OutputStream cdef class PassThroughLengthPrefixCoderImpl(StreamCoderImpl): cdef readonly StreamCoderImpl _value_coder -cdef class PassThroughPrefixCoderImpl(StreamCoderImpl): +cdef class FlinkFieldCoderBeamWrapper(StreamCoderImpl): cdef FieldCoderImpl _value_coder cdef OutputStream _data_out_stream cdef void _write_data_output_stream(self, BOutputStream out_stream) -cdef class BeamCoderImpl(StreamCoderImpl): +cdef class FlinkLengthPrefixCoderBeamWrapper(StreamCoderImpl): cdef readonly LengthPrefixBaseCoderImpl _value_coder - -cdef class InputStreamWrapper: - cdef LengthPrefixBaseCoderImpl _value_coder - cdef LengthPrefixInputStream _input_stream diff --git a/flink-python/pyflink/fn_execution/beam/beam_coder_impl_fast.pyx b/flink-python/pyflink/fn_execution/beam/beam_coder_impl_fast.pyx index da8da4cf9143a..4da3c5baa16ee 100644 --- a/flink-python/pyflink/fn_execution/beam/beam_coder_impl_fast.pyx +++ b/flink-python/pyflink/fn_execution/beam/beam_coder_impl_fast.pyx @@ -41,7 +41,10 @@ cdef class PassThroughLengthPrefixCoderImpl(StreamCoderImpl): cpdef get_estimated_size_and_observables(self, value, bint nested=False): return 0, [] -cdef class PassThroughPrefixCoderImpl(StreamCoderImpl): +cdef class FlinkFieldCoderBeamWrapper(StreamCoderImpl): + """ + Bridge between Beam coder and Flink coder for the low-level FieldCoder. + """ def __cinit__(self, value_coder): self._value_coder = value_coder self._data_out_stream = OutputStream() @@ -84,8 +87,10 @@ cdef class PassThroughPrefixCoderImpl(StreamCoderImpl): self._data_out_stream.pos = 0 - -cdef class BeamCoderImpl(StreamCoderImpl): +cdef class FlinkLengthPrefixCoderBeamWrapper(StreamCoderImpl): + """ + Bridge between Beam coder and Flink coder for the top-level LengthPrefixCoder. + """ def __cinit__(self, value_coder): self._value_coder = value_coder @@ -94,11 +99,4 @@ cdef class BeamCoderImpl(StreamCoderImpl): cpdef decode_from_stream(self, BInputStream in_stream, bint nested): cdef BeamInputStream input_stream = BeamInputStream(in_stream, in_stream.size()) - cdef InputStreamWrapper input_stream_wrapper = InputStreamWrapper(self._value_coder, - input_stream) - return input_stream_wrapper - -cdef class InputStreamWrapper: - def __cinit__(self, value_coder, input_stream): - self._value_coder = value_coder - self._input_stream = input_stream + return self._value_coder.decode_from_stream(input_stream) diff --git a/flink-python/pyflink/fn_execution/beam/beam_coder_impl_slow.py b/flink-python/pyflink/fn_execution/beam/beam_coder_impl_slow.py index cf67708b90e59..7cfa4c1f5bf76 100644 --- a/flink-python/pyflink/fn_execution/beam/beam_coder_impl_slow.py +++ b/flink-python/pyflink/fn_execution/beam/beam_coder_impl_slow.py @@ -40,7 +40,10 @@ def __repr__(self): return 'PassThroughLengthPrefixCoderImpl[%s]' % self._value_coder -class BeamCoderImpl(StreamCoderImpl): +class FlinkCoderBeamWrapper(StreamCoderImpl): + """ + Bridge between Beam coder and Flink coder. + """ def __init__(self, value_coder): self._value_coder = value_coder self._data_output_stream = OutputStream() @@ -55,4 +58,4 @@ def decode_from_stream(self, in_stream: create_InputStream, nested): return self._value_coder.decode_from_stream(data_input_stream) def __repr__(self): - return 'BeamCoderImpl[%s]' % self._value_coder + return 'FlinkCoderBeamWrapper[%s]' % self._value_coder diff --git a/flink-python/pyflink/fn_execution/beam/beam_coders.py b/flink-python/pyflink/fn_execution/beam/beam_coders.py index 53009623d2f61..a8afb273d7d61 100644 --- a/flink-python/pyflink/fn_execution/beam/beam_coders.py +++ b/flink-python/pyflink/fn_execution/beam/beam_coders.py @@ -20,15 +20,18 @@ from apache_beam.portability import common_urns from apache_beam.typehints import typehints -from pyflink.fn_execution.coders import ArrowCoder, OverWindowArrowCoder, LengthPrefixBaseCoder -from pyflink.fn_execution.flink_fn_execution_pb2 import CoderParam +from pyflink.fn_execution.coders import ( + ArrowCoder, OverWindowArrowCoder, LengthPrefixBaseCoder) +from pyflink.fn_execution.flink_fn_execution_pb2 import CoderInfoDescriptor try: from pyflink.fn_execution.beam import beam_coder_impl_fast as beam_coder_impl - from pyflink.fn_execution.beam.beam_coder_impl_fast import PassThroughPrefixCoderImpl + from pyflink.fn_execution.beam.beam_coder_impl_fast import FlinkFieldCoderBeamWrapper + from pyflink.fn_execution.beam.beam_coder_impl_fast import FlinkLengthPrefixCoderBeamWrapper except ImportError: from pyflink.fn_execution.beam import beam_coder_impl_slow as beam_coder_impl - PassThroughPrefixCoderImpl = beam_coder_impl.BeamCoderImpl + FlinkFieldCoderBeamWrapper = beam_coder_impl.FlinkCoderBeamWrapper + FlinkLengthPrefixCoderBeamWrapper = beam_coder_impl.FlinkCoderBeamWrapper FLINK_CODER_URN = "flink:coder:v1" @@ -64,19 +67,21 @@ def _create_impl(self): def get_impl(self): if isinstance(self._internal_coder, LengthPrefixBaseCoder): if isinstance(self._internal_coder._field_coder, (ArrowCoder, OverWindowArrowCoder)): - from pyflink.fn_execution.beam.beam_coder_impl_slow import BeamCoderImpl - return BeamCoderImpl(self._create_impl()) + from pyflink.fn_execution.beam.beam_coder_impl_slow import FlinkCoderBeamWrapper + return FlinkCoderBeamWrapper(self._create_impl()) else: - return beam_coder_impl.BeamCoderImpl(self._create_impl()) + return FlinkLengthPrefixCoderBeamWrapper(self._create_impl()) else: - return PassThroughPrefixCoderImpl(self._create_impl()) + return FlinkFieldCoderBeamWrapper(self._create_impl()) def to_type_hint(self): return typehints.Any - @Coder.register_urn(FLINK_CODER_URN, CoderParam) - def _pickle_from_runner_api_parameter(coder_praram_proto, unused_components, unused_context): - return FlinkCoder(LengthPrefixBaseCoder.from_coder_param_proto(coder_praram_proto)) + @Coder.register_urn(FLINK_CODER_URN, CoderInfoDescriptor) + def _pickle_from_runner_api_parameter( + coder_info_descriptor_proto, unused_components, unused_context): + return FlinkCoder(LengthPrefixBaseCoder.from_coder_info_descriptor_proto( + coder_info_descriptor_proto)) def __repr__(self): return 'FlinkCoder[%s]' % repr(self._internal_coder) diff --git a/flink-python/pyflink/fn_execution/beam/beam_operations_fast.pyx b/flink-python/pyflink/fn_execution/beam/beam_operations_fast.pyx index 4b363b4c0838f..fabc607b1abd7 100644 --- a/flink-python/pyflink/fn_execution/beam/beam_operations_fast.pyx +++ b/flink-python/pyflink/fn_execution/beam/beam_operations_fast.pyx @@ -22,9 +22,9 @@ from libc.stdint cimport * from apache_beam.utils.windowed_value cimport WindowedValue -from pyflink.fn_execution.coder_impl_fast cimport LengthPrefixBaseCoderImpl -from pyflink.fn_execution.beam.beam_stream_fast cimport BeamInputStream, BeamOutputStream -from pyflink.fn_execution.beam.beam_coder_impl_fast cimport InputStreamWrapper, BeamCoderImpl +from pyflink.fn_execution.beam.beam_stream_fast cimport BeamOutputStream +from pyflink.fn_execution.beam.beam_coder_impl_fast import FlinkLengthPrefixCoderBeamWrapper +from pyflink.fn_execution.coder_impl_fast cimport InputStreamWrapper from pyflink.fn_execution.table.operations import BundleOperation cdef class FunctionOperation(Operation): @@ -38,7 +38,7 @@ cdef class FunctionOperation(Operation): self.consumer = consumers['output'][0] self._value_coder_impl = self.consumer.windowed_coder.wrapped_value_coder.get_impl()._value_coder - if isinstance(self._value_coder_impl, BeamCoderImpl): + if isinstance(self._value_coder_impl, FlinkLengthPrefixCoderBeamWrapper): self._is_python_coder = False self._output_coder = self._value_coder_impl._value_coder else: @@ -64,8 +64,6 @@ cdef class FunctionOperation(Operation): cpdef process(self, WindowedValue o): cdef InputStreamWrapper input_stream_wrapper - cdef BeamInputStream input_stream - cdef LengthPrefixBaseCoderImpl input_coder cdef BeamOutputStream output_stream with self.scoped_process_state: if self._is_python_coder: @@ -75,19 +73,15 @@ cdef class FunctionOperation(Operation): self.consumer.output_stream.maybe_flush() else: input_stream_wrapper = o.value - input_stream = input_stream_wrapper._input_stream - input_coder = input_stream_wrapper._value_coder output_stream = BeamOutputStream(self.consumer.output_stream) if isinstance(self.operation, BundleOperation): - while input_stream.available(): - input_data = input_coder.decode_from_stream(input_stream) - self.process_element(input_data) + while input_stream_wrapper.has_next(): + self.process_element(input_stream_wrapper.next()) result = self.operation.finish_bundle() self._output_coder.encode_to_stream(result, output_stream) else: - while input_stream.available(): - input_data = input_coder.decode_from_stream(input_stream) - result = self.process_element(input_data) + while input_stream_wrapper.has_next(): + result = self.process_element(input_stream_wrapper.next()) self._output_coder.encode_to_stream(result, output_stream) output_stream.flush() diff --git a/flink-python/pyflink/fn_execution/coder_impl_fast.pxd b/flink-python/pyflink/fn_execution/coder_impl_fast.pxd index ed4af67fc4201..a6de3fcb35651 100644 --- a/flink-python/pyflink/fn_execution/coder_impl_fast.pxd +++ b/flink-python/pyflink/fn_execution/coder_impl_fast.pxd @@ -59,9 +59,16 @@ cdef class FieldCoderImpl: cpdef bytes encode(self, value) cpdef decode(self, encoded) +cdef class InputStreamWrapper: + cdef ValueCoderImpl _value_coder + cdef LengthPrefixInputStream _input_stream + + cpdef bint has_next(self) + cpdef next(self) + cdef class IterableCoderImpl(LengthPrefixBaseCoderImpl): cdef char*_end_message - cdef bint _writes_end_message + cdef bint _separated_with_end_message cdef class ValueCoderImpl(LengthPrefixBaseCoderImpl): pass diff --git a/flink-python/pyflink/fn_execution/coder_impl_fast.pyx b/flink-python/pyflink/fn_execution/coder_impl_fast.pyx index 0dafb13299a82..0c08a32296472 100644 --- a/flink-python/pyflink/fn_execution/coder_impl_fast.pyx +++ b/flink-python/pyflink/fn_execution/coder_impl_fast.pyx @@ -182,19 +182,7 @@ cdef class LengthPrefixBaseCoderImpl: pass cpdef decode_from_stream(self, LengthPrefixInputStream input_stream): - cdef char*input_data - cdef char*temp - cdef size_t size - cdef long long addr - cdef InputStream data_input_stream - # set input_data pointer to the input data - size = input_stream.read(&input_data) - - # create InputStream - data_input_stream = InputStream() - data_input_stream._input_data = input_data - - return self._field_coder.decode_from_stream(data_input_stream, size) + pass cdef void _write_data_to_output_stream(self, LengthPrefixOutputStream output_stream): cdef OutputStream data_out_stream @@ -247,6 +235,17 @@ cdef class FieldCoderImpl: input_stream._input_data = encoded return self.decode_from_stream(input_stream, len(encoded)) +cdef class InputStreamWrapper: + def __cinit__(self, value_coder: ValueCoderImpl, input_stream: LengthPrefixInputStream): + self._value_coder = value_coder + self._input_stream = input_stream + + cpdef bint has_next(self): + return self._input_stream.available() + + cpdef next(self): + return self._value_coder.decode_from_stream(self._input_stream) + cdef class IterableCoderImpl(LengthPrefixBaseCoderImpl): """ Encodes iterable data to output stream. The output mode will decide whether write a special end @@ -259,12 +258,9 @@ cdef class IterableCoderImpl(LengthPrefixBaseCoderImpl): raise MemoryError() self._end_message[0] = 0x00 - def __init__(self, field_coder: FieldCoderImpl, output_mode): + def __init__(self, field_coder: FieldCoderImpl, separated_with_end_message: bool): super(IterableCoderImpl, self).__init__(field_coder) - if output_mode == flink_fn_execution_pb2.CoderParam.MULTIPLE: - self._writes_end_message = False - else: - self._writes_end_message = True + self._separated_with_end_message = separated_with_end_message cpdef encode_to_stream(self, value, LengthPrefixOutputStream output_stream): for item in value: @@ -272,9 +268,12 @@ cdef class IterableCoderImpl(LengthPrefixBaseCoderImpl): self._write_data_to_output_stream(output_stream) # write end message - if self._writes_end_message: + if self._separated_with_end_message: output_stream.write(self._end_message, 1) + cpdef decode_from_stream(self, LengthPrefixInputStream input_stream): + return InputStreamWrapper(ValueCoderImpl(self._field_coder), input_stream) + def __dealloc__(self): if self._end_message != NULL: free(self._end_message) @@ -291,6 +290,21 @@ cdef class ValueCoderImpl(LengthPrefixBaseCoderImpl): self._field_coder.encode_to_stream(value, self._data_out_stream) self._write_data_to_output_stream(output_stream) + cpdef decode_from_stream(self, LengthPrefixInputStream input_stream): + cdef char*input_data + cdef char*temp + cdef size_t size + cdef long long addr + cdef InputStream data_input_stream + # set input_data pointer to the input data + size = input_stream.read(&input_data) + + # create InputStream + data_input_stream = InputStream() + data_input_stream._input_data = input_data + + return self._field_coder.decode_from_stream(data_input_stream, size) + cdef class FlattenRowCoderImpl(FieldCoderImpl): """ A coder for flatten row (List) object (without field names and row kind value is 0). diff --git a/flink-python/pyflink/fn_execution/coder_impl_slow.py b/flink-python/pyflink/fn_execution/coder_impl_slow.py index 1dafc468233c4..924232f5dd107 100644 --- a/flink-python/pyflink/fn_execution/coder_impl_slow.py +++ b/flink-python/pyflink/fn_execution/coder_impl_slow.py @@ -27,7 +27,6 @@ from pyflink.common import Row, RowKind from pyflink.datastream.window import TimeWindow, CountWindow from pyflink.fn_execution.ResettableIO import ResettableIO -from pyflink.fn_execution.flink_fn_execution_pb2 import CoderParam from pyflink.fn_execution.stream_slow import InputStream, OutputStream from pyflink.table.utils import pandas_to_arrow, arrow_to_pandas @@ -44,10 +43,6 @@ def __init__(self, field_coder: 'FieldCoderImpl'): self._field_coder = field_coder self._data_out_stream = OutputStream() - def decode_from_stream(self, in_stream: InputStream): - while in_stream.size() > 0: - yield self._field_coder.decode_from_stream(in_stream, in_stream.read_var_int64()) - def _write_data_to_output_stream(self, out_stream: OutputStream): out_stream.write_var_int64(self._data_out_stream.size()) out_stream.write(self._data_out_stream.get()) @@ -92,9 +87,9 @@ class IterableCoderImpl(LengthPrefixBaseCoderImpl): message 0x00 to output stream after encoding data. """ - def __init__(self, field_coder: 'FieldCoderImpl', output_mode): + def __init__(self, field_coder: 'FieldCoderImpl', separated_with_end_message: bool): super(IterableCoderImpl, self).__init__(field_coder) - self._output_mode = output_mode + self._separated_with_end_message = separated_with_end_message def encode_to_stream(self, value: List, out_stream): for item in value: @@ -102,10 +97,14 @@ def encode_to_stream(self, value: List, out_stream): self._write_data_to_output_stream(out_stream) # write end message - if self._output_mode == CoderParam.MULTIPLE_WITH_END: + if self._separated_with_end_message: out_stream.write_var_int64(1) out_stream.write_byte(0x00) + def decode_from_stream(self, in_stream: InputStream): + while in_stream.size() > 0: + yield self._field_coder.decode_from_stream(in_stream, in_stream.read_var_int64()) + class ValueCoderImpl(LengthPrefixBaseCoderImpl): """ @@ -119,6 +118,9 @@ def encode_to_stream(self, value, out_stream): self._field_coder.encode_to_stream(value, self._data_out_stream) self._write_data_to_output_stream(out_stream) + def decode_from_stream(self, in_stream: InputStream): + return self._field_coder.decode_from_stream(in_stream, in_stream.read_var_int64()) + class MaskUtils: """ diff --git a/flink-python/pyflink/fn_execution/coders.py b/flink-python/pyflink/fn_execution/coders.py index 488741c8982e8..42502bb522ebc 100644 --- a/flink-python/pyflink/fn_execution/coders.py +++ b/flink-python/pyflink/fn_execution/coders.py @@ -43,8 +43,11 @@ 'PickleCoder', 'CloudPickleCoder', 'DataViewFilterCoder'] -# LengthPrefixBaseCoder will be used in Operations and other coders will be the field coder -# of LengthPrefixBaseCoder +######################################################################### +# Top-level coder: ValueCoder & IterableCoder +######################################################################### + +# LengthPrefixBaseCoder is the top level coder and the other coders will be used as the field coder class LengthPrefixBaseCoder(ABC): def __init__(self, field_coder: 'FieldCoder'): self._field_coder = field_coder @@ -54,48 +57,43 @@ def get_impl(self): pass @classmethod - def from_coder_param_proto(cls, coder_param_proto): - field_coder = cls._to_field_coder(coder_param_proto) - output_mode = coder_param_proto.output_mode - if output_mode == flink_fn_execution_pb2.CoderParam.SINGLE: + def from_coder_info_descriptor_proto(cls, coder_info_descriptor_proto): + field_coder = cls._to_field_coder(coder_info_descriptor_proto) + mode = coder_info_descriptor_proto.mode + separated_with_end_message = coder_info_descriptor_proto.separated_with_end_message + if mode == flink_fn_execution_pb2.CoderInfoDescriptor.SINGLE: return ValueCoder(field_coder) else: - return IterableCoder(field_coder, output_mode) + return IterableCoder(field_coder, separated_with_end_message) @classmethod - def _to_field_coder(cls, coder_param_proto): - data_type = coder_param_proto.data_type - if data_type == flink_fn_execution_pb2.CoderParam.FLATTEN_ROW: - if coder_param_proto.HasField('schema'): - schema_proto = coder_param_proto.schema - field_coders = [from_proto(f.type) for f in schema_proto.fields] - else: - type_info_proto = coder_param_proto.type_info - field_coders = [from_type_info_proto(f.field_type) - for f in type_info_proto.row_type_info.fields] + def _to_field_coder(cls, coder_info_descriptor_proto): + if coder_info_descriptor_proto.HasField('flatten_row_type'): + schema_proto = coder_info_descriptor_proto.flatten_row_type.schema + field_coders = [from_proto(f.type) for f in schema_proto.fields] return FlattenRowCoder(field_coders) - elif data_type == flink_fn_execution_pb2.CoderParam.ROW: - schema_proto = coder_param_proto.schema + elif coder_info_descriptor_proto.HasField('row_type'): + schema_proto = coder_info_descriptor_proto.row_type.schema field_coders = [from_proto(f.type) for f in schema_proto.fields] field_names = [f.name for f in schema_proto.fields] return RowCoder(field_coders, field_names) - elif data_type == flink_fn_execution_pb2.CoderParam.RAW: - type_info_proto = coder_param_proto.type_info - field_coder = from_type_info_proto(type_info_proto) - return field_coder - elif data_type == flink_fn_execution_pb2.CoderParam.ARROW: + elif coder_info_descriptor_proto.HasField('arrow_type'): timezone = pytz.timezone(os.environ['table.exec.timezone']) - schema_proto = coder_param_proto.schema + schema_proto = coder_info_descriptor_proto.arrow_type.schema row_type = cls._to_row_type(schema_proto) return ArrowCoder(cls._to_arrow_schema(row_type), row_type, timezone) - elif data_type == flink_fn_execution_pb2.CoderParam.OVER_WINDOW_ARROW: + elif coder_info_descriptor_proto.HasField('over_window_arrow_type'): timezone = pytz.timezone(os.environ['table.exec.timezone']) - schema_proto = coder_param_proto.schema + schema_proto = coder_info_descriptor_proto.over_window_arrow_type.schema row_type = cls._to_row_type(schema_proto) return OverWindowArrowCoder( cls._to_arrow_schema(row_type), row_type, timezone) + elif coder_info_descriptor_proto.HasField('raw_type'): + type_info_proto = coder_info_descriptor_proto.raw_type.type_info + field_coder = from_type_info_proto(type_info_proto) + return field_coder else: - raise ValueError("Unexpected coder type %s" % data_type) + raise ValueError("Unexpected coder type %s" % coder_info_descriptor_proto) @classmethod def _to_arrow_schema(cls, row_type): @@ -151,26 +149,24 @@ def _to_row_type(cls, row_schema): return RowType([RowField(f.name, cls._to_data_type(f.type)) for f in row_schema.fields]) -class FieldCoder(ABC): - - def get_impl(self) -> coder_impl.FieldCoderImpl: - pass - - def __eq__(self, other): - return type(self) == type(other) - - class IterableCoder(LengthPrefixBaseCoder): """ Coder for iterable data. """ - def __init__(self, field_coder: FieldCoder, output_mode): + def __init__(self, field_coder: 'FieldCoder', separated_with_end_message): super(IterableCoder, self).__init__(field_coder) - self._output_mode = output_mode + self._separated_with_end_message = separated_with_end_message def get_impl(self): - return coder_impl.IterableCoderImpl(self._field_coder.get_impl(), self._output_mode) + if isinstance(self._field_coder, (ArrowCoder, OverWindowArrowCoder)): + # ArrowCoder and OverWindowArrowCoder doesn't support fast coder currently. + from pyflink.fn_execution import coder_impl_slow + return coder_impl_slow.IterableCoderImpl(self._field_coder.get_impl(), + self._separated_with_end_message) + else: + return coder_impl.IterableCoderImpl(self._field_coder.get_impl(), + self._separated_with_end_message) class ValueCoder(LengthPrefixBaseCoder): @@ -178,7 +174,7 @@ class ValueCoder(LengthPrefixBaseCoder): Coder for single data. """ - def __init__(self, field_coder: FieldCoder): + def __init__(self, field_coder: 'FieldCoder'): super(ValueCoder, self).__init__(field_coder) def get_impl(self): @@ -190,6 +186,20 @@ def get_impl(self): return coder_impl.ValueCoderImpl(self._field_coder.get_impl()) +######################################################################### +# Low-level coder: FieldCoder +######################################################################### + + +class FieldCoder(ABC): + + def get_impl(self) -> coder_impl.FieldCoderImpl: + pass + + def __eq__(self, other): + return type(self) == type(other) + + class FlattenRowCoder(FieldCoder): """ Coder for Row. The decoded result will be flattened as a list of column values of a row instead diff --git a/flink-python/pyflink/fn_execution/flink_fn_execution_pb2.py b/flink-python/pyflink/fn_execution/flink_fn_execution_pb2.py index e0e09340875ef..cb58fbfb14bbb 100644 --- a/flink-python/pyflink/fn_execution/flink_fn_execution_pb2.py +++ b/flink-python/pyflink/fn_execution/flink_fn_execution_pb2.py @@ -36,7 +36,7 @@ name='flink-fn-execution.proto', package='org.apache.flink.fn_execution.v1', syntax='proto3', - serialized_pb=_b('\n\x18\x66link-fn-execution.proto\x12 org.apache.flink.fn_execution.v1\"\x86\x01\n\x05Input\x12\x44\n\x03udf\x18\x01 \x01(\x0b\x32\x35.org.apache.flink.fn_execution.v1.UserDefinedFunctionH\x00\x12\x15\n\x0binputOffset\x18\x02 \x01(\x05H\x00\x12\x17\n\rinputConstant\x18\x03 \x01(\x0cH\x00\x42\x07\n\x05input\"\xa8\x01\n\x13UserDefinedFunction\x12\x0f\n\x07payload\x18\x01 \x01(\x0c\x12\x37\n\x06inputs\x18\x02 \x03(\x0b\x32\'.org.apache.flink.fn_execution.v1.Input\x12\x14\n\x0cwindow_index\x18\x03 \x01(\x05\x12\x1a\n\x12takes_row_as_input\x18\x04 \x01(\x08\x12\x15\n\ris_pandas_udf\x18\x05 \x01(\x08\"\xb2\x01\n\x14UserDefinedFunctions\x12\x43\n\x04udfs\x18\x01 \x03(\x0b\x32\x35.org.apache.flink.fn_execution.v1.UserDefinedFunction\x12\x16\n\x0emetric_enabled\x18\x02 \x01(\x08\x12=\n\x07windows\x18\x03 \x03(\x0b\x32,.org.apache.flink.fn_execution.v1.OverWindow\"\xdd\x02\n\nOverWindow\x12L\n\x0bwindow_type\x18\x01 \x01(\x0e\x32\x37.org.apache.flink.fn_execution.v1.OverWindow.WindowType\x12\x16\n\x0elower_boundary\x18\x02 \x01(\x03\x12\x16\n\x0eupper_boundary\x18\x03 \x01(\x03\"\xd0\x01\n\nWindowType\x12\x13\n\x0fRANGE_UNBOUNDED\x10\x00\x12\x1d\n\x19RANGE_UNBOUNDED_PRECEDING\x10\x01\x12\x1d\n\x19RANGE_UNBOUNDED_FOLLOWING\x10\x02\x12\x11\n\rRANGE_SLIDING\x10\x03\x12\x11\n\rROW_UNBOUNDED\x10\x04\x12\x1b\n\x17ROW_UNBOUNDED_PRECEDING\x10\x05\x12\x1b\n\x17ROW_UNBOUNDED_FOLLOWING\x10\x06\x12\x0f\n\x0bROW_SLIDING\x10\x07\"\xf7\x06\n\x1dUserDefinedDataStreamFunction\x12\x63\n\rfunction_type\x18\x01 \x01(\x0e\x32L.org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.FunctionType\x12g\n\x0fruntime_context\x18\x02 \x01(\x0b\x32N.org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.RuntimeContext\x12\x0f\n\x07payload\x18\x03 \x01(\x0c\x12\x16\n\x0emetric_enabled\x18\x04 \x01(\x08\x12\x41\n\rkey_type_info\x18\x05 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfo\x1a*\n\x0cJobParameter\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t\x1a\xd0\x02\n\x0eRuntimeContext\x12\x11\n\ttask_name\x18\x01 \x01(\t\x12\x1f\n\x17task_name_with_subtasks\x18\x02 \x01(\t\x12#\n\x1bnumber_of_parallel_subtasks\x18\x03 \x01(\x05\x12\'\n\x1fmax_number_of_parallel_subtasks\x18\x04 \x01(\x05\x12\x1d\n\x15index_of_this_subtask\x18\x05 \x01(\x05\x12\x16\n\x0e\x61ttempt_number\x18\x06 \x01(\x05\x12\x64\n\x0ejob_parameters\x18\x07 \x03(\x0b\x32L.org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.JobParameter\x12\x1f\n\x17in_batch_execution_mode\x18\x08 \x01(\x08\"\x9c\x01\n\x0c\x46unctionType\x12\x07\n\x03MAP\x10\x00\x12\x0c\n\x08\x46LAT_MAP\x10\x01\x12\n\n\x06\x43O_MAP\x10\x02\x12\x0f\n\x0b\x43O_FLAT_MAP\x10\x03\x12\x0b\n\x07PROCESS\x10\x04\x12\x11\n\rKEYED_PROCESS\x10\x05\x12\x14\n\x10KEYED_CO_PROCESS\x10\x06\x12\x16\n\x12TIMESTAMP_ASSIGNER\x10\x07\x12\n\n\x06WINDOW\x10\x08\"\x8b\x06\n\x1cUserDefinedAggregateFunction\x12\x0f\n\x07payload\x18\x01 \x01(\x0c\x12\x37\n\x06inputs\x18\x02 \x03(\x0b\x32\'.org.apache.flink.fn_execution.v1.Input\x12Z\n\x05specs\x18\x03 \x03(\x0b\x32K.org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec\x12\x12\n\nfilter_arg\x18\x04 \x01(\x05\x12\x10\n\x08\x64istinct\x18\x05 \x01(\x08\x12\x1a\n\x12takes_row_as_input\x18\x06 \x01(\x08\x1a\x82\x04\n\x0c\x44\x61taViewSpec\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x13\n\x0b\x66ield_index\x18\x02 \x01(\x05\x12i\n\tlist_view\x18\x03 \x01(\x0b\x32T.org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec.ListViewH\x00\x12g\n\x08map_view\x18\x04 \x01(\x0b\x32S.org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec.MapViewH\x00\x1aT\n\x08ListView\x12H\n\x0c\x65lement_type\x18\x01 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x1a\x97\x01\n\x07MapView\x12\x44\n\x08key_type\x18\x01 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x12\x46\n\nvalue_type\x18\x02 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldTypeB\x0b\n\tdata_view\"\xac\x04\n\x0bGroupWindow\x12M\n\x0bwindow_type\x18\x01 \x01(\x0e\x32\x38.org.apache.flink.fn_execution.v1.GroupWindow.WindowType\x12\x16\n\x0eis_time_window\x18\x02 \x01(\x08\x12\x14\n\x0cwindow_slide\x18\x03 \x01(\x03\x12\x13\n\x0bwindow_size\x18\x04 \x01(\x03\x12\x12\n\nwindow_gap\x18\x05 \x01(\x03\x12\x13\n\x0bis_row_time\x18\x06 \x01(\x08\x12\x18\n\x10time_field_index\x18\x07 \x01(\x05\x12\x17\n\x0f\x61llowedLateness\x18\x08 \x01(\x03\x12U\n\x0fnamedProperties\x18\t \x03(\x0e\x32<.org.apache.flink.fn_execution.v1.GroupWindow.WindowProperty\x12\x16\n\x0eshift_timezone\x18\n \x01(\t\"[\n\nWindowType\x12\x19\n\x15TUMBLING_GROUP_WINDOW\x10\x00\x12\x18\n\x14SLIDING_GROUP_WINDOW\x10\x01\x12\x18\n\x14SESSION_GROUP_WINDOW\x10\x02\"c\n\x0eWindowProperty\x12\x10\n\x0cWINDOW_START\x10\x00\x12\x0e\n\nWINDOW_END\x10\x01\x12\x16\n\x12ROW_TIME_ATTRIBUTE\x10\x02\x12\x17\n\x13PROC_TIME_ATTRIBUTE\x10\x03\"\xfd\x03\n\x1dUserDefinedAggregateFunctions\x12L\n\x04udfs\x18\x01 \x03(\x0b\x32>.org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction\x12\x16\n\x0emetric_enabled\x18\x02 \x01(\x08\x12\x10\n\x08grouping\x18\x03 \x03(\x05\x12\x1e\n\x16generate_update_before\x18\x04 \x01(\x08\x12\x44\n\x08key_type\x18\x05 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x12\x1b\n\x13index_of_count_star\x18\x06 \x01(\x05\x12\x1e\n\x16state_cleaning_enabled\x18\x07 \x01(\x08\x12\x18\n\x10state_cache_size\x18\x08 \x01(\x05\x12!\n\x19map_state_read_cache_size\x18\t \x01(\x05\x12\"\n\x1amap_state_write_cache_size\x18\n \x01(\x05\x12\x1b\n\x13\x63ount_star_inserted\x18\x0b \x01(\x08\x12\x43\n\x0cgroup_window\x18\x0c \x01(\x0b\x32-.org.apache.flink.fn_execution.v1.GroupWindow\"\xec\x0f\n\x06Schema\x12>\n\x06\x66ields\x18\x01 \x03(\x0b\x32..org.apache.flink.fn_execution.v1.Schema.Field\x1a\x97\x01\n\x07MapInfo\x12\x44\n\x08key_type\x18\x01 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x12\x46\n\nvalue_type\x18\x02 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x1a\x1d\n\x08TimeInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a\"\n\rTimestampInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a,\n\x17LocalZonedTimestampInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a\'\n\x12ZonedTimestampInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a/\n\x0b\x44\x65\x63imalInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x12\r\n\x05scale\x18\x02 \x01(\x05\x1a\x1c\n\nBinaryInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\x1f\n\rVarBinaryInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\x1a\n\x08\x43harInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\x1d\n\x0bVarCharInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\xb0\x08\n\tFieldType\x12\x44\n\ttype_name\x18\x01 \x01(\x0e\x32\x31.org.apache.flink.fn_execution.v1.Schema.TypeName\x12\x10\n\x08nullable\x18\x02 \x01(\x08\x12U\n\x17\x63ollection_element_type\x18\x03 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldTypeH\x00\x12\x44\n\x08map_info\x18\x04 \x01(\x0b\x32\x30.org.apache.flink.fn_execution.v1.Schema.MapInfoH\x00\x12>\n\nrow_schema\x18\x05 \x01(\x0b\x32(.org.apache.flink.fn_execution.v1.SchemaH\x00\x12L\n\x0c\x64\x65\x63imal_info\x18\x06 \x01(\x0b\x32\x34.org.apache.flink.fn_execution.v1.Schema.DecimalInfoH\x00\x12\x46\n\ttime_info\x18\x07 \x01(\x0b\x32\x31.org.apache.flink.fn_execution.v1.Schema.TimeInfoH\x00\x12P\n\x0etimestamp_info\x18\x08 \x01(\x0b\x32\x36.org.apache.flink.fn_execution.v1.Schema.TimestampInfoH\x00\x12\x66\n\x1alocal_zoned_timestamp_info\x18\t \x01(\x0b\x32@.org.apache.flink.fn_execution.v1.Schema.LocalZonedTimestampInfoH\x00\x12[\n\x14zoned_timestamp_info\x18\n \x01(\x0b\x32;.org.apache.flink.fn_execution.v1.Schema.ZonedTimestampInfoH\x00\x12J\n\x0b\x62inary_info\x18\x0b \x01(\x0b\x32\x33.org.apache.flink.fn_execution.v1.Schema.BinaryInfoH\x00\x12Q\n\x0fvar_binary_info\x18\x0c \x01(\x0b\x32\x36.org.apache.flink.fn_execution.v1.Schema.VarBinaryInfoH\x00\x12\x46\n\tchar_info\x18\r \x01(\x0b\x32\x31.org.apache.flink.fn_execution.v1.Schema.CharInfoH\x00\x12M\n\rvar_char_info\x18\x0e \x01(\x0b\x32\x34.org.apache.flink.fn_execution.v1.Schema.VarCharInfoH\x00\x42\x0b\n\ttype_info\x1al\n\x05\x46ield\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x13\n\x0b\x64\x65scription\x18\x02 \x01(\t\x12@\n\x04type\x18\x03 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\"\xa1\x02\n\x08TypeName\x12\x07\n\x03ROW\x10\x00\x12\x0b\n\x07TINYINT\x10\x01\x12\x0c\n\x08SMALLINT\x10\x02\x12\x07\n\x03INT\x10\x03\x12\n\n\x06\x42IGINT\x10\x04\x12\x0b\n\x07\x44\x45\x43IMAL\x10\x05\x12\t\n\x05\x46LOAT\x10\x06\x12\n\n\x06\x44OUBLE\x10\x07\x12\x08\n\x04\x44\x41TE\x10\x08\x12\x08\n\x04TIME\x10\t\x12\r\n\tTIMESTAMP\x10\n\x12\x0b\n\x07\x42OOLEAN\x10\x0b\x12\n\n\x06\x42INARY\x10\x0c\x12\r\n\tVARBINARY\x10\r\x12\x08\n\x04\x43HAR\x10\x0e\x12\x0b\n\x07VARCHAR\x10\x0f\x12\x0f\n\x0b\x42\x41SIC_ARRAY\x10\x10\x12\x07\n\x03MAP\x10\x11\x12\x0c\n\x08MULTISET\x10\x12\x12\x19\n\x15LOCAL_ZONED_TIMESTAMP\x10\x13\x12\x13\n\x0fZONED_TIMESTAMP\x10\x14\"\xbf\x03\n\nCoderParam\x12:\n\x06schema\x18\x01 \x01(\x0b\x32(.org.apache.flink.fn_execution.v1.SchemaH\x00\x12?\n\ttype_info\x18\x02 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfoH\x00\x12H\n\tdata_type\x18\x03 \x01(\x0e\x32\x35.org.apache.flink.fn_execution.v1.CoderParam.DataType\x12L\n\x0boutput_mode\x18\x04 \x01(\x0e\x32\x37.org.apache.flink.fn_execution.v1.CoderParam.OutputMode\"O\n\x08\x44\x61taType\x12\x0f\n\x0b\x46LATTEN_ROW\x10\x00\x12\x07\n\x03ROW\x10\x01\x12\x07\n\x03RAW\x10\x02\x12\t\n\x05\x41RROW\x10\x03\x12\x15\n\x11OVER_WINDOW_ARROW\x10\x04\"=\n\nOutputMode\x12\n\n\x06SINGLE\x10\x00\x12\x0c\n\x08MULTIPLE\x10\x01\x12\x15\n\x11MULTIPLE_WITH_END\x10\x02\x42\x0c\n\nfield_type\"\xea\x08\n\x08TypeInfo\x12\x46\n\ttype_name\x18\x01 \x01(\x0e\x32\x33.org.apache.flink.fn_execution.v1.TypeInfo.TypeName\x12M\n\x17\x63ollection_element_type\x18\x02 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfoH\x00\x12O\n\rrow_type_info\x18\x03 \x01(\x0b\x32\x36.org.apache.flink.fn_execution.v1.TypeInfo.RowTypeInfoH\x00\x12S\n\x0ftuple_type_info\x18\x04 \x01(\x0b\x32\x38.org.apache.flink.fn_execution.v1.TypeInfo.TupleTypeInfoH\x00\x12O\n\rmap_type_info\x18\x05 \x01(\x0b\x32\x36.org.apache.flink.fn_execution.v1.TypeInfo.MapTypeInfoH\x00\x1a\x8b\x01\n\x0bMapTypeInfo\x12<\n\x08key_type\x18\x01 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfo\x12>\n\nvalue_type\x18\x02 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfo\x1a\xb8\x01\n\x0bRowTypeInfo\x12L\n\x06\x66ields\x18\x01 \x03(\x0b\x32<.org.apache.flink.fn_execution.v1.TypeInfo.RowTypeInfo.Field\x1a[\n\x05\x46ield\x12\x12\n\nfield_name\x18\x01 \x01(\t\x12>\n\nfield_type\x18\x02 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfo\x1aP\n\rTupleTypeInfo\x12?\n\x0b\x66ield_types\x18\x01 \x03(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfo\"\xa7\x02\n\x08TypeName\x12\x07\n\x03ROW\x10\x00\x12\n\n\x06STRING\x10\x01\x12\x08\n\x04\x42YTE\x10\x02\x12\x0b\n\x07\x42OOLEAN\x10\x03\x12\t\n\x05SHORT\x10\x04\x12\x07\n\x03INT\x10\x05\x12\x08\n\x04LONG\x10\x06\x12\t\n\x05\x46LOAT\x10\x07\x12\n\n\x06\x44OUBLE\x10\x08\x12\x08\n\x04\x43HAR\x10\t\x12\x0b\n\x07\x42IG_INT\x10\n\x12\x0b\n\x07\x42IG_DEC\x10\x0b\x12\x0c\n\x08SQL_DATE\x10\x0c\x12\x0c\n\x08SQL_TIME\x10\r\x12\x11\n\rSQL_TIMESTAMP\x10\x0e\x12\x0f\n\x0b\x42\x41SIC_ARRAY\x10\x0f\x12\x13\n\x0fPRIMITIVE_ARRAY\x10\x10\x12\t\n\x05TUPLE\x10\x11\x12\x08\n\x04LIST\x10\x12\x12\x07\n\x03MAP\x10\x13\x12\x11\n\rPICKLED_BYTES\x10\x14\x12\x10\n\x0cOBJECT_ARRAY\x10\x15\x42\x0b\n\ttype_infoB-\n\x1forg.apache.flink.fnexecution.v1B\nFlinkFnApib\x06proto3') + serialized_pb=_b('\n\x18\x66link-fn-execution.proto\x12 org.apache.flink.fn_execution.v1\"\x86\x01\n\x05Input\x12\x44\n\x03udf\x18\x01 \x01(\x0b\x32\x35.org.apache.flink.fn_execution.v1.UserDefinedFunctionH\x00\x12\x15\n\x0binputOffset\x18\x02 \x01(\x05H\x00\x12\x17\n\rinputConstant\x18\x03 \x01(\x0cH\x00\x42\x07\n\x05input\"\xa8\x01\n\x13UserDefinedFunction\x12\x0f\n\x07payload\x18\x01 \x01(\x0c\x12\x37\n\x06inputs\x18\x02 \x03(\x0b\x32\'.org.apache.flink.fn_execution.v1.Input\x12\x14\n\x0cwindow_index\x18\x03 \x01(\x05\x12\x1a\n\x12takes_row_as_input\x18\x04 \x01(\x08\x12\x15\n\ris_pandas_udf\x18\x05 \x01(\x08\"\xb2\x01\n\x14UserDefinedFunctions\x12\x43\n\x04udfs\x18\x01 \x03(\x0b\x32\x35.org.apache.flink.fn_execution.v1.UserDefinedFunction\x12\x16\n\x0emetric_enabled\x18\x02 \x01(\x08\x12=\n\x07windows\x18\x03 \x03(\x0b\x32,.org.apache.flink.fn_execution.v1.OverWindow\"\xdd\x02\n\nOverWindow\x12L\n\x0bwindow_type\x18\x01 \x01(\x0e\x32\x37.org.apache.flink.fn_execution.v1.OverWindow.WindowType\x12\x16\n\x0elower_boundary\x18\x02 \x01(\x03\x12\x16\n\x0eupper_boundary\x18\x03 \x01(\x03\"\xd0\x01\n\nWindowType\x12\x13\n\x0fRANGE_UNBOUNDED\x10\x00\x12\x1d\n\x19RANGE_UNBOUNDED_PRECEDING\x10\x01\x12\x1d\n\x19RANGE_UNBOUNDED_FOLLOWING\x10\x02\x12\x11\n\rRANGE_SLIDING\x10\x03\x12\x11\n\rROW_UNBOUNDED\x10\x04\x12\x1b\n\x17ROW_UNBOUNDED_PRECEDING\x10\x05\x12\x1b\n\x17ROW_UNBOUNDED_FOLLOWING\x10\x06\x12\x0f\n\x0bROW_SLIDING\x10\x07\"\x8b\x06\n\x1cUserDefinedAggregateFunction\x12\x0f\n\x07payload\x18\x01 \x01(\x0c\x12\x37\n\x06inputs\x18\x02 \x03(\x0b\x32\'.org.apache.flink.fn_execution.v1.Input\x12Z\n\x05specs\x18\x03 \x03(\x0b\x32K.org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec\x12\x12\n\nfilter_arg\x18\x04 \x01(\x05\x12\x10\n\x08\x64istinct\x18\x05 \x01(\x08\x12\x1a\n\x12takes_row_as_input\x18\x06 \x01(\x08\x1a\x82\x04\n\x0c\x44\x61taViewSpec\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x13\n\x0b\x66ield_index\x18\x02 \x01(\x05\x12i\n\tlist_view\x18\x03 \x01(\x0b\x32T.org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec.ListViewH\x00\x12g\n\x08map_view\x18\x04 \x01(\x0b\x32S.org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec.MapViewH\x00\x1aT\n\x08ListView\x12H\n\x0c\x65lement_type\x18\x01 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x1a\x97\x01\n\x07MapView\x12\x44\n\x08key_type\x18\x01 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x12\x46\n\nvalue_type\x18\x02 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldTypeB\x0b\n\tdata_view\"\xac\x04\n\x0bGroupWindow\x12M\n\x0bwindow_type\x18\x01 \x01(\x0e\x32\x38.org.apache.flink.fn_execution.v1.GroupWindow.WindowType\x12\x16\n\x0eis_time_window\x18\x02 \x01(\x08\x12\x14\n\x0cwindow_slide\x18\x03 \x01(\x03\x12\x13\n\x0bwindow_size\x18\x04 \x01(\x03\x12\x12\n\nwindow_gap\x18\x05 \x01(\x03\x12\x13\n\x0bis_row_time\x18\x06 \x01(\x08\x12\x18\n\x10time_field_index\x18\x07 \x01(\x05\x12\x17\n\x0f\x61llowedLateness\x18\x08 \x01(\x03\x12U\n\x0fnamedProperties\x18\t \x03(\x0e\x32<.org.apache.flink.fn_execution.v1.GroupWindow.WindowProperty\x12\x16\n\x0eshift_timezone\x18\n \x01(\t\"[\n\nWindowType\x12\x19\n\x15TUMBLING_GROUP_WINDOW\x10\x00\x12\x18\n\x14SLIDING_GROUP_WINDOW\x10\x01\x12\x18\n\x14SESSION_GROUP_WINDOW\x10\x02\"c\n\x0eWindowProperty\x12\x10\n\x0cWINDOW_START\x10\x00\x12\x0e\n\nWINDOW_END\x10\x01\x12\x16\n\x12ROW_TIME_ATTRIBUTE\x10\x02\x12\x17\n\x13PROC_TIME_ATTRIBUTE\x10\x03\"\xfd\x03\n\x1dUserDefinedAggregateFunctions\x12L\n\x04udfs\x18\x01 \x03(\x0b\x32>.org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction\x12\x16\n\x0emetric_enabled\x18\x02 \x01(\x08\x12\x10\n\x08grouping\x18\x03 \x03(\x05\x12\x1e\n\x16generate_update_before\x18\x04 \x01(\x08\x12\x44\n\x08key_type\x18\x05 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x12\x1b\n\x13index_of_count_star\x18\x06 \x01(\x05\x12\x1e\n\x16state_cleaning_enabled\x18\x07 \x01(\x08\x12\x18\n\x10state_cache_size\x18\x08 \x01(\x05\x12!\n\x19map_state_read_cache_size\x18\t \x01(\x05\x12\"\n\x1amap_state_write_cache_size\x18\n \x01(\x05\x12\x1b\n\x13\x63ount_star_inserted\x18\x0b \x01(\x08\x12\x43\n\x0cgroup_window\x18\x0c \x01(\x0b\x32-.org.apache.flink.fn_execution.v1.GroupWindow\"\xec\x0f\n\x06Schema\x12>\n\x06\x66ields\x18\x01 \x03(\x0b\x32..org.apache.flink.fn_execution.v1.Schema.Field\x1a\x97\x01\n\x07MapInfo\x12\x44\n\x08key_type\x18\x01 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x12\x46\n\nvalue_type\x18\x02 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x1a\x1d\n\x08TimeInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a\"\n\rTimestampInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a,\n\x17LocalZonedTimestampInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a\'\n\x12ZonedTimestampInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a/\n\x0b\x44\x65\x63imalInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x12\r\n\x05scale\x18\x02 \x01(\x05\x1a\x1c\n\nBinaryInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\x1f\n\rVarBinaryInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\x1a\n\x08\x43harInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\x1d\n\x0bVarCharInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\xb0\x08\n\tFieldType\x12\x44\n\ttype_name\x18\x01 \x01(\x0e\x32\x31.org.apache.flink.fn_execution.v1.Schema.TypeName\x12\x10\n\x08nullable\x18\x02 \x01(\x08\x12U\n\x17\x63ollection_element_type\x18\x03 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldTypeH\x00\x12\x44\n\x08map_info\x18\x04 \x01(\x0b\x32\x30.org.apache.flink.fn_execution.v1.Schema.MapInfoH\x00\x12>\n\nrow_schema\x18\x05 \x01(\x0b\x32(.org.apache.flink.fn_execution.v1.SchemaH\x00\x12L\n\x0c\x64\x65\x63imal_info\x18\x06 \x01(\x0b\x32\x34.org.apache.flink.fn_execution.v1.Schema.DecimalInfoH\x00\x12\x46\n\ttime_info\x18\x07 \x01(\x0b\x32\x31.org.apache.flink.fn_execution.v1.Schema.TimeInfoH\x00\x12P\n\x0etimestamp_info\x18\x08 \x01(\x0b\x32\x36.org.apache.flink.fn_execution.v1.Schema.TimestampInfoH\x00\x12\x66\n\x1alocal_zoned_timestamp_info\x18\t \x01(\x0b\x32@.org.apache.flink.fn_execution.v1.Schema.LocalZonedTimestampInfoH\x00\x12[\n\x14zoned_timestamp_info\x18\n \x01(\x0b\x32;.org.apache.flink.fn_execution.v1.Schema.ZonedTimestampInfoH\x00\x12J\n\x0b\x62inary_info\x18\x0b \x01(\x0b\x32\x33.org.apache.flink.fn_execution.v1.Schema.BinaryInfoH\x00\x12Q\n\x0fvar_binary_info\x18\x0c \x01(\x0b\x32\x36.org.apache.flink.fn_execution.v1.Schema.VarBinaryInfoH\x00\x12\x46\n\tchar_info\x18\r \x01(\x0b\x32\x31.org.apache.flink.fn_execution.v1.Schema.CharInfoH\x00\x12M\n\rvar_char_info\x18\x0e \x01(\x0b\x32\x34.org.apache.flink.fn_execution.v1.Schema.VarCharInfoH\x00\x42\x0b\n\ttype_info\x1al\n\x05\x46ield\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x13\n\x0b\x64\x65scription\x18\x02 \x01(\t\x12@\n\x04type\x18\x03 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\"\xa1\x02\n\x08TypeName\x12\x07\n\x03ROW\x10\x00\x12\x0b\n\x07TINYINT\x10\x01\x12\x0c\n\x08SMALLINT\x10\x02\x12\x07\n\x03INT\x10\x03\x12\n\n\x06\x42IGINT\x10\x04\x12\x0b\n\x07\x44\x45\x43IMAL\x10\x05\x12\t\n\x05\x46LOAT\x10\x06\x12\n\n\x06\x44OUBLE\x10\x07\x12\x08\n\x04\x44\x41TE\x10\x08\x12\x08\n\x04TIME\x10\t\x12\r\n\tTIMESTAMP\x10\n\x12\x0b\n\x07\x42OOLEAN\x10\x0b\x12\n\n\x06\x42INARY\x10\x0c\x12\r\n\tVARBINARY\x10\r\x12\x08\n\x04\x43HAR\x10\x0e\x12\x0b\n\x07VARCHAR\x10\x0f\x12\x0f\n\x0b\x42\x41SIC_ARRAY\x10\x10\x12\x07\n\x03MAP\x10\x11\x12\x0c\n\x08MULTISET\x10\x12\x12\x19\n\x15LOCAL_ZONED_TIMESTAMP\x10\x13\x12\x13\n\x0fZONED_TIMESTAMP\x10\x14\"\xea\x08\n\x08TypeInfo\x12\x46\n\ttype_name\x18\x01 \x01(\x0e\x32\x33.org.apache.flink.fn_execution.v1.TypeInfo.TypeName\x12M\n\x17\x63ollection_element_type\x18\x02 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfoH\x00\x12O\n\rrow_type_info\x18\x03 \x01(\x0b\x32\x36.org.apache.flink.fn_execution.v1.TypeInfo.RowTypeInfoH\x00\x12S\n\x0ftuple_type_info\x18\x04 \x01(\x0b\x32\x38.org.apache.flink.fn_execution.v1.TypeInfo.TupleTypeInfoH\x00\x12O\n\rmap_type_info\x18\x05 \x01(\x0b\x32\x36.org.apache.flink.fn_execution.v1.TypeInfo.MapTypeInfoH\x00\x1a\x8b\x01\n\x0bMapTypeInfo\x12<\n\x08key_type\x18\x01 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfo\x12>\n\nvalue_type\x18\x02 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfo\x1a\xb8\x01\n\x0bRowTypeInfo\x12L\n\x06\x66ields\x18\x01 \x03(\x0b\x32<.org.apache.flink.fn_execution.v1.TypeInfo.RowTypeInfo.Field\x1a[\n\x05\x46ield\x12\x12\n\nfield_name\x18\x01 \x01(\t\x12>\n\nfield_type\x18\x02 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfo\x1aP\n\rTupleTypeInfo\x12?\n\x0b\x66ield_types\x18\x01 \x03(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfo\"\xa7\x02\n\x08TypeName\x12\x07\n\x03ROW\x10\x00\x12\n\n\x06STRING\x10\x01\x12\x08\n\x04\x42YTE\x10\x02\x12\x0b\n\x07\x42OOLEAN\x10\x03\x12\t\n\x05SHORT\x10\x04\x12\x07\n\x03INT\x10\x05\x12\x08\n\x04LONG\x10\x06\x12\t\n\x05\x46LOAT\x10\x07\x12\n\n\x06\x44OUBLE\x10\x08\x12\x08\n\x04\x43HAR\x10\t\x12\x0b\n\x07\x42IG_INT\x10\n\x12\x0b\n\x07\x42IG_DEC\x10\x0b\x12\x0c\n\x08SQL_DATE\x10\x0c\x12\x0c\n\x08SQL_TIME\x10\r\x12\x11\n\rSQL_TIMESTAMP\x10\x0e\x12\x0f\n\x0b\x42\x41SIC_ARRAY\x10\x0f\x12\x13\n\x0fPRIMITIVE_ARRAY\x10\x10\x12\t\n\x05TUPLE\x10\x11\x12\x08\n\x04LIST\x10\x12\x12\x07\n\x03MAP\x10\x13\x12\x11\n\rPICKLED_BYTES\x10\x14\x12\x10\n\x0cOBJECT_ARRAY\x10\x15\x42\x0b\n\ttype_info\"\xf7\x06\n\x1dUserDefinedDataStreamFunction\x12\x63\n\rfunction_type\x18\x01 \x01(\x0e\x32L.org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.FunctionType\x12g\n\x0fruntime_context\x18\x02 \x01(\x0b\x32N.org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.RuntimeContext\x12\x0f\n\x07payload\x18\x03 \x01(\x0c\x12\x16\n\x0emetric_enabled\x18\x04 \x01(\x08\x12\x41\n\rkey_type_info\x18\x05 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfo\x1a*\n\x0cJobParameter\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t\x1a\xd0\x02\n\x0eRuntimeContext\x12\x11\n\ttask_name\x18\x01 \x01(\t\x12\x1f\n\x17task_name_with_subtasks\x18\x02 \x01(\t\x12#\n\x1bnumber_of_parallel_subtasks\x18\x03 \x01(\x05\x12\'\n\x1fmax_number_of_parallel_subtasks\x18\x04 \x01(\x05\x12\x1d\n\x15index_of_this_subtask\x18\x05 \x01(\x05\x12\x16\n\x0e\x61ttempt_number\x18\x06 \x01(\x05\x12\x64\n\x0ejob_parameters\x18\x07 \x03(\x0b\x32L.org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.JobParameter\x12\x1f\n\x17in_batch_execution_mode\x18\x08 \x01(\x08\"\x9c\x01\n\x0c\x46unctionType\x12\x07\n\x03MAP\x10\x00\x12\x0c\n\x08\x46LAT_MAP\x10\x01\x12\n\n\x06\x43O_MAP\x10\x02\x12\x0f\n\x0b\x43O_FLAT_MAP\x10\x03\x12\x0b\n\x07PROCESS\x10\x04\x12\x11\n\rKEYED_PROCESS\x10\x05\x12\x14\n\x10KEYED_CO_PROCESS\x10\x06\x12\x16\n\x12TIMESTAMP_ASSIGNER\x10\x07\x12\n\n\x06WINDOW\x10\x08\"\xf1\x07\n\x13\x43oderInfoDescriptor\x12`\n\x10\x66latten_row_type\x18\x01 \x01(\x0b\x32\x44.org.apache.flink.fn_execution.v1.CoderInfoDescriptor.FlattenRowTypeH\x00\x12Q\n\x08row_type\x18\x02 \x01(\x0b\x32=.org.apache.flink.fn_execution.v1.CoderInfoDescriptor.RowTypeH\x00\x12U\n\narrow_type\x18\x03 \x01(\x0b\x32?.org.apache.flink.fn_execution.v1.CoderInfoDescriptor.ArrowTypeH\x00\x12k\n\x16over_window_arrow_type\x18\x04 \x01(\x0b\x32I.org.apache.flink.fn_execution.v1.CoderInfoDescriptor.OverWindowArrowTypeH\x00\x12Q\n\x08raw_type\x18\x05 \x01(\x0b\x32=.org.apache.flink.fn_execution.v1.CoderInfoDescriptor.RawTypeH\x00\x12H\n\x04mode\x18\x06 \x01(\x0e\x32:.org.apache.flink.fn_execution.v1.CoderInfoDescriptor.Mode\x12\"\n\x1aseparated_with_end_message\x18\x07 \x01(\x08\x1aJ\n\x0e\x46lattenRowType\x12\x38\n\x06schema\x18\x01 \x01(\x0b\x32(.org.apache.flink.fn_execution.v1.Schema\x1a\x43\n\x07RowType\x12\x38\n\x06schema\x18\x01 \x01(\x0b\x32(.org.apache.flink.fn_execution.v1.Schema\x1a\x45\n\tArrowType\x12\x38\n\x06schema\x18\x01 \x01(\x0b\x32(.org.apache.flink.fn_execution.v1.Schema\x1aO\n\x13OverWindowArrowType\x12\x38\n\x06schema\x18\x01 \x01(\x0b\x32(.org.apache.flink.fn_execution.v1.Schema\x1aH\n\x07RawType\x12=\n\ttype_info\x18\x01 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfo\" \n\x04Mode\x12\n\n\x06SINGLE\x10\x00\x12\x0c\n\x08MULTIPLE\x10\x01\x42\x0b\n\tdata_typeB-\n\x1forg.apache.flink.fnexecution.v1B\nFlinkFnApib\x06proto3') ) @@ -87,56 +87,6 @@ ) _sym_db.RegisterEnumDescriptor(_OVERWINDOW_WINDOWTYPE) -_USERDEFINEDDATASTREAMFUNCTION_FUNCTIONTYPE = _descriptor.EnumDescriptor( - name='FunctionType', - full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.FunctionType', - filename=None, - file=DESCRIPTOR, - values=[ - _descriptor.EnumValueDescriptor( - name='MAP', index=0, number=0, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='FLAT_MAP', index=1, number=1, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='CO_MAP', index=2, number=2, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='CO_FLAT_MAP', index=3, number=3, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='PROCESS', index=4, number=4, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='KEYED_PROCESS', index=5, number=5, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='KEYED_CO_PROCESS', index=6, number=6, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='TIMESTAMP_ASSIGNER', index=7, number=7, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='WINDOW', index=8, number=8, - options=None, - type=None), - ], - containing_type=None, - options=None, - serialized_start=1635, - serialized_end=1791, -) -_sym_db.RegisterEnumDescriptor(_USERDEFINEDDATASTREAMFUNCTION_FUNCTIONTYPE) - _GROUPWINDOW_WINDOWTYPE = _descriptor.EnumDescriptor( name='WindowType', full_name='org.apache.flink.fn_execution.v1.GroupWindow.WindowType', @@ -158,8 +108,8 @@ ], containing_type=None, options=None, - serialized_start=2940, - serialized_end=3031, + serialized_start=2050, + serialized_end=2141, ) _sym_db.RegisterEnumDescriptor(_GROUPWINDOW_WINDOWTYPE) @@ -188,8 +138,8 @@ ], containing_type=None, options=None, - serialized_start=3033, - serialized_end=3132, + serialized_start=2143, + serialized_end=2242, ) _sym_db.RegisterEnumDescriptor(_GROUPWINDOW_WINDOWPROPERTY) @@ -286,71 +236,11 @@ ], containing_type=None, options=None, - serialized_start=5386, - serialized_end=5675, + serialized_start=4496, + serialized_end=4785, ) _sym_db.RegisterEnumDescriptor(_SCHEMA_TYPENAME) -_CODERPARAM_DATATYPE = _descriptor.EnumDescriptor( - name='DataType', - full_name='org.apache.flink.fn_execution.v1.CoderParam.DataType', - filename=None, - file=DESCRIPTOR, - values=[ - _descriptor.EnumValueDescriptor( - name='FLATTEN_ROW', index=0, number=0, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='ROW', index=1, number=1, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='RAW', index=2, number=2, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='ARROW', index=3, number=3, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='OVER_WINDOW_ARROW', index=4, number=4, - options=None, - type=None), - ], - containing_type=None, - options=None, - serialized_start=5969, - serialized_end=6048, -) -_sym_db.RegisterEnumDescriptor(_CODERPARAM_DATATYPE) - -_CODERPARAM_OUTPUTMODE = _descriptor.EnumDescriptor( - name='OutputMode', - full_name='org.apache.flink.fn_execution.v1.CoderParam.OutputMode', - filename=None, - file=DESCRIPTOR, - values=[ - _descriptor.EnumValueDescriptor( - name='SINGLE', index=0, number=0, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='MULTIPLE', index=1, number=1, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='MULTIPLE_WITH_END', index=2, number=2, - options=None, - type=None), - ], - containing_type=None, - options=None, - serialized_start=6050, - serialized_end=6111, -) -_sym_db.RegisterEnumDescriptor(_CODERPARAM_OUTPUTMODE) - _TYPEINFO_TYPENAME = _descriptor.EnumDescriptor( name='TypeName', full_name='org.apache.flink.fn_execution.v1.TypeInfo.TypeName', @@ -448,11 +338,83 @@ ], containing_type=None, options=None, - serialized_start=6950, - serialized_end=7245, + serialized_start=5610, + serialized_end=5905, ) _sym_db.RegisterEnumDescriptor(_TYPEINFO_TYPENAME) +_USERDEFINEDDATASTREAMFUNCTION_FUNCTIONTYPE = _descriptor.EnumDescriptor( + name='FunctionType', + full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.FunctionType', + filename=None, + file=DESCRIPTOR, + values=[ + _descriptor.EnumValueDescriptor( + name='MAP', index=0, number=0, + options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='FLAT_MAP', index=1, number=1, + options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='CO_MAP', index=2, number=2, + options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='CO_FLAT_MAP', index=3, number=3, + options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='PROCESS', index=4, number=4, + options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='KEYED_PROCESS', index=5, number=5, + options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='KEYED_CO_PROCESS', index=6, number=6, + options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='TIMESTAMP_ASSIGNER', index=7, number=7, + options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='WINDOW', index=8, number=8, + options=None, + type=None), + ], + containing_type=None, + options=None, + serialized_start=6652, + serialized_end=6808, +) +_sym_db.RegisterEnumDescriptor(_USERDEFINEDDATASTREAMFUNCTION_FUNCTIONTYPE) + +_CODERINFODESCRIPTOR_MODE = _descriptor.EnumDescriptor( + name='Mode', + full_name='org.apache.flink.fn_execution.v1.CoderInfoDescriptor.Mode', + filename=None, + file=DESCRIPTOR, + values=[ + _descriptor.EnumValueDescriptor( + name='SINGLE', index=0, number=0, + options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='MULTIPLE', index=1, number=1, + options=None, + type=None), + ], + containing_type=None, + options=None, + serialized_start=7775, + serialized_end=7807, +) +_sym_db.RegisterEnumDescriptor(_CODERINFODESCRIPTOR_MODE) + _INPUT = _descriptor.Descriptor( name='Input', @@ -652,24 +614,17 @@ ) -_USERDEFINEDDATASTREAMFUNCTION_JOBPARAMETER = _descriptor.Descriptor( - name='JobParameter', - full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.JobParameter', +_USERDEFINEDAGGREGATEFUNCTION_DATAVIEWSPEC_LISTVIEW = _descriptor.Descriptor( + name='ListView', + full_name='org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec.ListView', filename=None, file=DESCRIPTOR, containing_type=None, fields=[ _descriptor.FieldDescriptor( - name='key', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.JobParameter.key', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), - _descriptor.FieldDescriptor( - name='value', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.JobParameter.value', index=1, - number=2, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), + name='element_type', full_name='org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec.ListView.element_type', index=0, + number=1, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, options=None, file=DESCRIPTOR), @@ -685,77 +640,86 @@ extension_ranges=[], oneofs=[ ], - serialized_start=1251, - serialized_end=1293, + serialized_start=1432, + serialized_end=1516, ) -_USERDEFINEDDATASTREAMFUNCTION_RUNTIMECONTEXT = _descriptor.Descriptor( - name='RuntimeContext', - full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.RuntimeContext', +_USERDEFINEDAGGREGATEFUNCTION_DATAVIEWSPEC_MAPVIEW = _descriptor.Descriptor( + name='MapView', + full_name='org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec.MapView', filename=None, file=DESCRIPTOR, containing_type=None, fields=[ _descriptor.FieldDescriptor( - name='task_name', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.RuntimeContext.task_name', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), - _descriptor.FieldDescriptor( - name='task_name_with_subtasks', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.RuntimeContext.task_name_with_subtasks', index=1, - number=2, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), - _descriptor.FieldDescriptor( - name='number_of_parallel_subtasks', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.RuntimeContext.number_of_parallel_subtasks', index=2, - number=3, type=5, cpp_type=1, label=1, - has_default_value=False, default_value=0, + name='key_type', full_name='org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec.MapView.key_type', index=0, + number=1, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( - name='max_number_of_parallel_subtasks', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.RuntimeContext.max_number_of_parallel_subtasks', index=3, - number=4, type=5, cpp_type=1, label=1, - has_default_value=False, default_value=0, + name='value_type', full_name='org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec.MapView.value_type', index=1, + number=2, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=1519, + serialized_end=1670, +) + +_USERDEFINEDAGGREGATEFUNCTION_DATAVIEWSPEC = _descriptor.Descriptor( + name='DataViewSpec', + full_name='org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ _descriptor.FieldDescriptor( - name='index_of_this_subtask', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.RuntimeContext.index_of_this_subtask', index=4, - number=5, type=5, cpp_type=1, label=1, - has_default_value=False, default_value=0, + name='name', full_name='org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec.name', index=0, + number=1, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( - name='attempt_number', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.RuntimeContext.attempt_number', index=5, - number=6, type=5, cpp_type=1, label=1, + name='field_index', full_name='org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec.field_index', index=1, + number=2, type=5, cpp_type=1, label=1, has_default_value=False, default_value=0, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( - name='job_parameters', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.RuntimeContext.job_parameters', index=6, - number=7, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], + name='list_view', full_name='org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec.list_view', index=2, + number=3, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( - name='in_batch_execution_mode', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.RuntimeContext.in_batch_execution_mode', index=7, - number=8, type=8, cpp_type=7, label=1, - has_default_value=False, default_value=False, + name='map_view', full_name='org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec.map_view', index=3, + number=4, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, options=None, file=DESCRIPTOR), ], extensions=[ ], - nested_types=[], + nested_types=[_USERDEFINEDAGGREGATEFUNCTION_DATAVIEWSPEC_LISTVIEW, _USERDEFINEDAGGREGATEFUNCTION_DATAVIEWSPEC_MAPVIEW, ], enum_types=[ ], options=None, @@ -763,190 +727,12 @@ syntax='proto3', extension_ranges=[], oneofs=[ + _descriptor.OneofDescriptor( + name='data_view', full_name='org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec.data_view', + index=0, containing_type=None, fields=[]), ], - serialized_start=1296, - serialized_end=1632, -) - -_USERDEFINEDDATASTREAMFUNCTION = _descriptor.Descriptor( - name='UserDefinedDataStreamFunction', - full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='function_type', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.function_type', index=0, - number=1, type=14, cpp_type=8, label=1, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), - _descriptor.FieldDescriptor( - name='runtime_context', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.runtime_context', index=1, - number=2, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), - _descriptor.FieldDescriptor( - name='payload', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.payload', index=2, - number=3, type=12, cpp_type=9, label=1, - has_default_value=False, default_value=_b(""), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), - _descriptor.FieldDescriptor( - name='metric_enabled', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.metric_enabled', index=3, - number=4, type=8, cpp_type=7, label=1, - has_default_value=False, default_value=False, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), - _descriptor.FieldDescriptor( - name='key_type_info', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.key_type_info', index=4, - number=5, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), - ], - extensions=[ - ], - nested_types=[_USERDEFINEDDATASTREAMFUNCTION_JOBPARAMETER, _USERDEFINEDDATASTREAMFUNCTION_RUNTIMECONTEXT, ], - enum_types=[ - _USERDEFINEDDATASTREAMFUNCTION_FUNCTIONTYPE, - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=904, - serialized_end=1791, -) - - -_USERDEFINEDAGGREGATEFUNCTION_DATAVIEWSPEC_LISTVIEW = _descriptor.Descriptor( - name='ListView', - full_name='org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec.ListView', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='element_type', full_name='org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec.ListView.element_type', index=0, - number=1, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=2322, - serialized_end=2406, -) - -_USERDEFINEDAGGREGATEFUNCTION_DATAVIEWSPEC_MAPVIEW = _descriptor.Descriptor( - name='MapView', - full_name='org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec.MapView', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='key_type', full_name='org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec.MapView.key_type', index=0, - number=1, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), - _descriptor.FieldDescriptor( - name='value_type', full_name='org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec.MapView.value_type', index=1, - number=2, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=2409, - serialized_end=2560, -) - -_USERDEFINEDAGGREGATEFUNCTION_DATAVIEWSPEC = _descriptor.Descriptor( - name='DataViewSpec', - full_name='org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='name', full_name='org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec.name', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), - _descriptor.FieldDescriptor( - name='field_index', full_name='org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec.field_index', index=1, - number=2, type=5, cpp_type=1, label=1, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), - _descriptor.FieldDescriptor( - name='list_view', full_name='org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec.list_view', index=2, - number=3, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), - _descriptor.FieldDescriptor( - name='map_view', full_name='org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec.map_view', index=3, - number=4, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), - ], - extensions=[ - ], - nested_types=[_USERDEFINEDAGGREGATEFUNCTION_DATAVIEWSPEC_LISTVIEW, _USERDEFINEDAGGREGATEFUNCTION_DATAVIEWSPEC_MAPVIEW, ], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - _descriptor.OneofDescriptor( - name='data_view', full_name='org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec.data_view', - index=0, containing_type=None, fields=[]), - ], - serialized_start=2059, - serialized_end=2573, + serialized_start=1169, + serialized_end=1683, ) _USERDEFINEDAGGREGATEFUNCTION = _descriptor.Descriptor( @@ -1010,8 +796,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=1794, - serialized_end=2573, + serialized_start=904, + serialized_end=1683, ) @@ -1106,8 +892,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=2576, - serialized_end=3132, + serialized_start=1686, + serialized_end=2242, ) @@ -1214,8 +1000,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=3135, - serialized_end=3644, + serialized_start=2245, + serialized_end=2754, ) @@ -1252,8 +1038,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=3722, - serialized_end=3873, + serialized_start=2832, + serialized_end=2983, ) _SCHEMA_TIMEINFO = _descriptor.Descriptor( @@ -1282,8 +1068,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=3875, - serialized_end=3904, + serialized_start=2985, + serialized_end=3014, ) _SCHEMA_TIMESTAMPINFO = _descriptor.Descriptor( @@ -1312,8 +1098,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=3906, - serialized_end=3940, + serialized_start=3016, + serialized_end=3050, ) _SCHEMA_LOCALZONEDTIMESTAMPINFO = _descriptor.Descriptor( @@ -1342,8 +1128,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=3942, - serialized_end=3986, + serialized_start=3052, + serialized_end=3096, ) _SCHEMA_ZONEDTIMESTAMPINFO = _descriptor.Descriptor( @@ -1372,8 +1158,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=3988, - serialized_end=4027, + serialized_start=3098, + serialized_end=3137, ) _SCHEMA_DECIMALINFO = _descriptor.Descriptor( @@ -1409,8 +1195,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=4029, - serialized_end=4076, + serialized_start=3139, + serialized_end=3186, ) _SCHEMA_BINARYINFO = _descriptor.Descriptor( @@ -1439,8 +1225,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=4078, - serialized_end=4106, + serialized_start=3188, + serialized_end=3216, ) _SCHEMA_VARBINARYINFO = _descriptor.Descriptor( @@ -1469,8 +1255,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=4108, - serialized_end=4139, + serialized_start=3218, + serialized_end=3249, ) _SCHEMA_CHARINFO = _descriptor.Descriptor( @@ -1499,8 +1285,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=4141, - serialized_end=4167, + serialized_start=3251, + serialized_end=3277, ) _SCHEMA_VARCHARINFO = _descriptor.Descriptor( @@ -1529,8 +1315,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=4169, - serialized_end=4198, + serialized_start=3279, + serialized_end=3308, ) _SCHEMA_FIELDTYPE = _descriptor.Descriptor( @@ -1625,16 +1411,408 @@ is_extension=False, extension_scope=None, options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( - name='char_info', full_name='org.apache.flink.fn_execution.v1.Schema.FieldType.char_info', index=12, - number=13, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, + name='char_info', full_name='org.apache.flink.fn_execution.v1.Schema.FieldType.char_info', index=12, + number=13, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='var_char_info', full_name='org.apache.flink.fn_execution.v1.Schema.FieldType.var_char_info', index=13, + number=14, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + _descriptor.OneofDescriptor( + name='type_info', full_name='org.apache.flink.fn_execution.v1.Schema.FieldType.type_info', + index=0, containing_type=None, fields=[]), + ], + serialized_start=3311, + serialized_end=4383, +) + +_SCHEMA_FIELD = _descriptor.Descriptor( + name='Field', + full_name='org.apache.flink.fn_execution.v1.Schema.Field', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='name', full_name='org.apache.flink.fn_execution.v1.Schema.Field.name', index=0, + number=1, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='description', full_name='org.apache.flink.fn_execution.v1.Schema.Field.description', index=1, + number=2, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='type', full_name='org.apache.flink.fn_execution.v1.Schema.Field.type', index=2, + number=3, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=4385, + serialized_end=4493, +) + +_SCHEMA = _descriptor.Descriptor( + name='Schema', + full_name='org.apache.flink.fn_execution.v1.Schema', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='fields', full_name='org.apache.flink.fn_execution.v1.Schema.fields', index=0, + number=1, type=11, cpp_type=10, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[_SCHEMA_MAPINFO, _SCHEMA_TIMEINFO, _SCHEMA_TIMESTAMPINFO, _SCHEMA_LOCALZONEDTIMESTAMPINFO, _SCHEMA_ZONEDTIMESTAMPINFO, _SCHEMA_DECIMALINFO, _SCHEMA_BINARYINFO, _SCHEMA_VARBINARYINFO, _SCHEMA_CHARINFO, _SCHEMA_VARCHARINFO, _SCHEMA_FIELDTYPE, _SCHEMA_FIELD, ], + enum_types=[ + _SCHEMA_TYPENAME, + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=2757, + serialized_end=4785, +) + + +_TYPEINFO_MAPTYPEINFO = _descriptor.Descriptor( + name='MapTypeInfo', + full_name='org.apache.flink.fn_execution.v1.TypeInfo.MapTypeInfo', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='key_type', full_name='org.apache.flink.fn_execution.v1.TypeInfo.MapTypeInfo.key_type', index=0, + number=1, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='value_type', full_name='org.apache.flink.fn_execution.v1.TypeInfo.MapTypeInfo.value_type', index=1, + number=2, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=5199, + serialized_end=5338, +) + +_TYPEINFO_ROWTYPEINFO_FIELD = _descriptor.Descriptor( + name='Field', + full_name='org.apache.flink.fn_execution.v1.TypeInfo.RowTypeInfo.Field', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='field_name', full_name='org.apache.flink.fn_execution.v1.TypeInfo.RowTypeInfo.Field.field_name', index=0, + number=1, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='field_type', full_name='org.apache.flink.fn_execution.v1.TypeInfo.RowTypeInfo.Field.field_type', index=1, + number=2, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=5434, + serialized_end=5525, +) + +_TYPEINFO_ROWTYPEINFO = _descriptor.Descriptor( + name='RowTypeInfo', + full_name='org.apache.flink.fn_execution.v1.TypeInfo.RowTypeInfo', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='fields', full_name='org.apache.flink.fn_execution.v1.TypeInfo.RowTypeInfo.fields', index=0, + number=1, type=11, cpp_type=10, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[_TYPEINFO_ROWTYPEINFO_FIELD, ], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=5341, + serialized_end=5525, +) + +_TYPEINFO_TUPLETYPEINFO = _descriptor.Descriptor( + name='TupleTypeInfo', + full_name='org.apache.flink.fn_execution.v1.TypeInfo.TupleTypeInfo', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='field_types', full_name='org.apache.flink.fn_execution.v1.TypeInfo.TupleTypeInfo.field_types', index=0, + number=1, type=11, cpp_type=10, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=5527, + serialized_end=5607, +) + +_TYPEINFO = _descriptor.Descriptor( + name='TypeInfo', + full_name='org.apache.flink.fn_execution.v1.TypeInfo', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='type_name', full_name='org.apache.flink.fn_execution.v1.TypeInfo.type_name', index=0, + number=1, type=14, cpp_type=8, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='collection_element_type', full_name='org.apache.flink.fn_execution.v1.TypeInfo.collection_element_type', index=1, + number=2, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='row_type_info', full_name='org.apache.flink.fn_execution.v1.TypeInfo.row_type_info', index=2, + number=3, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='tuple_type_info', full_name='org.apache.flink.fn_execution.v1.TypeInfo.tuple_type_info', index=3, + number=4, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='map_type_info', full_name='org.apache.flink.fn_execution.v1.TypeInfo.map_type_info', index=4, + number=5, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[_TYPEINFO_MAPTYPEINFO, _TYPEINFO_ROWTYPEINFO, _TYPEINFO_TUPLETYPEINFO, ], + enum_types=[ + _TYPEINFO_TYPENAME, + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + _descriptor.OneofDescriptor( + name='type_info', full_name='org.apache.flink.fn_execution.v1.TypeInfo.type_info', + index=0, containing_type=None, fields=[]), + ], + serialized_start=4788, + serialized_end=5918, +) + + +_USERDEFINEDDATASTREAMFUNCTION_JOBPARAMETER = _descriptor.Descriptor( + name='JobParameter', + full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.JobParameter', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='key', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.JobParameter.key', index=0, + number=1, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='value', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.JobParameter.value', index=1, + number=2, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=6268, + serialized_end=6310, +) + +_USERDEFINEDDATASTREAMFUNCTION_RUNTIMECONTEXT = _descriptor.Descriptor( + name='RuntimeContext', + full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.RuntimeContext', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='task_name', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.RuntimeContext.task_name', index=0, + number=1, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='task_name_with_subtasks', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.RuntimeContext.task_name_with_subtasks', index=1, + number=2, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='number_of_parallel_subtasks', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.RuntimeContext.number_of_parallel_subtasks', index=2, + number=3, type=5, cpp_type=1, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='max_number_of_parallel_subtasks', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.RuntimeContext.max_number_of_parallel_subtasks', index=3, + number=4, type=5, cpp_type=1, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='index_of_this_subtask', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.RuntimeContext.index_of_this_subtask', index=4, + number=5, type=5, cpp_type=1, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='attempt_number', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.RuntimeContext.attempt_number', index=5, + number=6, type=5, cpp_type=1, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='job_parameters', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.RuntimeContext.job_parameters', index=6, + number=7, type=11, cpp_type=10, label=3, + has_default_value=False, default_value=[], message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( - name='var_char_info', full_name='org.apache.flink.fn_execution.v1.Schema.FieldType.var_char_info', index=13, - number=14, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, + name='in_batch_execution_mode', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.RuntimeContext.in_batch_execution_mode', index=7, + number=8, type=8, cpp_type=7, label=1, + has_default_value=False, default_value=False, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, options=None, file=DESCRIPTOR), @@ -1649,78 +1827,59 @@ syntax='proto3', extension_ranges=[], oneofs=[ - _descriptor.OneofDescriptor( - name='type_info', full_name='org.apache.flink.fn_execution.v1.Schema.FieldType.type_info', - index=0, containing_type=None, fields=[]), ], - serialized_start=4201, - serialized_end=5273, + serialized_start=6313, + serialized_end=6649, ) -_SCHEMA_FIELD = _descriptor.Descriptor( - name='Field', - full_name='org.apache.flink.fn_execution.v1.Schema.Field', +_USERDEFINEDDATASTREAMFUNCTION = _descriptor.Descriptor( + name='UserDefinedDataStreamFunction', + full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction', filename=None, file=DESCRIPTOR, containing_type=None, fields=[ _descriptor.FieldDescriptor( - name='name', full_name='org.apache.flink.fn_execution.v1.Schema.Field.name', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), + name='function_type', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.function_type', index=0, + number=1, type=14, cpp_type=8, label=1, + has_default_value=False, default_value=0, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( - name='description', full_name='org.apache.flink.fn_execution.v1.Schema.Field.description', index=1, - number=2, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), + name='runtime_context', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.runtime_context', index=1, + number=2, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( - name='type', full_name='org.apache.flink.fn_execution.v1.Schema.Field.type', index=2, - number=3, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, + name='payload', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.payload', index=2, + number=3, type=12, cpp_type=9, label=1, + has_default_value=False, default_value=_b(""), message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, options=None, file=DESCRIPTOR), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=5275, - serialized_end=5383, -) - -_SCHEMA = _descriptor.Descriptor( - name='Schema', - full_name='org.apache.flink.fn_execution.v1.Schema', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ _descriptor.FieldDescriptor( - name='fields', full_name='org.apache.flink.fn_execution.v1.Schema.fields', index=0, - number=1, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], + name='metric_enabled', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.metric_enabled', index=3, + number=4, type=8, cpp_type=7, label=1, + has_default_value=False, default_value=False, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='key_type_info', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.key_type_info', index=4, + number=5, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, options=None, file=DESCRIPTOR), ], extensions=[ ], - nested_types=[_SCHEMA_MAPINFO, _SCHEMA_TIMEINFO, _SCHEMA_TIMESTAMPINFO, _SCHEMA_LOCALZONEDTIMESTAMPINFO, _SCHEMA_ZONEDTIMESTAMPINFO, _SCHEMA_DECIMALINFO, _SCHEMA_BINARYINFO, _SCHEMA_VARBINARYINFO, _SCHEMA_CHARINFO, _SCHEMA_VARCHARINFO, _SCHEMA_FIELDTYPE, _SCHEMA_FIELD, ], + nested_types=[_USERDEFINEDDATASTREAMFUNCTION_JOBPARAMETER, _USERDEFINEDDATASTREAMFUNCTION_RUNTIMECONTEXT, ], enum_types=[ - _SCHEMA_TYPENAME, + _USERDEFINEDDATASTREAMFUNCTION_FUNCTIONTYPE, ], options=None, is_extendable=False, @@ -1728,89 +1887,55 @@ extension_ranges=[], oneofs=[ ], - serialized_start=3647, - serialized_end=5675, + serialized_start=5921, + serialized_end=6808, ) -_CODERPARAM = _descriptor.Descriptor( - name='CoderParam', - full_name='org.apache.flink.fn_execution.v1.CoderParam', +_CODERINFODESCRIPTOR_FLATTENROWTYPE = _descriptor.Descriptor( + name='FlattenRowType', + full_name='org.apache.flink.fn_execution.v1.CoderInfoDescriptor.FlattenRowType', filename=None, file=DESCRIPTOR, containing_type=None, fields=[ _descriptor.FieldDescriptor( - name='schema', full_name='org.apache.flink.fn_execution.v1.CoderParam.schema', index=0, + name='schema', full_name='org.apache.flink.fn_execution.v1.CoderInfoDescriptor.FlattenRowType.schema', index=0, number=1, type=11, cpp_type=10, label=1, has_default_value=False, default_value=None, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, options=None, file=DESCRIPTOR), - _descriptor.FieldDescriptor( - name='type_info', full_name='org.apache.flink.fn_execution.v1.CoderParam.type_info', index=1, - number=2, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), - _descriptor.FieldDescriptor( - name='data_type', full_name='org.apache.flink.fn_execution.v1.CoderParam.data_type', index=2, - number=3, type=14, cpp_type=8, label=1, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), - _descriptor.FieldDescriptor( - name='output_mode', full_name='org.apache.flink.fn_execution.v1.CoderParam.output_mode', index=3, - number=4, type=14, cpp_type=8, label=1, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), ], extensions=[ ], nested_types=[], enum_types=[ - _CODERPARAM_DATATYPE, - _CODERPARAM_OUTPUTMODE, ], options=None, is_extendable=False, syntax='proto3', extension_ranges=[], oneofs=[ - _descriptor.OneofDescriptor( - name='field_type', full_name='org.apache.flink.fn_execution.v1.CoderParam.field_type', - index=0, containing_type=None, fields=[]), ], - serialized_start=5678, - serialized_end=6125, + serialized_start=7404, + serialized_end=7478, ) - -_TYPEINFO_MAPTYPEINFO = _descriptor.Descriptor( - name='MapTypeInfo', - full_name='org.apache.flink.fn_execution.v1.TypeInfo.MapTypeInfo', +_CODERINFODESCRIPTOR_ROWTYPE = _descriptor.Descriptor( + name='RowType', + full_name='org.apache.flink.fn_execution.v1.CoderInfoDescriptor.RowType', filename=None, file=DESCRIPTOR, containing_type=None, fields=[ _descriptor.FieldDescriptor( - name='key_type', full_name='org.apache.flink.fn_execution.v1.TypeInfo.MapTypeInfo.key_type', index=0, + name='schema', full_name='org.apache.flink.fn_execution.v1.CoderInfoDescriptor.RowType.schema', index=0, number=1, type=11, cpp_type=10, label=1, has_default_value=False, default_value=None, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, options=None, file=DESCRIPTOR), - _descriptor.FieldDescriptor( - name='value_type', full_name='org.apache.flink.fn_execution.v1.TypeInfo.MapTypeInfo.value_type', index=1, - number=2, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), ], extensions=[ ], @@ -1823,27 +1948,20 @@ extension_ranges=[], oneofs=[ ], - serialized_start=6539, - serialized_end=6678, + serialized_start=7480, + serialized_end=7547, ) -_TYPEINFO_ROWTYPEINFO_FIELD = _descriptor.Descriptor( - name='Field', - full_name='org.apache.flink.fn_execution.v1.TypeInfo.RowTypeInfo.Field', +_CODERINFODESCRIPTOR_ARROWTYPE = _descriptor.Descriptor( + name='ArrowType', + full_name='org.apache.flink.fn_execution.v1.CoderInfoDescriptor.ArrowType', filename=None, file=DESCRIPTOR, containing_type=None, fields=[ _descriptor.FieldDescriptor( - name='field_name', full_name='org.apache.flink.fn_execution.v1.TypeInfo.RowTypeInfo.Field.field_name', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), - _descriptor.FieldDescriptor( - name='field_type', full_name='org.apache.flink.fn_execution.v1.TypeInfo.RowTypeInfo.Field.field_type', index=1, - number=2, type=11, cpp_type=10, label=1, + name='schema', full_name='org.apache.flink.fn_execution.v1.CoderInfoDescriptor.ArrowType.schema', index=0, + number=1, type=11, cpp_type=10, label=1, has_default_value=False, default_value=None, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, @@ -1860,28 +1978,28 @@ extension_ranges=[], oneofs=[ ], - serialized_start=6774, - serialized_end=6865, + serialized_start=7549, + serialized_end=7618, ) -_TYPEINFO_ROWTYPEINFO = _descriptor.Descriptor( - name='RowTypeInfo', - full_name='org.apache.flink.fn_execution.v1.TypeInfo.RowTypeInfo', +_CODERINFODESCRIPTOR_OVERWINDOWARROWTYPE = _descriptor.Descriptor( + name='OverWindowArrowType', + full_name='org.apache.flink.fn_execution.v1.CoderInfoDescriptor.OverWindowArrowType', filename=None, file=DESCRIPTOR, containing_type=None, fields=[ _descriptor.FieldDescriptor( - name='fields', full_name='org.apache.flink.fn_execution.v1.TypeInfo.RowTypeInfo.fields', index=0, - number=1, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], + name='schema', full_name='org.apache.flink.fn_execution.v1.CoderInfoDescriptor.OverWindowArrowType.schema', index=0, + number=1, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, options=None, file=DESCRIPTOR), ], extensions=[ ], - nested_types=[_TYPEINFO_ROWTYPEINFO_FIELD, ], + nested_types=[], enum_types=[ ], options=None, @@ -1890,21 +2008,21 @@ extension_ranges=[], oneofs=[ ], - serialized_start=6681, - serialized_end=6865, + serialized_start=7620, + serialized_end=7699, ) -_TYPEINFO_TUPLETYPEINFO = _descriptor.Descriptor( - name='TupleTypeInfo', - full_name='org.apache.flink.fn_execution.v1.TypeInfo.TupleTypeInfo', +_CODERINFODESCRIPTOR_RAWTYPE = _descriptor.Descriptor( + name='RawType', + full_name='org.apache.flink.fn_execution.v1.CoderInfoDescriptor.RawType', filename=None, file=DESCRIPTOR, containing_type=None, fields=[ _descriptor.FieldDescriptor( - name='field_types', full_name='org.apache.flink.fn_execution.v1.TypeInfo.TupleTypeInfo.field_types', index=0, - number=1, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], + name='type_info', full_name='org.apache.flink.fn_execution.v1.CoderInfoDescriptor.RawType.type_info', index=0, + number=1, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, options=None, file=DESCRIPTOR), @@ -1920,58 +2038,72 @@ extension_ranges=[], oneofs=[ ], - serialized_start=6867, - serialized_end=6947, + serialized_start=7701, + serialized_end=7773, ) -_TYPEINFO = _descriptor.Descriptor( - name='TypeInfo', - full_name='org.apache.flink.fn_execution.v1.TypeInfo', +_CODERINFODESCRIPTOR = _descriptor.Descriptor( + name='CoderInfoDescriptor', + full_name='org.apache.flink.fn_execution.v1.CoderInfoDescriptor', filename=None, file=DESCRIPTOR, containing_type=None, fields=[ _descriptor.FieldDescriptor( - name='type_name', full_name='org.apache.flink.fn_execution.v1.TypeInfo.type_name', index=0, - number=1, type=14, cpp_type=8, label=1, - has_default_value=False, default_value=0, + name='flatten_row_type', full_name='org.apache.flink.fn_execution.v1.CoderInfoDescriptor.flatten_row_type', index=0, + number=1, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( - name='collection_element_type', full_name='org.apache.flink.fn_execution.v1.TypeInfo.collection_element_type', index=1, + name='row_type', full_name='org.apache.flink.fn_execution.v1.CoderInfoDescriptor.row_type', index=1, number=2, type=11, cpp_type=10, label=1, has_default_value=False, default_value=None, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( - name='row_type_info', full_name='org.apache.flink.fn_execution.v1.TypeInfo.row_type_info', index=2, + name='arrow_type', full_name='org.apache.flink.fn_execution.v1.CoderInfoDescriptor.arrow_type', index=2, number=3, type=11, cpp_type=10, label=1, has_default_value=False, default_value=None, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( - name='tuple_type_info', full_name='org.apache.flink.fn_execution.v1.TypeInfo.tuple_type_info', index=3, + name='over_window_arrow_type', full_name='org.apache.flink.fn_execution.v1.CoderInfoDescriptor.over_window_arrow_type', index=3, number=4, type=11, cpp_type=10, label=1, has_default_value=False, default_value=None, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( - name='map_type_info', full_name='org.apache.flink.fn_execution.v1.TypeInfo.map_type_info', index=4, + name='raw_type', full_name='org.apache.flink.fn_execution.v1.CoderInfoDescriptor.raw_type', index=4, number=5, type=11, cpp_type=10, label=1, has_default_value=False, default_value=None, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='mode', full_name='org.apache.flink.fn_execution.v1.CoderInfoDescriptor.mode', index=5, + number=6, type=14, cpp_type=8, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='separated_with_end_message', full_name='org.apache.flink.fn_execution.v1.CoderInfoDescriptor.separated_with_end_message', index=6, + number=7, type=8, cpp_type=7, label=1, + has_default_value=False, default_value=False, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), ], extensions=[ ], - nested_types=[_TYPEINFO_MAPTYPEINFO, _TYPEINFO_ROWTYPEINFO, _TYPEINFO_TUPLETYPEINFO, ], + nested_types=[_CODERINFODESCRIPTOR_FLATTENROWTYPE, _CODERINFODESCRIPTOR_ROWTYPE, _CODERINFODESCRIPTOR_ARROWTYPE, _CODERINFODESCRIPTOR_OVERWINDOWARROWTYPE, _CODERINFODESCRIPTOR_RAWTYPE, ], enum_types=[ - _TYPEINFO_TYPENAME, + _CODERINFODESCRIPTOR_MODE, ], options=None, is_extendable=False, @@ -1979,11 +2111,11 @@ extension_ranges=[], oneofs=[ _descriptor.OneofDescriptor( - name='type_info', full_name='org.apache.flink.fn_execution.v1.TypeInfo.type_info', + name='data_type', full_name='org.apache.flink.fn_execution.v1.CoderInfoDescriptor.data_type', index=0, containing_type=None, fields=[]), ], - serialized_start=6128, - serialized_end=7258, + serialized_start=6811, + serialized_end=7820, ) _INPUT.fields_by_name['udf'].message_type = _USERDEFINEDFUNCTION @@ -2001,13 +2133,6 @@ _USERDEFINEDFUNCTIONS.fields_by_name['windows'].message_type = _OVERWINDOW _OVERWINDOW.fields_by_name['window_type'].enum_type = _OVERWINDOW_WINDOWTYPE _OVERWINDOW_WINDOWTYPE.containing_type = _OVERWINDOW -_USERDEFINEDDATASTREAMFUNCTION_JOBPARAMETER.containing_type = _USERDEFINEDDATASTREAMFUNCTION -_USERDEFINEDDATASTREAMFUNCTION_RUNTIMECONTEXT.fields_by_name['job_parameters'].message_type = _USERDEFINEDDATASTREAMFUNCTION_JOBPARAMETER -_USERDEFINEDDATASTREAMFUNCTION_RUNTIMECONTEXT.containing_type = _USERDEFINEDDATASTREAMFUNCTION -_USERDEFINEDDATASTREAMFUNCTION.fields_by_name['function_type'].enum_type = _USERDEFINEDDATASTREAMFUNCTION_FUNCTIONTYPE -_USERDEFINEDDATASTREAMFUNCTION.fields_by_name['runtime_context'].message_type = _USERDEFINEDDATASTREAMFUNCTION_RUNTIMECONTEXT -_USERDEFINEDDATASTREAMFUNCTION.fields_by_name['key_type_info'].message_type = _TYPEINFO -_USERDEFINEDDATASTREAMFUNCTION_FUNCTIONTYPE.containing_type = _USERDEFINEDDATASTREAMFUNCTION _USERDEFINEDAGGREGATEFUNCTION_DATAVIEWSPEC_LISTVIEW.fields_by_name['element_type'].message_type = _SCHEMA_FIELDTYPE _USERDEFINEDAGGREGATEFUNCTION_DATAVIEWSPEC_LISTVIEW.containing_type = _USERDEFINEDAGGREGATEFUNCTION_DATAVIEWSPEC _USERDEFINEDAGGREGATEFUNCTION_DATAVIEWSPEC_MAPVIEW.fields_by_name['key_type'].message_type = _SCHEMA_FIELDTYPE @@ -2097,18 +2222,6 @@ _SCHEMA_FIELD.containing_type = _SCHEMA _SCHEMA.fields_by_name['fields'].message_type = _SCHEMA_FIELD _SCHEMA_TYPENAME.containing_type = _SCHEMA -_CODERPARAM.fields_by_name['schema'].message_type = _SCHEMA -_CODERPARAM.fields_by_name['type_info'].message_type = _TYPEINFO -_CODERPARAM.fields_by_name['data_type'].enum_type = _CODERPARAM_DATATYPE -_CODERPARAM.fields_by_name['output_mode'].enum_type = _CODERPARAM_OUTPUTMODE -_CODERPARAM_DATATYPE.containing_type = _CODERPARAM -_CODERPARAM_OUTPUTMODE.containing_type = _CODERPARAM -_CODERPARAM.oneofs_by_name['field_type'].fields.append( - _CODERPARAM.fields_by_name['schema']) -_CODERPARAM.fields_by_name['schema'].containing_oneof = _CODERPARAM.oneofs_by_name['field_type'] -_CODERPARAM.oneofs_by_name['field_type'].fields.append( - _CODERPARAM.fields_by_name['type_info']) -_CODERPARAM.fields_by_name['type_info'].containing_oneof = _CODERPARAM.oneofs_by_name['field_type'] _TYPEINFO_MAPTYPEINFO.fields_by_name['key_type'].message_type = _TYPEINFO _TYPEINFO_MAPTYPEINFO.fields_by_name['value_type'].message_type = _TYPEINFO _TYPEINFO_MAPTYPEINFO.containing_type = _TYPEINFO @@ -2136,17 +2249,56 @@ _TYPEINFO.oneofs_by_name['type_info'].fields.append( _TYPEINFO.fields_by_name['map_type_info']) _TYPEINFO.fields_by_name['map_type_info'].containing_oneof = _TYPEINFO.oneofs_by_name['type_info'] +_USERDEFINEDDATASTREAMFUNCTION_JOBPARAMETER.containing_type = _USERDEFINEDDATASTREAMFUNCTION +_USERDEFINEDDATASTREAMFUNCTION_RUNTIMECONTEXT.fields_by_name['job_parameters'].message_type = _USERDEFINEDDATASTREAMFUNCTION_JOBPARAMETER +_USERDEFINEDDATASTREAMFUNCTION_RUNTIMECONTEXT.containing_type = _USERDEFINEDDATASTREAMFUNCTION +_USERDEFINEDDATASTREAMFUNCTION.fields_by_name['function_type'].enum_type = _USERDEFINEDDATASTREAMFUNCTION_FUNCTIONTYPE +_USERDEFINEDDATASTREAMFUNCTION.fields_by_name['runtime_context'].message_type = _USERDEFINEDDATASTREAMFUNCTION_RUNTIMECONTEXT +_USERDEFINEDDATASTREAMFUNCTION.fields_by_name['key_type_info'].message_type = _TYPEINFO +_USERDEFINEDDATASTREAMFUNCTION_FUNCTIONTYPE.containing_type = _USERDEFINEDDATASTREAMFUNCTION +_CODERINFODESCRIPTOR_FLATTENROWTYPE.fields_by_name['schema'].message_type = _SCHEMA +_CODERINFODESCRIPTOR_FLATTENROWTYPE.containing_type = _CODERINFODESCRIPTOR +_CODERINFODESCRIPTOR_ROWTYPE.fields_by_name['schema'].message_type = _SCHEMA +_CODERINFODESCRIPTOR_ROWTYPE.containing_type = _CODERINFODESCRIPTOR +_CODERINFODESCRIPTOR_ARROWTYPE.fields_by_name['schema'].message_type = _SCHEMA +_CODERINFODESCRIPTOR_ARROWTYPE.containing_type = _CODERINFODESCRIPTOR +_CODERINFODESCRIPTOR_OVERWINDOWARROWTYPE.fields_by_name['schema'].message_type = _SCHEMA +_CODERINFODESCRIPTOR_OVERWINDOWARROWTYPE.containing_type = _CODERINFODESCRIPTOR +_CODERINFODESCRIPTOR_RAWTYPE.fields_by_name['type_info'].message_type = _TYPEINFO +_CODERINFODESCRIPTOR_RAWTYPE.containing_type = _CODERINFODESCRIPTOR +_CODERINFODESCRIPTOR.fields_by_name['flatten_row_type'].message_type = _CODERINFODESCRIPTOR_FLATTENROWTYPE +_CODERINFODESCRIPTOR.fields_by_name['row_type'].message_type = _CODERINFODESCRIPTOR_ROWTYPE +_CODERINFODESCRIPTOR.fields_by_name['arrow_type'].message_type = _CODERINFODESCRIPTOR_ARROWTYPE +_CODERINFODESCRIPTOR.fields_by_name['over_window_arrow_type'].message_type = _CODERINFODESCRIPTOR_OVERWINDOWARROWTYPE +_CODERINFODESCRIPTOR.fields_by_name['raw_type'].message_type = _CODERINFODESCRIPTOR_RAWTYPE +_CODERINFODESCRIPTOR.fields_by_name['mode'].enum_type = _CODERINFODESCRIPTOR_MODE +_CODERINFODESCRIPTOR_MODE.containing_type = _CODERINFODESCRIPTOR +_CODERINFODESCRIPTOR.oneofs_by_name['data_type'].fields.append( + _CODERINFODESCRIPTOR.fields_by_name['flatten_row_type']) +_CODERINFODESCRIPTOR.fields_by_name['flatten_row_type'].containing_oneof = _CODERINFODESCRIPTOR.oneofs_by_name['data_type'] +_CODERINFODESCRIPTOR.oneofs_by_name['data_type'].fields.append( + _CODERINFODESCRIPTOR.fields_by_name['row_type']) +_CODERINFODESCRIPTOR.fields_by_name['row_type'].containing_oneof = _CODERINFODESCRIPTOR.oneofs_by_name['data_type'] +_CODERINFODESCRIPTOR.oneofs_by_name['data_type'].fields.append( + _CODERINFODESCRIPTOR.fields_by_name['arrow_type']) +_CODERINFODESCRIPTOR.fields_by_name['arrow_type'].containing_oneof = _CODERINFODESCRIPTOR.oneofs_by_name['data_type'] +_CODERINFODESCRIPTOR.oneofs_by_name['data_type'].fields.append( + _CODERINFODESCRIPTOR.fields_by_name['over_window_arrow_type']) +_CODERINFODESCRIPTOR.fields_by_name['over_window_arrow_type'].containing_oneof = _CODERINFODESCRIPTOR.oneofs_by_name['data_type'] +_CODERINFODESCRIPTOR.oneofs_by_name['data_type'].fields.append( + _CODERINFODESCRIPTOR.fields_by_name['raw_type']) +_CODERINFODESCRIPTOR.fields_by_name['raw_type'].containing_oneof = _CODERINFODESCRIPTOR.oneofs_by_name['data_type'] DESCRIPTOR.message_types_by_name['Input'] = _INPUT DESCRIPTOR.message_types_by_name['UserDefinedFunction'] = _USERDEFINEDFUNCTION DESCRIPTOR.message_types_by_name['UserDefinedFunctions'] = _USERDEFINEDFUNCTIONS DESCRIPTOR.message_types_by_name['OverWindow'] = _OVERWINDOW -DESCRIPTOR.message_types_by_name['UserDefinedDataStreamFunction'] = _USERDEFINEDDATASTREAMFUNCTION DESCRIPTOR.message_types_by_name['UserDefinedAggregateFunction'] = _USERDEFINEDAGGREGATEFUNCTION DESCRIPTOR.message_types_by_name['GroupWindow'] = _GROUPWINDOW DESCRIPTOR.message_types_by_name['UserDefinedAggregateFunctions'] = _USERDEFINEDAGGREGATEFUNCTIONS DESCRIPTOR.message_types_by_name['Schema'] = _SCHEMA -DESCRIPTOR.message_types_by_name['CoderParam'] = _CODERPARAM DESCRIPTOR.message_types_by_name['TypeInfo'] = _TYPEINFO +DESCRIPTOR.message_types_by_name['UserDefinedDataStreamFunction'] = _USERDEFINEDDATASTREAMFUNCTION +DESCRIPTOR.message_types_by_name['CoderInfoDescriptor'] = _CODERINFODESCRIPTOR _sym_db.RegisterFileDescriptor(DESCRIPTOR) Input = _reflection.GeneratedProtocolMessageType('Input', (_message.Message,), dict( @@ -2177,29 +2329,6 @@ )) _sym_db.RegisterMessage(OverWindow) -UserDefinedDataStreamFunction = _reflection.GeneratedProtocolMessageType('UserDefinedDataStreamFunction', (_message.Message,), dict( - - JobParameter = _reflection.GeneratedProtocolMessageType('JobParameter', (_message.Message,), dict( - DESCRIPTOR = _USERDEFINEDDATASTREAMFUNCTION_JOBPARAMETER, - __module__ = 'flink_fn_execution_pb2' - # @@protoc_insertion_point(class_scope:org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.JobParameter) - )) - , - - RuntimeContext = _reflection.GeneratedProtocolMessageType('RuntimeContext', (_message.Message,), dict( - DESCRIPTOR = _USERDEFINEDDATASTREAMFUNCTION_RUNTIMECONTEXT, - __module__ = 'flink_fn_execution_pb2' - # @@protoc_insertion_point(class_scope:org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.RuntimeContext) - )) - , - DESCRIPTOR = _USERDEFINEDDATASTREAMFUNCTION, - __module__ = 'flink_fn_execution_pb2' - # @@protoc_insertion_point(class_scope:org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction) - )) -_sym_db.RegisterMessage(UserDefinedDataStreamFunction) -_sym_db.RegisterMessage(UserDefinedDataStreamFunction.JobParameter) -_sym_db.RegisterMessage(UserDefinedDataStreamFunction.RuntimeContext) - UserDefinedAggregateFunction = _reflection.GeneratedProtocolMessageType('UserDefinedAggregateFunction', (_message.Message,), dict( DataViewSpec = _reflection.GeneratedProtocolMessageType('DataViewSpec', (_message.Message,), dict( @@ -2348,13 +2477,6 @@ _sym_db.RegisterMessage(Schema.FieldType) _sym_db.RegisterMessage(Schema.Field) -CoderParam = _reflection.GeneratedProtocolMessageType('CoderParam', (_message.Message,), dict( - DESCRIPTOR = _CODERPARAM, - __module__ = 'flink_fn_execution_pb2' - # @@protoc_insertion_point(class_scope:org.apache.flink.fn_execution.v1.CoderParam) - )) -_sym_db.RegisterMessage(CoderParam) - TypeInfo = _reflection.GeneratedProtocolMessageType('TypeInfo', (_message.Message,), dict( MapTypeInfo = _reflection.GeneratedProtocolMessageType('MapTypeInfo', (_message.Message,), dict( @@ -2394,6 +2516,76 @@ _sym_db.RegisterMessage(TypeInfo.RowTypeInfo.Field) _sym_db.RegisterMessage(TypeInfo.TupleTypeInfo) +UserDefinedDataStreamFunction = _reflection.GeneratedProtocolMessageType('UserDefinedDataStreamFunction', (_message.Message,), dict( + + JobParameter = _reflection.GeneratedProtocolMessageType('JobParameter', (_message.Message,), dict( + DESCRIPTOR = _USERDEFINEDDATASTREAMFUNCTION_JOBPARAMETER, + __module__ = 'flink_fn_execution_pb2' + # @@protoc_insertion_point(class_scope:org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.JobParameter) + )) + , + + RuntimeContext = _reflection.GeneratedProtocolMessageType('RuntimeContext', (_message.Message,), dict( + DESCRIPTOR = _USERDEFINEDDATASTREAMFUNCTION_RUNTIMECONTEXT, + __module__ = 'flink_fn_execution_pb2' + # @@protoc_insertion_point(class_scope:org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.RuntimeContext) + )) + , + DESCRIPTOR = _USERDEFINEDDATASTREAMFUNCTION, + __module__ = 'flink_fn_execution_pb2' + # @@protoc_insertion_point(class_scope:org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction) + )) +_sym_db.RegisterMessage(UserDefinedDataStreamFunction) +_sym_db.RegisterMessage(UserDefinedDataStreamFunction.JobParameter) +_sym_db.RegisterMessage(UserDefinedDataStreamFunction.RuntimeContext) + +CoderInfoDescriptor = _reflection.GeneratedProtocolMessageType('CoderInfoDescriptor', (_message.Message,), dict( + + FlattenRowType = _reflection.GeneratedProtocolMessageType('FlattenRowType', (_message.Message,), dict( + DESCRIPTOR = _CODERINFODESCRIPTOR_FLATTENROWTYPE, + __module__ = 'flink_fn_execution_pb2' + # @@protoc_insertion_point(class_scope:org.apache.flink.fn_execution.v1.CoderInfoDescriptor.FlattenRowType) + )) + , + + RowType = _reflection.GeneratedProtocolMessageType('RowType', (_message.Message,), dict( + DESCRIPTOR = _CODERINFODESCRIPTOR_ROWTYPE, + __module__ = 'flink_fn_execution_pb2' + # @@protoc_insertion_point(class_scope:org.apache.flink.fn_execution.v1.CoderInfoDescriptor.RowType) + )) + , + + ArrowType = _reflection.GeneratedProtocolMessageType('ArrowType', (_message.Message,), dict( + DESCRIPTOR = _CODERINFODESCRIPTOR_ARROWTYPE, + __module__ = 'flink_fn_execution_pb2' + # @@protoc_insertion_point(class_scope:org.apache.flink.fn_execution.v1.CoderInfoDescriptor.ArrowType) + )) + , + + OverWindowArrowType = _reflection.GeneratedProtocolMessageType('OverWindowArrowType', (_message.Message,), dict( + DESCRIPTOR = _CODERINFODESCRIPTOR_OVERWINDOWARROWTYPE, + __module__ = 'flink_fn_execution_pb2' + # @@protoc_insertion_point(class_scope:org.apache.flink.fn_execution.v1.CoderInfoDescriptor.OverWindowArrowType) + )) + , + + RawType = _reflection.GeneratedProtocolMessageType('RawType', (_message.Message,), dict( + DESCRIPTOR = _CODERINFODESCRIPTOR_RAWTYPE, + __module__ = 'flink_fn_execution_pb2' + # @@protoc_insertion_point(class_scope:org.apache.flink.fn_execution.v1.CoderInfoDescriptor.RawType) + )) + , + DESCRIPTOR = _CODERINFODESCRIPTOR, + __module__ = 'flink_fn_execution_pb2' + # @@protoc_insertion_point(class_scope:org.apache.flink.fn_execution.v1.CoderInfoDescriptor) + )) +_sym_db.RegisterMessage(CoderInfoDescriptor) +_sym_db.RegisterMessage(CoderInfoDescriptor.FlattenRowType) +_sym_db.RegisterMessage(CoderInfoDescriptor.RowType) +_sym_db.RegisterMessage(CoderInfoDescriptor.ArrowType) +_sym_db.RegisterMessage(CoderInfoDescriptor.OverWindowArrowType) +_sym_db.RegisterMessage(CoderInfoDescriptor.RawType) + DESCRIPTOR.has_options = True DESCRIPTOR._options = _descriptor._ParseOptions(descriptor_pb2.FileOptions(), _b('\n\037org.apache.flink.fnexecution.v1B\nFlinkFnApi')) diff --git a/flink-python/pyflink/proto/flink-fn-execution.proto b/flink-python/pyflink/proto/flink-fn-execution.proto index 78e25f4a1a983..7dfa83eab8c47 100644 --- a/flink-python/pyflink/proto/flink-fn-execution.proto +++ b/flink-python/pyflink/proto/flink-fn-execution.proto @@ -25,6 +25,10 @@ package org.apache.flink.fn_execution.v1; option java_package = "org.apache.flink.fnexecution.v1"; option java_outer_classname = "FlinkFnApi"; +// ------------------------------------------------------------------------ +// Table API & SQL +// ------------------------------------------------------------------------ + message Input { oneof input { UserDefinedFunction udf = 1; @@ -79,43 +83,6 @@ message OverWindow { int64 upper_boundary = 3; } -// User defined DataStream function definition. -message UserDefinedDataStreamFunction { - enum FunctionType { - MAP = 0; - FLAT_MAP = 1; - CO_MAP = 2; - CO_FLAT_MAP = 3; - PROCESS = 4; - KEYED_PROCESS = 5; - KEYED_CO_PROCESS = 6; - TIMESTAMP_ASSIGNER = 7; - WINDOW = 8; - } - - message JobParameter { - string key = 1; - string value = 2; - } - - message RuntimeContext { - string task_name = 1; - string task_name_with_subtasks = 2; - int32 number_of_parallel_subtasks = 3; - int32 max_number_of_parallel_subtasks = 4; - int32 index_of_this_subtask = 5; - int32 attempt_number = 6; - repeated JobParameter job_parameters = 7; - bool in_batch_execution_mode = 8; - } - - FunctionType function_type = 1; - RuntimeContext runtime_context = 2; - bytes payload = 3; - bool metric_enabled = 4; - TypeInfo key_type_info = 5; -} - message UserDefinedAggregateFunction { message DataViewSpec { message ListView { @@ -310,40 +277,9 @@ message Schema { repeated Field fields = 1; } -// A representation of the coder -message CoderParam { - enum DataType { - // used in udf/udtf - FLATTEN_ROW = 0; - // used in udaf/udtaf - ROW = 1; - // used in datastream - RAW = 2; - // used pandas udf/udaf - ARROW = 3; - // only used in batch over window - // the data consists of [window data][arrow data] - OVER_WINDOW_ARROW=4; - } - - enum OutputMode { - // output single result - SINGLE = 0; - // output multiple result - MULTIPLE = 1; - // output multiple result and a end message(0x00) - MULTIPLE_WITH_END = 2; - } - - oneof field_type { - Schema schema = 1; - TypeInfo type_info = 2; - } - - DataType data_type = 3; - - OutputMode output_mode = 4; -} +// ------------------------------------------------------------------------ +// DataStream API +// ------------------------------------------------------------------------ // A representation of the data type information in DataStream. message TypeInfo { @@ -397,3 +333,94 @@ message TypeInfo { MapTypeInfo map_type_info = 5; } } + +// User defined DataStream function definition. +message UserDefinedDataStreamFunction { + enum FunctionType { + MAP = 0; + FLAT_MAP = 1; + CO_MAP = 2; + CO_FLAT_MAP = 3; + PROCESS = 4; + KEYED_PROCESS = 5; + KEYED_CO_PROCESS = 6; + TIMESTAMP_ASSIGNER = 7; + WINDOW = 8; + } + + message JobParameter { + string key = 1; + string value = 2; + } + + message RuntimeContext { + string task_name = 1; + string task_name_with_subtasks = 2; + int32 number_of_parallel_subtasks = 3; + int32 max_number_of_parallel_subtasks = 4; + int32 index_of_this_subtask = 5; + int32 attempt_number = 6; + repeated JobParameter job_parameters = 7; + bool in_batch_execution_mode = 8; + } + + FunctionType function_type = 1; + RuntimeContext runtime_context = 2; + bytes payload = 3; + bool metric_enabled = 4; + TypeInfo key_type_info = 5; +} + +// ------------------------------------------------------------------------ +// Common of Table API and DataStream API +// ------------------------------------------------------------------------ + +// A representation of the coder +message CoderInfoDescriptor { + // for Table & SQL + message FlattenRowType { + Schema schema = 1; + } + + message RowType { + Schema schema = 1; + } + + message ArrowType { + Schema schema = 1; + } + + // only used in batch over window + // the data consists of [window data][arrow data] + message OverWindowArrowType { + Schema schema = 1; + } + + // for DataStream + message RawType { + TypeInfo type_info = 1; + } + + enum Mode { + // process one record at a time + SINGLE = 0; + // process multiple records at a time + MULTIPLE = 1; + } + + oneof data_type { + // for Table & SQL + FlattenRowType flatten_row_type = 1; + RowType row_type = 2; + ArrowType arrow_type = 3; + OverWindowArrowType over_window_arrow_type = 4; + + // for DataStream + RawType raw_type = 5; + } + + Mode mode = 6; + + // append an end message (0x00) after serializing the records belonging to the same batch. + bool separated_with_end_message = 7; +} diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/OneInputPythonFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/OneInputPythonFunctionOperator.java index 53e4b4621f48a..94c7aad494c9f 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/OneInputPythonFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/OneInputPythonFunctionOperator.java @@ -40,7 +40,8 @@ import java.util.Map; import static org.apache.flink.python.Constants.STATELESS_FUNCTION_URN; -import static org.apache.flink.streaming.api.utils.PythonOperatorUtils.getUserDefinedDataStreamFunctionProto; +import static org.apache.flink.streaming.api.utils.ProtoUtils.createRawTypeCoderInfoDescriptorProto; +import static org.apache.flink.streaming.api.utils.ProtoUtils.getUserDefinedDataStreamFunctionProto; import static org.apache.flink.streaming.api.utils.PythonOperatorUtils.inBatchExecutionMode; /** @@ -68,10 +69,6 @@ public abstract class OneInputPythonFunctionOperator /** The serialized python function to be executed. */ private final DataStreamPythonFunctionInfo pythonFunctionInfo; - private final FlinkFnApi.CoderParam.DataType inputDataType; - private final FlinkFnApi.CoderParam.DataType outputDataType; - private final FlinkFnApi.CoderParam.OutputMode outputMode; - /** The TypeSerializer of python worker input data. */ transient TypeSerializer runnerInputTypeSerializer; @@ -98,18 +95,12 @@ public OneInputPythonFunctionOperator( Configuration config, TypeInformation runnerInputTypeInfo, TypeInformation runnerOutputTypeInfo, - FlinkFnApi.CoderParam.DataType inputDataType, - FlinkFnApi.CoderParam.DataType outputDataType, - FlinkFnApi.CoderParam.OutputMode outputMode, DataStreamPythonFunctionInfo pythonFunctionInfo) { super(config); this.jobOptions = config.toMap(); this.runnerInputTypeInfo = runnerInputTypeInfo; this.runnerOutputTypeInfo = runnerOutputTypeInfo; this.pythonFunctionInfo = pythonFunctionInfo; - this.inputDataType = inputDataType; - this.outputDataType = outputDataType; - this.outputMode = outputMode; } @Override @@ -136,8 +127,6 @@ public PythonFunctionRunner createPythonFunctionRunner() throws Exception { return new BeamDataStreamPythonFunctionRunner( getRuntimeContext().getTaskName(), createPythonEnvironmentManager(), - runnerInputTypeInfo, - runnerOutputTypeInfo, getFunctionUrn(), getUserDefinedDataStreamFunctionProto( pythonFunctionInfo, @@ -161,9 +150,8 @@ public PythonFunctionRunner createPythonFunctionRunner() throws Exception { .getEnvironment() .getUserCodeClassLoader() .asClassLoader()), - inputDataType, - outputDataType, - outputMode); + createInputCoderInfoDescriptor(runnerInputTypeInfo), + createOutputCoderInfoDescriptor(runnerOutputTypeInfo)); } @Override @@ -189,4 +177,16 @@ public String getFunctionUrn() { public Map getInternalParameters() { return Collections.EMPTY_MAP; } + + public FlinkFnApi.CoderInfoDescriptor createInputCoderInfoDescriptor( + TypeInformation runnerInputType) { + return createRawTypeCoderInfoDescriptorProto( + runnerInputType, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, false); + } + + public FlinkFnApi.CoderInfoDescriptor createOutputCoderInfoDescriptor( + TypeInformation runnerOutType) { + return createRawTypeCoderInfoDescriptorProto( + runnerOutType, FlinkFnApi.CoderInfoDescriptor.Mode.SINGLE, false); + } } diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonCoFlatMapOperator.java b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonCoFlatMapOperator.java index cd48f30fea460..9468ef1a058f9 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonCoFlatMapOperator.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonCoFlatMapOperator.java @@ -25,6 +25,8 @@ import org.apache.flink.streaming.api.functions.python.DataStreamPythonFunctionInfo; import org.apache.flink.streaming.api.utils.PythonOperatorUtils; +import static org.apache.flink.streaming.api.utils.ProtoUtils.createRawTypeCoderInfoDescriptorProto; + /** * The {@link PythonCoFlatMapOperator} is responsible for executing the Python CoMap Function. * @@ -44,13 +46,7 @@ public PythonCoFlatMapOperator( TypeInformation inputTypeInfo2, TypeInformation outputTypeInfo, DataStreamPythonFunctionInfo pythonFunctionInfo) { - super( - config, - inputTypeInfo1, - inputTypeInfo2, - outputTypeInfo, - pythonFunctionInfo, - FlinkFnApi.CoderParam.OutputMode.MULTIPLE_WITH_END); + super(config, inputTypeInfo1, inputTypeInfo2, outputTypeInfo, pythonFunctionInfo); } @Override @@ -66,4 +62,18 @@ public void emitResult(Tuple2 resultTuple) throws Exception { collector.collect(outputRow); } } + + @Override + public FlinkFnApi.CoderInfoDescriptor createInputCoderInfoDescriptor( + TypeInformation runnerInputType) { + return createRawTypeCoderInfoDescriptorProto( + runnerInputType, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, true); + } + + @Override + public FlinkFnApi.CoderInfoDescriptor createOutputCoderInfoDescriptor( + TypeInformation runnerOutType) { + return createRawTypeCoderInfoDescriptorProto( + runnerOutType, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, true); + } } diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonCoMapOperator.java b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonCoMapOperator.java index 468fcb31d2a57..19370b70a9bbf 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonCoMapOperator.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonCoMapOperator.java @@ -24,6 +24,8 @@ import org.apache.flink.fnexecution.v1.FlinkFnApi; import org.apache.flink.streaming.api.functions.python.DataStreamPythonFunctionInfo; +import static org.apache.flink.streaming.api.utils.ProtoUtils.createRawTypeCoderInfoDescriptorProto; + /** * The {@link PythonCoFlatMapOperator} is responsible for executing the Python CoMap Function. * @@ -43,13 +45,7 @@ public PythonCoMapOperator( TypeInformation inputTypeInfo2, TypeInformation outputTypeInfo, DataStreamPythonFunctionInfo pythonFunctionInfo) { - super( - config, - inputTypeInfo1, - inputTypeInfo2, - outputTypeInfo, - pythonFunctionInfo, - FlinkFnApi.CoderParam.OutputMode.SINGLE); + super(config, inputTypeInfo1, inputTypeInfo2, outputTypeInfo, pythonFunctionInfo); } @Override @@ -61,4 +57,18 @@ public void emitResult(Tuple2 resultTuple) throws Exception { collector.setAbsoluteTimestamp(bufferedTimestamp.poll()); collector.collect(output); } + + @Override + public FlinkFnApi.CoderInfoDescriptor createInputCoderInfoDescriptor( + TypeInformation runnerInputType) { + return createRawTypeCoderInfoDescriptorProto( + runnerInputType, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, false); + } + + @Override + public FlinkFnApi.CoderInfoDescriptor createOutputCoderInfoDescriptor( + TypeInformation runnerOutType) { + return createRawTypeCoderInfoDescriptorProto( + runnerOutType, FlinkFnApi.CoderInfoDescriptor.Mode.SINGLE, false); + } } diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonFlatMapOperator.java b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonFlatMapOperator.java index b371d59d755e1..7d0dcb272556c 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonFlatMapOperator.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonFlatMapOperator.java @@ -26,6 +26,8 @@ import org.apache.flink.streaming.api.functions.python.DataStreamPythonFunctionInfo; import org.apache.flink.streaming.api.utils.PythonOperatorUtils; +import static org.apache.flink.streaming.api.utils.ProtoUtils.createRawTypeCoderInfoDescriptorProto; + /** * The {@link PythonFlatMapOperator} is responsible for executing Python functions that gets one * input and produces zero/one or more outputs. @@ -43,14 +45,7 @@ public PythonFlatMapOperator( TypeInformation inputTypeInfo, TypeInformation outputTypeInfo, DataStreamPythonFunctionInfo pythonFunctionInfo) { - super( - config, - inputTypeInfo, - outputTypeInfo, - FlinkFnApi.CoderParam.DataType.RAW, - FlinkFnApi.CoderParam.DataType.RAW, - FlinkFnApi.CoderParam.OutputMode.MULTIPLE_WITH_END, - pythonFunctionInfo); + super(config, inputTypeInfo, outputTypeInfo, pythonFunctionInfo); } @Override @@ -65,4 +60,18 @@ public void emitResult(Tuple2 resultTuple) throws Exception { collector.collect(runnerOutputTypeSerializer.deserialize(baisWrapper)); } } + + @Override + public FlinkFnApi.CoderInfoDescriptor createInputCoderInfoDescriptor( + TypeInformation runnerInputType) { + return createRawTypeCoderInfoDescriptorProto( + runnerInputType, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, true); + } + + @Override + public FlinkFnApi.CoderInfoDescriptor createOutputCoderInfoDescriptor( + TypeInformation runnerOutType) { + return createRawTypeCoderInfoDescriptorProto( + runnerOutType, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, true); + } } diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonKeyedCoProcessOperator.java b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonKeyedCoProcessOperator.java index d1c982b2913cd..8f3841acbc76d 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonKeyedCoProcessOperator.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonKeyedCoProcessOperator.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.api.operators.python; +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.tuple.Tuple2; @@ -36,6 +37,7 @@ import org.apache.flink.streaming.api.operators.TimestampedCollector; import org.apache.flink.streaming.api.operators.Triggerable; import org.apache.flink.streaming.api.runners.python.beam.BeamDataStreamPythonFunctionRunner; +import org.apache.flink.streaming.api.utils.ProtoUtils; import org.apache.flink.streaming.api.utils.PythonOperatorUtils; import org.apache.flink.streaming.api.utils.PythonTypeUtils; import org.apache.flink.streaming.api.utils.input.KeyedTwoInputWithTimerRowFactory; @@ -46,13 +48,17 @@ import java.util.Collections; import static org.apache.flink.python.Constants.STATEFUL_FUNCTION_URN; +import static org.apache.flink.streaming.api.utils.ProtoUtils.createRawTypeCoderInfoDescriptorProto; import static org.apache.flink.streaming.api.utils.PythonOperatorUtils.inBatchExecutionMode; /** KeyedCoProcessOperator. */ +@Internal public class PythonKeyedCoProcessOperator extends TwoInputPythonFunctionOperator implements ResultTypeQueryable, Triggerable { + private static final long serialVersionUID = 1L; + /** The TypeInformation of current key. */ private final TypeInformation keyTypeInfo; @@ -81,8 +87,7 @@ public PythonKeyedCoProcessOperator( KeyedTwoInputWithTimerRowFactory.getRunnerInputTypeInfo( inputTypeInfo1, inputTypeInfo2, constructKeyTypeInfo(inputTypeInfo1)), OutputWithTimerRowHandler.getRunnerOutputTypeInfo( - outputTypeInfo, constructKeyTypeInfo(inputTypeInfo1)), - FlinkFnApi.CoderParam.OutputMode.MULTIPLE_WITH_END); + outputTypeInfo, constructKeyTypeInfo(inputTypeInfo1))); this.keyTypeInfo = constructKeyTypeInfo(inputTypeInfo1); this.keyTypeSerializer = PythonTypeUtils.TypeInfoToSerializerConverter.typeInfoSerializerConverter( @@ -95,13 +100,11 @@ public PythonFunctionRunner createPythonFunctionRunner() throws Exception { return new BeamDataStreamPythonFunctionRunner( getRuntimeContext().getTaskName(), createPythonEnvironmentManager(), - getRunnerInputTypeInfo(), - getRunnerOutputTypeInfo(), STATEFUL_FUNCTION_URN, - PythonOperatorUtils.getUserDefinedDataStreamStatefulFunctionProto( + ProtoUtils.getUserDefinedDataStreamStatefulFunctionProto( getPythonFunctionInfo(), getRuntimeContext(), - Collections.EMPTY_MAP, + Collections.emptyMap(), keyTypeInfo, inBatchExecutionMode(getKeyedStateBackend())), getJobOptions(), @@ -121,9 +124,8 @@ public PythonFunctionRunner createPythonFunctionRunner() throws Exception { .getEnvironment() .getUserCodeClassLoader() .asClassLoader()), - FlinkFnApi.CoderParam.DataType.FLATTEN_ROW, - FlinkFnApi.CoderParam.DataType.RAW, - FlinkFnApi.CoderParam.OutputMode.MULTIPLE_WITH_END); + createInputCoderInfoDescriptor(runnerInputTypeInfo), + createOutputCoderInfoDescriptor(runnerOutputTypeInfo)); } @Override @@ -245,4 +247,18 @@ public void setCurrentKey(Object key) { public Object getCurrentKey() { return keyForTimerService; } + + @Override + public FlinkFnApi.CoderInfoDescriptor createInputCoderInfoDescriptor( + TypeInformation runnerInputType) { + return createRawTypeCoderInfoDescriptorProto( + runnerInputType, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, true); + } + + @Override + public FlinkFnApi.CoderInfoDescriptor createOutputCoderInfoDescriptor( + TypeInformation runnerOutType) { + return createRawTypeCoderInfoDescriptorProto( + runnerOutType, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, true); + } } diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonKeyedProcessOperator.java b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonKeyedProcessOperator.java index 7e4929f9ce4f6..0efedfb5efdd5 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonKeyedProcessOperator.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonKeyedProcessOperator.java @@ -41,6 +41,7 @@ import org.apache.flink.streaming.api.operators.TimestampedCollector; import org.apache.flink.streaming.api.operators.Triggerable; import org.apache.flink.streaming.api.runners.python.beam.BeamDataStreamPythonFunctionRunner; +import org.apache.flink.streaming.api.utils.ProtoUtils; import org.apache.flink.streaming.api.utils.PythonOperatorUtils; import org.apache.flink.streaming.api.utils.PythonTypeUtils; import org.apache.flink.streaming.api.utils.input.KeyedInputWithTimerRowFactory; @@ -54,6 +55,7 @@ import java.util.Map; import static org.apache.flink.python.Constants.STATEFUL_FUNCTION_URN; +import static org.apache.flink.streaming.api.utils.ProtoUtils.createRawTypeCoderInfoDescriptorProto; import static org.apache.flink.streaming.api.utils.PythonOperatorUtils.inBatchExecutionMode; /** @@ -207,10 +209,8 @@ public PythonFunctionRunner createPythonFunctionRunner() throws Exception { return new BeamDataStreamPythonFunctionRunner( getRuntimeContext().getTaskName(), createPythonEnvironmentManager(), - runnerInputTypeInfo, - runnerOutputTypeInfo, STATEFUL_FUNCTION_URN, - PythonOperatorUtils.getUserDefinedDataStreamStatefulFunctionProto( + ProtoUtils.getUserDefinedDataStreamStatefulFunctionProto( pythonFunctionInfo, getRuntimeContext(), Collections.EMPTY_MAP, @@ -233,9 +233,10 @@ public PythonFunctionRunner createPythonFunctionRunner() throws Exception { .getEnvironment() .getUserCodeClassLoader() .asClassLoader()), - FlinkFnApi.CoderParam.DataType.RAW, - FlinkFnApi.CoderParam.DataType.RAW, - FlinkFnApi.CoderParam.OutputMode.MULTIPLE_WITH_END); + createRawTypeCoderInfoDescriptorProto( + runnerInputTypeInfo, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, true), + createRawTypeCoderInfoDescriptorProto( + runnerOutputTypeInfo, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, true)); } @Override diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonMapOperator.java b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonMapOperator.java index 01a6554373800..9e9a64c33d19f 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonMapOperator.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonMapOperator.java @@ -22,7 +22,6 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; -import org.apache.flink.fnexecution.v1.FlinkFnApi; import org.apache.flink.streaming.api.functions.python.DataStreamPythonFunctionInfo; /** @@ -41,14 +40,7 @@ public PythonMapOperator( TypeInformation inputTypeInfo, TypeInformation outputTypeInfo, DataStreamPythonFunctionInfo pythonFunctionInfo) { - super( - config, - inputTypeInfo, - outputTypeInfo, - FlinkFnApi.CoderParam.DataType.RAW, - FlinkFnApi.CoderParam.DataType.RAW, - FlinkFnApi.CoderParam.OutputMode.SINGLE, - pythonFunctionInfo); + super(config, inputTypeInfo, outputTypeInfo, pythonFunctionInfo); } @Override diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonPartitionCustomOperator.java b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonPartitionCustomOperator.java index 749e8f3887a9c..a0f3025be9472 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonPartitionCustomOperator.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonPartitionCustomOperator.java @@ -23,7 +23,6 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.fnexecution.v1.FlinkFnApi; import org.apache.flink.streaming.api.functions.python.DataStreamPythonFunctionInfo; import java.util.HashMap; @@ -51,14 +50,7 @@ public PythonPartitionCustomOperator( TypeInformation inputTypeInfo, TypeInformation outputTypeInfo, DataStreamPythonFunctionInfo pythonFunctionInfo) { - super( - config, - inputTypeInfo, - outputTypeInfo, - FlinkFnApi.CoderParam.DataType.RAW, - FlinkFnApi.CoderParam.DataType.RAW, - FlinkFnApi.CoderParam.OutputMode.SINGLE, - pythonFunctionInfo); + super(config, inputTypeInfo, outputTypeInfo, pythonFunctionInfo); } @Override diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonProcessOperator.java b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonProcessOperator.java index 2795a42545bdb..9553e7e9cbe8b 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonProcessOperator.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonProcessOperator.java @@ -32,6 +32,7 @@ import org.apache.flink.types.Row; import static org.apache.flink.python.Constants.STATELESS_FUNCTION_URN; +import static org.apache.flink.streaming.api.utils.ProtoUtils.createRawTypeCoderInfoDescriptorProto; /** * {@link PythonProcessOperator} is responsible for launching beam runner which will start a python @@ -58,9 +59,6 @@ public PythonProcessOperator( config, Types.ROW(Types.LONG, Types.LONG, inputTypeInfo), outputTypeInfo, - FlinkFnApi.CoderParam.DataType.RAW, - FlinkFnApi.CoderParam.DataType.RAW, - FlinkFnApi.CoderParam.OutputMode.MULTIPLE_WITH_END, pythonFunctionInfo); } @@ -104,4 +102,18 @@ public void processWatermark(Watermark mark) throws Exception { public String getFunctionUrn() { return STATELESS_FUNCTION_URN; } + + @Override + public FlinkFnApi.CoderInfoDescriptor createInputCoderInfoDescriptor( + TypeInformation runnerInputType) { + return createRawTypeCoderInfoDescriptorProto( + runnerInputType, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, true); + } + + @Override + public FlinkFnApi.CoderInfoDescriptor createOutputCoderInfoDescriptor( + TypeInformation runnerOutType) { + return createRawTypeCoderInfoDescriptorProto( + runnerOutType, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, true); + } } diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonTimestampsAndWatermarksOperator.java b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonTimestampsAndWatermarksOperator.java index 57aea3d121bae..70d45bfd0ba8a 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonTimestampsAndWatermarksOperator.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonTimestampsAndWatermarksOperator.java @@ -28,7 +28,6 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; -import org.apache.flink.fnexecution.v1.FlinkFnApi; import org.apache.flink.streaming.api.functions.python.DataStreamPythonFunctionInfo; import org.apache.flink.streaming.api.utils.PythonTypeUtils; import org.apache.flink.streaming.runtime.operators.TimestampsAndWatermarksOperator; @@ -91,14 +90,7 @@ public PythonTimestampsAndWatermarksOperator( TypeInformation inputTypeInfo, DataStreamPythonFunctionInfo pythonFunctionInfo, WatermarkStrategy watermarkStrategy) { - super( - config, - Types.ROW(Types.LONG, inputTypeInfo), - Types.LONG, - FlinkFnApi.CoderParam.DataType.RAW, - FlinkFnApi.CoderParam.DataType.RAW, - FlinkFnApi.CoderParam.OutputMode.SINGLE, - pythonFunctionInfo); + super(config, Types.ROW(Types.LONG, inputTypeInfo), Types.LONG, pythonFunctionInfo); this.watermarkStrategy = watermarkStrategy; this.inputTypeInfo = inputTypeInfo; } diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/TwoInputPythonFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/TwoInputPythonFunctionOperator.java index c9744f2bb0f4a..20da8d6571d48 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/TwoInputPythonFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/TwoInputPythonFunctionOperator.java @@ -42,7 +42,7 @@ import java.util.Map; import static org.apache.flink.python.Constants.STATELESS_FUNCTION_URN; -import static org.apache.flink.streaming.api.utils.PythonOperatorUtils.getUserDefinedDataStreamFunctionProto; +import static org.apache.flink.streaming.api.utils.ProtoUtils.getUserDefinedDataStreamFunctionProto; import static org.apache.flink.streaming.api.utils.PythonOperatorUtils.inBatchExecutionMode; import static org.apache.flink.streaming.api.utils.PythonTypeUtils.TypeInfoToSerializerConverter.typeInfoSerializerConverter; @@ -63,9 +63,9 @@ public abstract class TwoInputPythonFunctionOperator private final DataStreamPythonFunctionInfo pythonFunctionInfo; /** The TypeInformation of python worker input data. */ - private final TypeInformation runnerInputTypeInfo; + protected final TypeInformation runnerInputTypeInfo; - private final TypeInformation runnerOutputTypeInfo; + protected final TypeInformation runnerOutputTypeInfo; /** The TypeSerializer of python worker input data. */ private final TypeSerializer runnerInputTypeSerializer; @@ -73,8 +73,6 @@ public abstract class TwoInputPythonFunctionOperator /** The TypeSerializer of the runner output. */ private final TypeSerializer runnerOutputTypeSerializer; - private final FlinkFnApi.CoderParam.OutputMode outputMode; - protected transient ByteArrayInputStreamWithPos bais; protected transient DataInputViewStreamWrapper baisWrapper; @@ -83,18 +81,30 @@ public abstract class TwoInputPythonFunctionOperator protected transient DataOutputViewStreamWrapper baosWrapper; - protected transient TimestampedCollector collector; + protected transient TimestampedCollector collector; protected transient Row reuseRow; transient LinkedList bufferedTimestamp; + public TwoInputPythonFunctionOperator( + Configuration config, + TypeInformation inputTypeInfo1, + TypeInformation inputTypeInfo2, + TypeInformation outputTypeInfo, + DataStreamPythonFunctionInfo pythonFunctionInfo) { + this( + config, + pythonFunctionInfo, + new RowTypeInfo(Types.BOOLEAN, inputTypeInfo1, inputTypeInfo2), + (TypeInformation) outputTypeInfo); + } + public TwoInputPythonFunctionOperator( Configuration config, DataStreamPythonFunctionInfo pythonFunctionInfo, TypeInformation runnerInputTypeInfo, - TypeInformation runnerOutputTypeInfo, - FlinkFnApi.CoderParam.OutputMode outputMode) { + TypeInformation runnerOutputTypeInfo) { super(config); this.jobOptions = config.toMap(); this.pythonFunctionInfo = pythonFunctionInfo; @@ -102,22 +112,6 @@ public TwoInputPythonFunctionOperator( this.runnerOutputTypeInfo = runnerOutputTypeInfo; this.runnerInputTypeSerializer = typeInfoSerializerConverter(runnerInputTypeInfo); this.runnerOutputTypeSerializer = typeInfoSerializerConverter(runnerOutputTypeInfo); - this.outputMode = outputMode; - } - - public TwoInputPythonFunctionOperator( - Configuration config, - TypeInformation inputTypeInfo1, - TypeInformation inputTypeInfo2, - TypeInformation outputTypeInfo, - DataStreamPythonFunctionInfo pythonFunctionInfo, - FlinkFnApi.CoderParam.OutputMode outputMode) { - this( - config, - pythonFunctionInfo, - new RowTypeInfo(Types.BOOLEAN, inputTypeInfo1, inputTypeInfo2), - (TypeInformation) outputTypeInfo, - outputMode); } @Override @@ -129,7 +123,7 @@ public void open() throws Exception { bufferedTimestamp = new LinkedList<>(); - collector = new TimestampedCollector(output); + collector = new TimestampedCollector<>(output); reuseRow = new Row(3); super.open(); @@ -140,13 +134,11 @@ public PythonFunctionRunner createPythonFunctionRunner() throws Exception { return new BeamDataStreamPythonFunctionRunner( getRuntimeContext().getTaskName(), createPythonEnvironmentManager(), - runnerInputTypeInfo, - runnerOutputTypeInfo, STATELESS_FUNCTION_URN, getUserDefinedDataStreamFunctionProto( pythonFunctionInfo, getRuntimeContext(), - Collections.EMPTY_MAP, + Collections.emptyMap(), inBatchExecutionMode(getKeyedStateBackend())), jobOptions, getFlinkMetricContainer(), @@ -165,9 +157,8 @@ public PythonFunctionRunner createPythonFunctionRunner() throws Exception { .getEnvironment() .getUserCodeClassLoader() .asClassLoader()), - FlinkFnApi.CoderParam.DataType.FLATTEN_ROW, - FlinkFnApi.CoderParam.DataType.RAW, - outputMode); + createInputCoderInfoDescriptor(runnerInputTypeInfo), + createOutputCoderInfoDescriptor(runnerOutputTypeInfo)); } @Override @@ -195,16 +186,14 @@ public void processElement2(StreamRecord element) throws Exception { processElementInternal(); } - protected Map getJobOptions() { - return jobOptions; - } + public abstract FlinkFnApi.CoderInfoDescriptor createInputCoderInfoDescriptor( + TypeInformation runnerInputType); - protected TypeInformation getRunnerInputTypeInfo() { - return runnerInputTypeInfo; - } + public abstract FlinkFnApi.CoderInfoDescriptor createOutputCoderInfoDescriptor( + TypeInformation runnerOutType); - protected TypeInformation getRunnerOutputTypeInfo() { - return runnerOutputTypeInfo; + protected Map getJobOptions() { + return jobOptions; } protected DataStreamPythonFunctionInfo getPythonFunctionInfo() { diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamDataStreamPythonFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamDataStreamPythonFunctionRunner.java index fa138d65eb143..5024ed0ce6d5f 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamDataStreamPythonFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamDataStreamPythonFunctionRunner.java @@ -19,23 +19,17 @@ package org.apache.flink.streaming.api.runners.python.beam; 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.fnexecution.v1.FlinkFnApi; import org.apache.flink.python.env.PythonEnvironmentManager; import org.apache.flink.python.metric.FlinkMetricContainer; import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.runtime.state.KeyedStateBackend; -import org.apache.flink.streaming.api.utils.PythonTypeUtils; - -import org.apache.beam.model.pipeline.v1.RunnerApi; import javax.annotation.Nullable; import java.util.Map; -import static org.apache.flink.python.Constants.FLINK_CODER_URN; - /** * {@link BeamDataStreamPythonFunctionRunner} is responsible for starting a beam python harness to * execute user defined python function. @@ -43,15 +37,11 @@ @Internal public class BeamDataStreamPythonFunctionRunner extends BeamPythonFunctionRunner { - private final TypeInformation inputType; - private final TypeInformation outputType; private final FlinkFnApi.UserDefinedDataStreamFunction userDefinedDataStreamFunction; public BeamDataStreamPythonFunctionRunner( String taskName, PythonEnvironmentManager environmentManager, - TypeInformation inputType, - TypeInformation outputType, String functionUrn, FlinkFnApi.UserDefinedDataStreamFunction userDefinedDataStreamFunction, Map jobOptions, @@ -61,9 +51,8 @@ public BeamDataStreamPythonFunctionRunner( TypeSerializer namespaceSerializer, MemoryManager memoryManager, double managedMemoryFraction, - FlinkFnApi.CoderParam.DataType inputDataType, - FlinkFnApi.CoderParam.DataType outputDataType, - FlinkFnApi.CoderParam.OutputMode outputMode) { + FlinkFnApi.CoderInfoDescriptor inputCoderDescriptor, + FlinkFnApi.CoderInfoDescriptor outputCoderDescriptor) { super( taskName, environmentManager, @@ -75,11 +64,8 @@ public BeamDataStreamPythonFunctionRunner( namespaceSerializer, memoryManager, managedMemoryFraction, - inputDataType, - outputDataType, - outputMode); - this.inputType = inputType; - this.outputType = outputType; + inputCoderDescriptor, + outputCoderDescriptor); this.userDefinedDataStreamFunction = userDefinedDataStreamFunction; } @@ -87,34 +73,4 @@ public BeamDataStreamPythonFunctionRunner( protected byte[] getUserDefinedFunctionsProtoBytes() { return this.userDefinedDataStreamFunction.toByteArray(); } - - @Override - protected RunnerApi.Coder getInputCoderProto() { - return getInputOutputCoderProto(inputType, inputDataType); - } - - @Override - protected RunnerApi.Coder getOutputCoderProto() { - return getInputOutputCoderProto(outputType, outputDataType); - } - - private RunnerApi.Coder getInputOutputCoderProto( - TypeInformation typeInformation, FlinkFnApi.CoderParam.DataType dataType) { - FlinkFnApi.CoderParam.Builder coderParamBuilder = FlinkFnApi.CoderParam.newBuilder(); - FlinkFnApi.TypeInfo typeinfo = - PythonTypeUtils.TypeInfoToProtoConverter.toTypeInfoProto(typeInformation); - coderParamBuilder.setTypeInfo(typeinfo); - coderParamBuilder.setDataType(dataType); - coderParamBuilder.setOutputMode(outputMode); - return RunnerApi.Coder.newBuilder() - .setSpec( - RunnerApi.FunctionSpec.newBuilder() - .setUrn(FLINK_CODER_URN) - .setPayload( - org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf - .ByteString.copyFrom( - coderParamBuilder.build().toByteArray())) - .build()) - .build(); - } } diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamPythonFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamPythonFunctionRunner.java index 30992b6c41845..d485d6cd804ef 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamPythonFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamPythonFunctionRunner.java @@ -81,6 +81,7 @@ import java.util.concurrent.LinkedBlockingQueue; import static org.apache.beam.runners.core.construction.BeamUrns.getUrn; +import static org.apache.flink.streaming.api.utils.ProtoUtils.createCoderProto; /** A {@link BeamPythonFunctionRunner} used to execute Python functions. */ @Internal @@ -122,10 +123,8 @@ public abstract class BeamPythonFunctionRunner implements PythonFunctionRunner { /** The fraction of total managed memory in the slot that the Python worker could use. */ private final double managedMemoryFraction; - protected final FlinkFnApi.CoderParam.DataType inputDataType; - protected final FlinkFnApi.CoderParam.DataType outputDataType; - - protected final FlinkFnApi.CoderParam.OutputMode outputMode; + protected final FlinkFnApi.CoderInfoDescriptor inputCoderDescriptor; + protected final FlinkFnApi.CoderInfoDescriptor outputCoderDescriptor; // ------------------------------------------------------------------------ @@ -182,9 +181,8 @@ public BeamPythonFunctionRunner( @Nullable TypeSerializer namespaceSerializer, MemoryManager memoryManager, double managedMemoryFraction, - FlinkFnApi.CoderParam.DataType inputDataType, - FlinkFnApi.CoderParam.DataType outputDataType, - FlinkFnApi.CoderParam.OutputMode outputMode) { + FlinkFnApi.CoderInfoDescriptor inputCoderDescriptor, + FlinkFnApi.CoderInfoDescriptor outputCoderDescriptor) { this.taskName = Preconditions.checkNotNull(taskName); this.environmentManager = Preconditions.checkNotNull(environmentManager); this.functionUrn = Preconditions.checkNotNull(functionUrn); @@ -195,9 +193,8 @@ public BeamPythonFunctionRunner( keyedStateBackend, keySerializer, namespaceSerializer, jobOptions); this.memoryManager = memoryManager; this.managedMemoryFraction = managedMemoryFraction; - this.inputDataType = Preconditions.checkNotNull(inputDataType); - this.outputDataType = Preconditions.checkNotNull(outputDataType); - this.outputMode = Preconditions.checkNotNull(outputMode); + this.inputCoderDescriptor = Preconditions.checkNotNull(inputCoderDescriptor); + this.outputCoderDescriptor = Preconditions.checkNotNull(outputCoderDescriptor); } // ------------------------------------------------------------------------ @@ -417,8 +414,8 @@ private ExecutableStage createExecutableStage(RunnerApi.Environment environment) RunnerApi.WindowingStrategy.newBuilder() .setWindowCoderId(WINDOW_CODER_ID) .build()) - .putCoders(INPUT_CODER_ID, getInputCoderProto()) - .putCoders(OUTPUT_CODER_ID, getOutputCoderProto()) + .putCoders(INPUT_CODER_ID, createCoderProto(inputCoderDescriptor)) + .putCoders(OUTPUT_CODER_ID, createCoderProto(outputCoderDescriptor)) .putCoders(WINDOW_CODER_ID, getWindowCoderProto()) .build(); @@ -485,10 +482,6 @@ private RunnerApi.Coder getWindowCoderProto() { protected abstract byte[] getUserDefinedFunctionsProtoBytes(); - protected abstract RunnerApi.Coder getInputCoderProto(); - - protected abstract RunnerApi.Coder getOutputCoderProto(); - // ------------------------------------------------------------------------ // Construct RemoteBundler // ------------------------------------------------------------------------ diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/utils/ProtoUtils.java b/flink-python/src/main/java/org/apache/flink/streaming/api/utils/ProtoUtils.java new file mode 100644 index 0000000000000..fbdab2b89f1f5 --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/utils/ProtoUtils.java @@ -0,0 +1,309 @@ +/* + * 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.utils; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.fnexecution.v1.FlinkFnApi; +import org.apache.flink.streaming.api.functions.python.DataStreamPythonFunctionInfo; +import org.apache.flink.table.functions.python.PythonAggregateFunctionInfo; +import org.apache.flink.table.functions.python.PythonFunctionInfo; +import org.apache.flink.table.functions.python.PythonFunctionKind; +import org.apache.flink.table.planner.typeutils.DataViewUtils; +import org.apache.flink.table.types.logical.RowType; + +import com.google.protobuf.ByteString; +import org.apache.beam.model.pipeline.v1.RunnerApi; + +import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.flink.python.Constants.FLINK_CODER_URN; +import static org.apache.flink.table.runtime.typeutils.PythonTypeUtils.toProtoType; + +/** Utilities used to construct protobuf objects. */ +@Internal +public enum ProtoUtils { + ; + + public static FlinkFnApi.UserDefinedFunction getUserDefinedFunctionProto( + PythonFunctionInfo pythonFunctionInfo) { + FlinkFnApi.UserDefinedFunction.Builder builder = + FlinkFnApi.UserDefinedFunction.newBuilder(); + builder.setPayload( + ByteString.copyFrom( + pythonFunctionInfo.getPythonFunction().getSerializedPythonFunction())); + for (Object input : pythonFunctionInfo.getInputs()) { + FlinkFnApi.Input.Builder inputProto = FlinkFnApi.Input.newBuilder(); + if (input instanceof PythonFunctionInfo) { + inputProto.setUdf(getUserDefinedFunctionProto((PythonFunctionInfo) input)); + } else if (input instanceof Integer) { + inputProto.setInputOffset((Integer) input); + } else { + inputProto.setInputConstant(ByteString.copyFrom((byte[]) input)); + } + builder.addInputs(inputProto); + } + builder.setTakesRowAsInput(pythonFunctionInfo.getPythonFunction().takesRowAsInput()); + builder.setIsPandasUdf( + pythonFunctionInfo.getPythonFunction().getPythonFunctionKind() + == PythonFunctionKind.PANDAS); + return builder.build(); + } + + public static FlinkFnApi.UserDefinedAggregateFunction getUserDefinedAggregateFunctionProto( + PythonAggregateFunctionInfo pythonFunctionInfo, + DataViewUtils.DataViewSpec[] dataViewSpecs) { + FlinkFnApi.UserDefinedAggregateFunction.Builder builder = + FlinkFnApi.UserDefinedAggregateFunction.newBuilder(); + builder.setPayload( + ByteString.copyFrom( + pythonFunctionInfo.getPythonFunction().getSerializedPythonFunction())); + builder.setDistinct(pythonFunctionInfo.isDistinct()); + builder.setFilterArg(pythonFunctionInfo.getFilterArg()); + builder.setTakesRowAsInput(pythonFunctionInfo.getPythonFunction().takesRowAsInput()); + for (Object input : pythonFunctionInfo.getInputs()) { + FlinkFnApi.Input.Builder inputProto = FlinkFnApi.Input.newBuilder(); + if (input instanceof Integer) { + inputProto.setInputOffset((Integer) input); + } else { + inputProto.setInputConstant(ByteString.copyFrom((byte[]) input)); + } + builder.addInputs(inputProto); + } + if (dataViewSpecs != null) { + for (DataViewUtils.DataViewSpec spec : dataViewSpecs) { + FlinkFnApi.UserDefinedAggregateFunction.DataViewSpec.Builder specBuilder = + FlinkFnApi.UserDefinedAggregateFunction.DataViewSpec.newBuilder(); + specBuilder.setName(spec.getStateId()); + if (spec instanceof DataViewUtils.ListViewSpec) { + DataViewUtils.ListViewSpec listViewSpec = (DataViewUtils.ListViewSpec) spec; + specBuilder.setListView( + FlinkFnApi.UserDefinedAggregateFunction.DataViewSpec.ListView + .newBuilder() + .setElementType( + toProtoType( + listViewSpec + .getElementDataType() + .getLogicalType()))); + } else { + DataViewUtils.MapViewSpec mapViewSpec = (DataViewUtils.MapViewSpec) spec; + FlinkFnApi.UserDefinedAggregateFunction.DataViewSpec.MapView.Builder + mapViewBuilder = + FlinkFnApi.UserDefinedAggregateFunction.DataViewSpec.MapView + .newBuilder(); + mapViewBuilder.setKeyType( + toProtoType(mapViewSpec.getKeyDataType().getLogicalType())); + mapViewBuilder.setValueType( + toProtoType(mapViewSpec.getValueDataType().getLogicalType())); + specBuilder.setMapView(mapViewBuilder.build()); + } + specBuilder.setFieldIndex(spec.getFieldIndex()); + builder.addSpecs(specBuilder.build()); + } + } + return builder.build(); + } + + public static FlinkFnApi.UserDefinedDataStreamFunction getUserDefinedDataStreamFunctionProto( + DataStreamPythonFunctionInfo dataStreamPythonFunctionInfo, + RuntimeContext runtimeContext, + Map internalParameters, + boolean inBatchExecutionMode) { + FlinkFnApi.UserDefinedDataStreamFunction.Builder builder = + FlinkFnApi.UserDefinedDataStreamFunction.newBuilder(); + builder.setFunctionType( + FlinkFnApi.UserDefinedDataStreamFunction.FunctionType.forNumber( + dataStreamPythonFunctionInfo.getFunctionType())); + builder.setRuntimeContext( + FlinkFnApi.UserDefinedDataStreamFunction.RuntimeContext.newBuilder() + .setTaskName(runtimeContext.getTaskName()) + .setTaskNameWithSubtasks(runtimeContext.getTaskNameWithSubtasks()) + .setNumberOfParallelSubtasks(runtimeContext.getNumberOfParallelSubtasks()) + .setMaxNumberOfParallelSubtasks( + runtimeContext.getMaxNumberOfParallelSubtasks()) + .setIndexOfThisSubtask(runtimeContext.getIndexOfThisSubtask()) + .setAttemptNumber(runtimeContext.getAttemptNumber()) + .addAllJobParameters( + runtimeContext.getExecutionConfig().getGlobalJobParameters().toMap() + .entrySet().stream() + .map( + entry -> + FlinkFnApi.UserDefinedDataStreamFunction + .JobParameter.newBuilder() + .setKey(entry.getKey()) + .setValue(entry.getValue()) + .build()) + .collect(Collectors.toList())) + .addAllJobParameters( + internalParameters.entrySet().stream() + .map( + entry -> + FlinkFnApi.UserDefinedDataStreamFunction + .JobParameter.newBuilder() + .setKey(entry.getKey()) + .setValue(entry.getValue()) + .build()) + .collect(Collectors.toList())) + .setInBatchExecutionMode(inBatchExecutionMode) + .build()); + builder.setPayload( + ByteString.copyFrom( + dataStreamPythonFunctionInfo + .getPythonFunction() + .getSerializedPythonFunction())); + builder.setMetricEnabled(true); + return builder.build(); + } + + public static FlinkFnApi.UserDefinedDataStreamFunction + getUserDefinedDataStreamStatefulFunctionProto( + DataStreamPythonFunctionInfo dataStreamPythonFunctionInfo, + RuntimeContext runtimeContext, + Map internalParameters, + TypeInformation keyTypeInfo, + boolean inBatchExecutionMode) { + FlinkFnApi.UserDefinedDataStreamFunction userDefinedDataStreamFunction = + getUserDefinedDataStreamFunctionProto( + dataStreamPythonFunctionInfo, + runtimeContext, + internalParameters, + inBatchExecutionMode); + FlinkFnApi.TypeInfo builtKeyTypeInfo = + PythonTypeUtils.TypeInfoToProtoConverter.toTypeInfoProto(keyTypeInfo); + return userDefinedDataStreamFunction.toBuilder().setKeyTypeInfo(builtKeyTypeInfo).build(); + } + + public static RunnerApi.Coder createCoderProto( + FlinkFnApi.CoderInfoDescriptor coderInfoDescriptor) { + return RunnerApi.Coder.newBuilder() + .setSpec( + RunnerApi.FunctionSpec.newBuilder() + .setUrn(FLINK_CODER_URN) + .setPayload( + org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf + .ByteString.copyFrom( + coderInfoDescriptor.toByteArray())) + .build()) + .build(); + } + + public static FlinkFnApi.CoderInfoDescriptor createFlattenRowTypeCoderInfoDescriptorProto( + RowType rowType, + FlinkFnApi.CoderInfoDescriptor.Mode mode, + boolean separatedWithEndMessage) { + FlinkFnApi.CoderInfoDescriptor.FlattenRowType flattenRowType = + FlinkFnApi.CoderInfoDescriptor.FlattenRowType.newBuilder() + .setSchema(toProtoType(rowType).getRowSchema()) + .build(); + return createCoderInfoDescriptorProto( + flattenRowType, null, null, null, null, mode, separatedWithEndMessage); + } + + public static FlinkFnApi.CoderInfoDescriptor createRowTypeCoderInfoDescriptorProto( + RowType rowType, + FlinkFnApi.CoderInfoDescriptor.Mode mode, + boolean separatedWithEndMessage) { + return createCoderInfoDescriptorProto( + null, + FlinkFnApi.CoderInfoDescriptor.RowType.newBuilder() + .setSchema(toProtoType(rowType).getRowSchema()) + .build(), + null, + null, + null, + mode, + separatedWithEndMessage); + } + + public static FlinkFnApi.CoderInfoDescriptor createArrowTypeCoderInfoDescriptorProto( + RowType rowType, + FlinkFnApi.CoderInfoDescriptor.Mode mode, + boolean separatedWithEndMessage) { + return createCoderInfoDescriptorProto( + null, + null, + FlinkFnApi.CoderInfoDescriptor.ArrowType.newBuilder() + .setSchema(toProtoType(rowType).getRowSchema()) + .build(), + null, + null, + mode, + separatedWithEndMessage); + } + + public static FlinkFnApi.CoderInfoDescriptor createOverWindowArrowTypeCoderInfoDescriptorProto( + RowType rowType, + FlinkFnApi.CoderInfoDescriptor.Mode mode, + boolean separatedWithEndMessage) { + return createCoderInfoDescriptorProto( + null, + null, + null, + FlinkFnApi.CoderInfoDescriptor.OverWindowArrowType.newBuilder() + .setSchema(toProtoType(rowType).getRowSchema()) + .build(), + null, + mode, + separatedWithEndMessage); + } + + public static FlinkFnApi.CoderInfoDescriptor createRawTypeCoderInfoDescriptorProto( + TypeInformation typeInformation, + FlinkFnApi.CoderInfoDescriptor.Mode mode, + boolean separatedWithEndMessage) { + FlinkFnApi.TypeInfo typeinfo = + PythonTypeUtils.TypeInfoToProtoConverter.toTypeInfoProto(typeInformation); + return createCoderInfoDescriptorProto( + null, + null, + null, + null, + FlinkFnApi.CoderInfoDescriptor.RawType.newBuilder().setTypeInfo(typeinfo).build(), + mode, + separatedWithEndMessage); + } + + private static FlinkFnApi.CoderInfoDescriptor createCoderInfoDescriptorProto( + FlinkFnApi.CoderInfoDescriptor.FlattenRowType flattenRowType, + FlinkFnApi.CoderInfoDescriptor.RowType rowType, + FlinkFnApi.CoderInfoDescriptor.ArrowType arrowType, + FlinkFnApi.CoderInfoDescriptor.OverWindowArrowType overWindowArrowType, + FlinkFnApi.CoderInfoDescriptor.RawType rawType, + FlinkFnApi.CoderInfoDescriptor.Mode mode, + boolean separatedWithEndMessage) { + FlinkFnApi.CoderInfoDescriptor.Builder builder = + FlinkFnApi.CoderInfoDescriptor.newBuilder(); + if (flattenRowType != null) { + builder.setFlattenRowType(flattenRowType); + } else if (rowType != null) { + builder.setRowType(rowType); + } else if (arrowType != null) { + builder.setArrowType(arrowType); + } else if (overWindowArrowType != null) { + builder.setOverWindowArrowType(overWindowArrowType); + } else if (rawType != null) { + builder.setRawType(rawType); + } + builder.setMode(mode); + builder.setSeparatedWithEndMessage(separatedWithEndMessage); + return builder.build(); + } +} diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/utils/PythonOperatorUtils.java b/flink-python/src/main/java/org/apache/flink/streaming/api/utils/PythonOperatorUtils.java index 76a8a45f2c234..f30ca93b30351 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/utils/PythonOperatorUtils.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/utils/PythonOperatorUtils.java @@ -18,181 +18,20 @@ package org.apache.flink.streaming.api.utils; -import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.fnexecution.v1.FlinkFnApi; +import org.apache.flink.annotation.Internal; import org.apache.flink.runtime.state.KeyedStateBackend; -import org.apache.flink.streaming.api.functions.python.DataStreamPythonFunctionInfo; import org.apache.flink.streaming.api.operators.sorted.state.BatchExecutionKeyedStateBackend; -import org.apache.flink.table.functions.python.PythonAggregateFunctionInfo; -import org.apache.flink.table.functions.python.PythonFunctionInfo; -import org.apache.flink.table.functions.python.PythonFunctionKind; -import org.apache.flink.table.planner.typeutils.DataViewUtils; - -import com.google.protobuf.ByteString; import java.util.Arrays; -import java.util.Map; -import java.util.stream.Collectors; - -import static org.apache.flink.table.runtime.typeutils.PythonTypeUtils.toProtoType; -/** The collectors used to collect Row values. */ -public enum PythonOperatorUtils { - ; +/** Utilities used by Python operators. */ +@Internal +public class PythonOperatorUtils { - private static final byte[] RECORD_SPLITER = new byte[] {0x00}; - - public static FlinkFnApi.UserDefinedFunction getUserDefinedFunctionProto( - PythonFunctionInfo pythonFunctionInfo) { - FlinkFnApi.UserDefinedFunction.Builder builder = - FlinkFnApi.UserDefinedFunction.newBuilder(); - builder.setPayload( - ByteString.copyFrom( - pythonFunctionInfo.getPythonFunction().getSerializedPythonFunction())); - for (Object input : pythonFunctionInfo.getInputs()) { - FlinkFnApi.Input.Builder inputProto = FlinkFnApi.Input.newBuilder(); - if (input instanceof PythonFunctionInfo) { - inputProto.setUdf(getUserDefinedFunctionProto((PythonFunctionInfo) input)); - } else if (input instanceof Integer) { - inputProto.setInputOffset((Integer) input); - } else { - inputProto.setInputConstant(ByteString.copyFrom((byte[]) input)); - } - builder.addInputs(inputProto); - } - builder.setTakesRowAsInput(pythonFunctionInfo.getPythonFunction().takesRowAsInput()); - builder.setIsPandasUdf( - pythonFunctionInfo.getPythonFunction().getPythonFunctionKind() - == PythonFunctionKind.PANDAS); - return builder.build(); - } - - public static FlinkFnApi.UserDefinedAggregateFunction getUserDefinedAggregateFunctionProto( - PythonAggregateFunctionInfo pythonFunctionInfo, - DataViewUtils.DataViewSpec[] dataViewSpecs) { - FlinkFnApi.UserDefinedAggregateFunction.Builder builder = - FlinkFnApi.UserDefinedAggregateFunction.newBuilder(); - builder.setPayload( - ByteString.copyFrom( - pythonFunctionInfo.getPythonFunction().getSerializedPythonFunction())); - builder.setDistinct(pythonFunctionInfo.isDistinct()); - builder.setFilterArg(pythonFunctionInfo.getFilterArg()); - builder.setTakesRowAsInput(pythonFunctionInfo.getPythonFunction().takesRowAsInput()); - for (Object input : pythonFunctionInfo.getInputs()) { - FlinkFnApi.Input.Builder inputProto = FlinkFnApi.Input.newBuilder(); - if (input instanceof Integer) { - inputProto.setInputOffset((Integer) input); - } else { - inputProto.setInputConstant(ByteString.copyFrom((byte[]) input)); - } - builder.addInputs(inputProto); - } - if (dataViewSpecs != null) { - for (DataViewUtils.DataViewSpec spec : dataViewSpecs) { - FlinkFnApi.UserDefinedAggregateFunction.DataViewSpec.Builder specBuilder = - FlinkFnApi.UserDefinedAggregateFunction.DataViewSpec.newBuilder(); - specBuilder.setName(spec.getStateId()); - if (spec instanceof DataViewUtils.ListViewSpec) { - DataViewUtils.ListViewSpec listViewSpec = (DataViewUtils.ListViewSpec) spec; - specBuilder.setListView( - FlinkFnApi.UserDefinedAggregateFunction.DataViewSpec.ListView - .newBuilder() - .setElementType( - toProtoType( - listViewSpec - .getElementDataType() - .getLogicalType()))); - } else { - DataViewUtils.MapViewSpec mapViewSpec = (DataViewUtils.MapViewSpec) spec; - FlinkFnApi.UserDefinedAggregateFunction.DataViewSpec.MapView.Builder - mapViewBuilder = - FlinkFnApi.UserDefinedAggregateFunction.DataViewSpec.MapView - .newBuilder(); - mapViewBuilder.setKeyType( - toProtoType(mapViewSpec.getKeyDataType().getLogicalType())); - mapViewBuilder.setValueType( - toProtoType(mapViewSpec.getValueDataType().getLogicalType())); - specBuilder.setMapView(mapViewBuilder.build()); - } - specBuilder.setFieldIndex(spec.getFieldIndex()); - builder.addSpecs(specBuilder.build()); - } - } - return builder.build(); - } - - public static FlinkFnApi.UserDefinedDataStreamFunction getUserDefinedDataStreamFunctionProto( - DataStreamPythonFunctionInfo dataStreamPythonFunctionInfo, - RuntimeContext runtimeContext, - Map internalParameters, - boolean inBatchExecutionMode) { - FlinkFnApi.UserDefinedDataStreamFunction.Builder builder = - FlinkFnApi.UserDefinedDataStreamFunction.newBuilder(); - builder.setFunctionType( - FlinkFnApi.UserDefinedDataStreamFunction.FunctionType.forNumber( - dataStreamPythonFunctionInfo.getFunctionType())); - builder.setRuntimeContext( - FlinkFnApi.UserDefinedDataStreamFunction.RuntimeContext.newBuilder() - .setTaskName(runtimeContext.getTaskName()) - .setTaskNameWithSubtasks(runtimeContext.getTaskNameWithSubtasks()) - .setNumberOfParallelSubtasks(runtimeContext.getNumberOfParallelSubtasks()) - .setMaxNumberOfParallelSubtasks( - runtimeContext.getMaxNumberOfParallelSubtasks()) - .setIndexOfThisSubtask(runtimeContext.getIndexOfThisSubtask()) - .setAttemptNumber(runtimeContext.getAttemptNumber()) - .addAllJobParameters( - runtimeContext.getExecutionConfig().getGlobalJobParameters().toMap() - .entrySet().stream() - .map( - entry -> - FlinkFnApi.UserDefinedDataStreamFunction - .JobParameter.newBuilder() - .setKey(entry.getKey()) - .setValue(entry.getValue()) - .build()) - .collect(Collectors.toList())) - .addAllJobParameters( - internalParameters.entrySet().stream() - .map( - entry -> - FlinkFnApi.UserDefinedDataStreamFunction - .JobParameter.newBuilder() - .setKey(entry.getKey()) - .setValue(entry.getValue()) - .build()) - .collect(Collectors.toList())) - .setInBatchExecutionMode(inBatchExecutionMode) - .build()); - builder.setPayload( - ByteString.copyFrom( - dataStreamPythonFunctionInfo - .getPythonFunction() - .getSerializedPythonFunction())); - builder.setMetricEnabled(true); - return builder.build(); - } - - public static FlinkFnApi.UserDefinedDataStreamFunction - getUserDefinedDataStreamStatefulFunctionProto( - DataStreamPythonFunctionInfo dataStreamPythonFunctionInfo, - RuntimeContext runtimeContext, - Map internalParameters, - TypeInformation keyTypeInfo, - boolean inBatchExecutionMode) { - FlinkFnApi.UserDefinedDataStreamFunction userDefinedDataStreamFunction = - getUserDefinedDataStreamFunctionProto( - dataStreamPythonFunctionInfo, - runtimeContext, - internalParameters, - inBatchExecutionMode); - FlinkFnApi.TypeInfo builtKeyTypeInfo = - PythonTypeUtils.TypeInfoToProtoConverter.toTypeInfoProto(keyTypeInfo); - return userDefinedDataStreamFunction.toBuilder().setKeyTypeInfo(builtKeyTypeInfo).build(); - } + private static final byte[] RECORD_SPLITTER = new byte[] {0x00}; public static boolean endOfLastFlatMap(int length, byte[] rawData) { - return length == 1 && Arrays.equals(rawData, RECORD_SPLITER); + return length == 1 && Arrays.equals(rawData, RECORD_SPLITTER); } /** Set the current key for streaming operator. */ diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/AbstractStatelessFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/AbstractStatelessFunctionOperator.java index 21dd9356a8a6f..e16cd06ab2eaf 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/AbstractStatelessFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/AbstractStatelessFunctionOperator.java @@ -65,15 +65,6 @@ public abstract class AbstractStatelessFunctionOperator /** The options used to configure the Python worker process. */ private final Map jobOptions; - /** The Input DataType of BaseCoder in Python. */ - private final FlinkFnApi.CoderParam.DataType inputDataType; - - /** The output DataType of BaseCoder in Python. */ - private final FlinkFnApi.CoderParam.DataType outputDataType; - - /** The output mode of BaseCoder in Python. */ - private final FlinkFnApi.CoderParam.OutputMode outputMode; - /** The user-defined function input logical type. */ protected transient RowType userDefinedFunctionInputType; @@ -101,19 +92,13 @@ public AbstractStatelessFunctionOperator( Configuration config, RowType inputType, RowType outputType, - int[] userDefinedFunctionInputOffsets, - FlinkFnApi.CoderParam.DataType inputDataType, - FlinkFnApi.CoderParam.DataType outputDataType, - FlinkFnApi.CoderParam.OutputMode outputMode) { + int[] userDefinedFunctionInputOffsets) { super(config); this.inputType = Preconditions.checkNotNull(inputType); this.outputType = Preconditions.checkNotNull(outputType); this.userDefinedFunctionInputOffsets = Preconditions.checkNotNull(userDefinedFunctionInputOffsets); this.jobOptions = buildJobOptions(config); - this.inputDataType = Preconditions.checkNotNull(inputDataType); - this.outputDataType = Preconditions.checkNotNull(outputDataType); - this.outputMode = Preconditions.checkNotNull(outputMode); } @Override @@ -146,8 +131,6 @@ public PythonFunctionRunner createPythonFunctionRunner() throws IOException { return new BeamTablePythonFunctionRunner( getRuntimeContext().getTaskName(), createPythonEnvironmentManager(), - userDefinedFunctionInputType, - userDefinedFunctionOutputType, getFunctionUrn(), getUserDefinedFunctionsProto(), jobOptions, @@ -167,9 +150,8 @@ public PythonFunctionRunner createPythonFunctionRunner() throws IOException { .getEnvironment() .getUserCodeClassLoader() .asClassLoader()), - inputDataType, - outputDataType, - outputMode); + createInputCoderInfoDescriptor(userDefinedFunctionInputType), + createOutputCoderInfoDescriptor(userDefinedFunctionOutputType)); } /** @@ -185,6 +167,12 @@ public PythonFunctionRunner createPythonFunctionRunner() throws IOException { public abstract String getFunctionUrn(); + public abstract FlinkFnApi.CoderInfoDescriptor createInputCoderInfoDescriptor( + RowType runnerInputType); + + public abstract FlinkFnApi.CoderInfoDescriptor createOutputCoderInfoDescriptor( + RowType runnerOutType); + public abstract void processElementInternal(IN value) throws Exception; private Map buildJobOptions(Configuration config) { diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/AbstractPythonStreamAggregateOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/AbstractPythonStreamAggregateOperator.java index ebe9d45f0522d..745c2f8b5b267 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/AbstractPythonStreamAggregateOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/AbstractPythonStreamAggregateOperator.java @@ -31,7 +31,7 @@ import org.apache.flink.python.PythonFunctionRunner; import org.apache.flink.python.PythonOptions; import org.apache.flink.streaming.api.operators.python.AbstractOneInputPythonFunctionOperator; -import org.apache.flink.streaming.api.utils.PythonOperatorUtils; +import org.apache.flink.streaming.api.utils.ProtoUtils; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.table.data.RowData; import org.apache.flink.table.functions.python.PythonAggregateFunctionInfo; @@ -51,6 +51,7 @@ import java.util.Map; import java.util.stream.Collectors; +import static org.apache.flink.streaming.api.utils.ProtoUtils.createRowTypeCoderInfoDescriptorProto; import static org.apache.flink.table.runtime.typeutils.PythonTypeUtils.toProtoType; /** @@ -97,15 +98,6 @@ public abstract class AbstractPythonStreamAggregateOperator private final int mapStateWriteCacheSize; - /** The Input DataType of BaseCoder in Python. */ - private final FlinkFnApi.CoderParam.DataType inputDataType; - - /** The output DataType of BaseCoder in Python. */ - private final FlinkFnApi.CoderParam.DataType outputDataType; - - /** The output mode of BaseCoder in Python. */ - private final FlinkFnApi.CoderParam.OutputMode outputMode; - private transient Object keyForTimerService; /** The user-defined function input logical type. */ @@ -143,10 +135,7 @@ public AbstractPythonStreamAggregateOperator( DataViewUtils.DataViewSpec[][] dataViewSpecs, int[] grouping, int indexOfCountStar, - boolean generateUpdateBefore, - FlinkFnApi.CoderParam.DataType inputDataType, - FlinkFnApi.CoderParam.DataType outputDataType, - FlinkFnApi.CoderParam.OutputMode outputMode) { + boolean generateUpdateBefore) { super(config); this.inputType = Preconditions.checkNotNull(inputType); this.outputType = Preconditions.checkNotNull(outputType); @@ -156,9 +145,6 @@ public AbstractPythonStreamAggregateOperator( this.grouping = grouping; this.indexOfCountStar = indexOfCountStar; this.generateUpdateBefore = generateUpdateBefore; - this.inputDataType = Preconditions.checkNotNull(inputDataType); - this.outputDataType = Preconditions.checkNotNull(outputDataType); - this.outputMode = Preconditions.checkNotNull(outputMode); this.stateCacheSize = config.get(PythonOptions.STATE_CACHE_SIZE); this.mapStateReadCacheSize = config.get(PythonOptions.MAP_STATE_READ_CACHE_SIZE); this.mapStateWriteCacheSize = config.get(PythonOptions.MAP_STATE_WRITE_CACHE_SIZE); @@ -194,8 +180,6 @@ public PythonFunctionRunner createPythonFunctionRunner() throws Exception { return new BeamTablePythonFunctionRunner( getRuntimeContext().getTaskName(), createPythonEnvironmentManager(), - userDefinedFunctionInputType, - userDefinedFunctionOutputType, getFunctionUrn(), getUserDefinedFunctionsProto(), jobOptions, @@ -215,9 +199,8 @@ public PythonFunctionRunner createPythonFunctionRunner() throws Exception { .getEnvironment() .getUserCodeClassLoader() .asClassLoader()), - inputDataType, - outputDataType, - outputMode); + createInputCoderInfoDescriptor(userDefinedFunctionInputType), + createOutputCoderInfoDescriptor(userDefinedFunctionOutputType)); } /** @@ -275,8 +258,7 @@ protected FlinkFnApi.UserDefinedAggregateFunctions getUserDefinedFunctionsProto( specs = dataViewSpecs[i]; } builder.addUdfs( - PythonOperatorUtils.getUserDefinedAggregateFunctionProto( - aggregateFunctions[i], specs)); + ProtoUtils.getUserDefinedAggregateFunctionProto(aggregateFunctions[i], specs)); } return builder.build(); } @@ -302,4 +284,14 @@ private Map buildJobOptions(Configuration config) { String.valueOf(config.get(PythonOptions.MAP_STATE_ITERATE_RESPONSE_BATCH_SIZE))); return jobOptions; } + + public FlinkFnApi.CoderInfoDescriptor createInputCoderInfoDescriptor(RowType runnerInputType) { + return createRowTypeCoderInfoDescriptorProto( + runnerInputType, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, false); + } + + public FlinkFnApi.CoderInfoDescriptor createOutputCoderInfoDescriptor(RowType runnerOutType) { + return createRowTypeCoderInfoDescriptorProto( + runnerOutType, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, false); + } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/AbstractPythonStreamGroupAggregateOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/AbstractPythonStreamGroupAggregateOperator.java index 4d8b665fcd00b..d5f228bbcc752 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/AbstractPythonStreamGroupAggregateOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/AbstractPythonStreamGroupAggregateOperator.java @@ -94,10 +94,7 @@ public AbstractPythonStreamGroupAggregateOperator( dataViewSpecs, grouping, indexOfCountStar, - generateUpdateBefore, - FlinkFnApi.CoderParam.DataType.ROW, - FlinkFnApi.CoderParam.DataType.ROW, - FlinkFnApi.CoderParam.OutputMode.MULTIPLE); + generateUpdateBefore); this.minRetentionTime = minRetentionTime; this.maxRetentionTime = maxRetentionTime; this.stateCleaningEnabled = minRetentionTime > 1; diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupWindowAggregateOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupWindowAggregateOperator.java index ff9ad1d8c9a1a..861941c3b703f 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupWindowAggregateOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupWindowAggregateOperator.java @@ -62,6 +62,7 @@ import static org.apache.flink.fnexecution.v1.FlinkFnApi.GroupWindow.WindowProperty.WINDOW_END; import static org.apache.flink.fnexecution.v1.FlinkFnApi.GroupWindow.WindowProperty.WINDOW_START; +import static org.apache.flink.streaming.api.utils.ProtoUtils.createFlattenRowTypeCoderInfoDescriptorProto; import static org.apache.flink.table.runtime.util.TimeWindowUtil.toEpochMillsForTimer; import static org.apache.flink.table.runtime.util.TimeWindowUtil.toUtcTimestampMills; @@ -170,10 +171,7 @@ public PythonStreamGroupWindowAggregateOperator( dataViewSpecs, grouping, indexOfCountStar, - generateUpdateBefore, - FlinkFnApi.CoderParam.DataType.FLATTEN_ROW, - FlinkFnApi.CoderParam.DataType.FLATTEN_ROW, - FlinkFnApi.CoderParam.OutputMode.MULTIPLE); + generateUpdateBefore); this.countStarInserted = countStarInserted; this.inputTimeFieldIndex = inputTimeFieldIndex; this.windowAssigner = windowAssigner; @@ -349,6 +347,18 @@ public void onProcessingTime(InternalTimer timer) throws Exception { emitTriggerTimerData(timer, REGISTER_PROCESSING_TIMER); } + @Override + public FlinkFnApi.CoderInfoDescriptor createInputCoderInfoDescriptor(RowType runnerInputType) { + return createFlattenRowTypeCoderInfoDescriptorProto( + runnerInputType, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, false); + } + + @Override + public FlinkFnApi.CoderInfoDescriptor createOutputCoderInfoDescriptor(RowType runnerOutType) { + return createFlattenRowTypeCoderInfoDescriptorProto( + runnerOutType, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, false); + } + @VisibleForTesting long getShiftEpochMills(long utcTimestampMills) { return TimeWindowUtil.toEpochMills(utcTimestampMills, shiftTimeZone); diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/AbstractArrowPythonAggregateFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/AbstractArrowPythonAggregateFunctionOperator.java index a7c63cf6afa46..801e1dc9f7fb5 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/AbstractArrowPythonAggregateFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/AbstractArrowPythonAggregateFunctionOperator.java @@ -40,7 +40,8 @@ import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.Preconditions; -import static org.apache.flink.streaming.api.utils.PythonOperatorUtils.getUserDefinedFunctionProto; +import static org.apache.flink.streaming.api.utils.ProtoUtils.createArrowTypeCoderInfoDescriptorProto; +import static org.apache.flink.streaming.api.utils.ProtoUtils.getUserDefinedFunctionProto; /** The Abstract class of Arrow Aggregate Operator for Pandas {@link AggregateFunction}. */ @Internal @@ -77,17 +78,8 @@ public AbstractArrowPythonAggregateFunctionOperator( RowType inputType, RowType outputType, int[] groupingSet, - int[] udafInputOffsets, - FlinkFnApi.CoderParam.DataType inputDataType, - FlinkFnApi.CoderParam.DataType outputDataType) { - super( - config, - inputType, - outputType, - udafInputOffsets, - inputDataType, - outputDataType, - FlinkFnApi.CoderParam.OutputMode.SINGLE); + int[] udafInputOffsets) { + super(config, inputType, outputType, udafInputOffsets); this.pandasAggFunctions = Preconditions.checkNotNull(pandasAggFunctions); this.groupingSet = Preconditions.checkNotNull(groupingSet); } @@ -138,6 +130,18 @@ public String getFunctionUrn() { return PANDAS_AGGREGATE_FUNCTION_URN; } + @Override + public FlinkFnApi.CoderInfoDescriptor createInputCoderInfoDescriptor(RowType runnerInputType) { + return createArrowTypeCoderInfoDescriptorProto( + runnerInputType, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, false); + } + + @Override + public FlinkFnApi.CoderInfoDescriptor createOutputCoderInfoDescriptor(RowType runnerOutType) { + return createArrowTypeCoderInfoDescriptorProto( + runnerOutType, FlinkFnApi.CoderInfoDescriptor.Mode.SINGLE, false); + } + @Override public RowData getFunctionInput(RowData element) { return udafInputProjection.apply(element); diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/AbstractBatchArrowPythonAggregateFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/AbstractBatchArrowPythonAggregateFunctionOperator.java index 4442cbd5a6cd1..d717667a4f821 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/AbstractBatchArrowPythonAggregateFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/AbstractBatchArrowPythonAggregateFunctionOperator.java @@ -20,7 +20,6 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.configuration.Configuration; -import org.apache.flink.fnexecution.v1.FlinkFnApi; import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.binary.BinaryRowData; @@ -69,18 +68,8 @@ abstract class AbstractBatchArrowPythonAggregateFunctionOperator RowType outputType, int[] groupKey, int[] groupingSet, - int[] udafInputOffsets, - FlinkFnApi.CoderParam.DataType inputDataType, - FlinkFnApi.CoderParam.DataType outputDataType) { - super( - config, - pandasAggFunctions, - inputType, - outputType, - groupingSet, - udafInputOffsets, - inputDataType, - outputDataType); + int[] udafInputOffsets) { + super(config, pandasAggFunctions, inputType, outputType, groupingSet, udafInputOffsets); this.groupKey = Preconditions.checkNotNull(groupKey); } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonGroupAggregateFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonGroupAggregateFunctionOperator.java index 6cc784afe2462..d003c011afce8 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonGroupAggregateFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonGroupAggregateFunctionOperator.java @@ -21,7 +21,6 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; -import org.apache.flink.fnexecution.v1.FlinkFnApi; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.binary.BinaryRowData; import org.apache.flink.table.functions.AggregateFunction; @@ -50,9 +49,7 @@ public BatchArrowPythonGroupAggregateFunctionOperator( outputType, groupKey, groupingSet, - udafInputOffsets, - FlinkFnApi.CoderParam.DataType.ARROW, - FlinkFnApi.CoderParam.DataType.ARROW); + udafInputOffsets); } @Override diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonGroupWindowAggregateFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonGroupWindowAggregateFunctionOperator.java index a1939e44510b4..3a6d18831a5dd 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonGroupWindowAggregateFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonGroupWindowAggregateFunctionOperator.java @@ -21,7 +21,6 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; -import org.apache.flink.fnexecution.v1.FlinkFnApi; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.TimestampData; @@ -103,9 +102,7 @@ public BatchArrowPythonGroupWindowAggregateFunctionOperator( outputType, groupKey, groupingSet, - udafInputOffsets, - FlinkFnApi.CoderParam.DataType.ARROW, - FlinkFnApi.CoderParam.DataType.ARROW); + udafInputOffsets); this.namedProperties = namedProperties; this.inputTimeFieldIndex = inputTimeFieldIndex; this.maxLimitSize = maxLimitSize; diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonOverWindowAggregateFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonOverWindowAggregateFunctionOperator.java index acc94a2f19c8f..959943e563d1c 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonOverWindowAggregateFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonOverWindowAggregateFunctionOperator.java @@ -36,7 +36,8 @@ import java.util.List; import java.util.ListIterator; -import static org.apache.flink.streaming.api.utils.PythonOperatorUtils.getUserDefinedFunctionProto; +import static org.apache.flink.streaming.api.utils.ProtoUtils.createOverWindowArrowTypeCoderInfoDescriptorProto; +import static org.apache.flink.streaming.api.utils.ProtoUtils.getUserDefinedFunctionProto; /** The Batch Arrow Python {@link AggregateFunction} Operator for Over Window Aggregation. */ @Internal @@ -108,9 +109,7 @@ public BatchArrowPythonOverWindowAggregateFunctionOperator( outputType, groupKey, groupingSet, - udafInputOffsets, - FlinkFnApi.CoderParam.DataType.OVER_WINDOW_ARROW, - FlinkFnApi.CoderParam.DataType.ARROW); + udafInputOffsets); this.lowerBoundary = lowerBoundary; this.upperBoundary = upperBoundary; this.isRangeWindows = isRangeWindows; @@ -323,6 +322,12 @@ public String getFunctionUrn() { return PANDAS_BATCH_OVER_WINDOW_AGG_FUNCTION_URN; } + @Override + public FlinkFnApi.CoderInfoDescriptor createInputCoderInfoDescriptor(RowType runnerInputType) { + return createOverWindowArrowTypeCoderInfoDescriptorProto( + runnerInputType, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, false); + } + private boolean isInCurrentOverWindow(RowData data, long time, boolean includeEqual) { long dataTime = data.getTimestamp(inputTimeFieldIndex, 3).getMillisecond(); long diff = time - dataTime; diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/AbstractStreamArrowPythonOverWindowAggregateFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/AbstractStreamArrowPythonOverWindowAggregateFunctionOperator.java index 4c0ce4a75702c..99430ef5ed859 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/AbstractStreamArrowPythonOverWindowAggregateFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/AbstractStreamArrowPythonOverWindowAggregateFunctionOperator.java @@ -26,7 +26,6 @@ import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.typeutils.ListTypeInfo; import org.apache.flink.configuration.Configuration; -import org.apache.flink.fnexecution.v1.FlinkFnApi; import org.apache.flink.runtime.state.VoidNamespace; import org.apache.flink.runtime.state.VoidNamespaceSerializer; import org.apache.flink.streaming.api.SimpleTimerService; @@ -84,15 +83,7 @@ public AbstractStreamArrowPythonOverWindowAggregateFunctionOperator( long lowerBoundary, int[] groupingSet, int[] udafInputOffsets) { - super( - config, - pandasAggFunctions, - inputType, - outputType, - groupingSet, - udafInputOffsets, - FlinkFnApi.CoderParam.DataType.ARROW, - FlinkFnApi.CoderParam.DataType.ARROW); + super(config, pandasAggFunctions, inputType, outputType, groupingSet, udafInputOffsets); this.inputTimeFieldIndex = inputTimeFieldIndex; this.lowerBoundary = lowerBoundary; } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonGroupWindowAggregateFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonGroupWindowAggregateFunctionOperator.java index dc701e6c01640..2471d82ba5dcb 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonGroupWindowAggregateFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonGroupWindowAggregateFunctionOperator.java @@ -27,7 +27,6 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; -import org.apache.flink.fnexecution.v1.FlinkFnApi; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.state.internal.InternalListState; import org.apache.flink.streaming.api.operators.InternalTimer; @@ -154,15 +153,7 @@ public StreamArrowPythonGroupWindowAggregateFunctionOperator( int[] groupingSet, int[] udafInputOffsets, ZoneId shiftTimeZone) { - super( - config, - pandasAggFunctions, - inputType, - outputType, - groupingSet, - udafInputOffsets, - FlinkFnApi.CoderParam.DataType.ARROW, - FlinkFnApi.CoderParam.DataType.ARROW); + super(config, pandasAggFunctions, inputType, outputType, groupingSet, udafInputOffsets); this.inputTimeFieldIndex = inputTimeFieldIndex; this.windowAssigner = windowAssigner; this.trigger = trigger; diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/AbstractPythonScalarFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/AbstractPythonScalarFunctionOperator.java index e4887c984cf0b..d44eec3b1a8fb 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/AbstractPythonScalarFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/AbstractPythonScalarFunctionOperator.java @@ -21,7 +21,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.configuration.Configuration; import org.apache.flink.fnexecution.v1.FlinkFnApi; -import org.apache.flink.streaming.api.utils.PythonOperatorUtils; +import org.apache.flink.streaming.api.utils.ProtoUtils; import org.apache.flink.table.functions.ScalarFunction; import org.apache.flink.table.functions.python.PythonEnv; import org.apache.flink.table.functions.python.PythonFunctionInfo; @@ -69,17 +69,8 @@ public abstract class AbstractPythonScalarFunctionOperator RowType inputType, RowType outputType, int[] udfInputOffsets, - int[] forwardedFields, - FlinkFnApi.CoderParam.DataType inputDataType, - FlinkFnApi.CoderParam.DataType outputDataType) { - super( - config, - inputType, - outputType, - udfInputOffsets, - inputDataType, - outputDataType, - FlinkFnApi.CoderParam.OutputMode.SINGLE); + int[] forwardedFields) { + super(config, inputType, outputType, udfInputOffsets); this.scalarFunctions = Preconditions.checkNotNull(scalarFunctions); this.forwardedFields = Preconditions.checkNotNull(forwardedFields); } @@ -106,7 +97,7 @@ public FlinkFnApi.UserDefinedFunctions getUserDefinedFunctionsProto() { FlinkFnApi.UserDefinedFunctions.newBuilder(); // add udf proto for (PythonFunctionInfo pythonFunctionInfo : scalarFunctions) { - builder.addUdfs(PythonOperatorUtils.getUserDefinedFunctionProto(pythonFunctionInfo)); + builder.addUdfs(ProtoUtils.getUserDefinedFunctionProto(pythonFunctionInfo)); } builder.setMetricEnabled(getPythonConfig().isMetricEnabled()); return builder.build(); diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/AbstractRowDataPythonScalarFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/AbstractRowDataPythonScalarFunctionOperator.java index 64f0cf4ed41db..0b6dd8977601e 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/AbstractRowDataPythonScalarFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/AbstractRowDataPythonScalarFunctionOperator.java @@ -20,7 +20,6 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.configuration.Configuration; -import org.apache.flink.fnexecution.v1.FlinkFnApi; import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.binary.BinaryRowData; @@ -62,18 +61,8 @@ public AbstractRowDataPythonScalarFunctionOperator( RowType inputType, RowType outputType, int[] udfInputOffsets, - int[] forwardedFields, - FlinkFnApi.CoderParam.DataType inputDataType, - FlinkFnApi.CoderParam.DataType outputDataType) { - super( - config, - scalarFunctions, - inputType, - outputType, - udfInputOffsets, - forwardedFields, - inputDataType, - outputDataType); + int[] forwardedFields) { + super(config, scalarFunctions, inputType, outputType, udfInputOffsets, forwardedFields); } @Override diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/RowDataPythonScalarFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/RowDataPythonScalarFunctionOperator.java index 11c9f68d6ba5c..901baeb867130 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/RowDataPythonScalarFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/RowDataPythonScalarFunctionOperator.java @@ -31,6 +31,9 @@ import java.io.IOException; +import static org.apache.flink.streaming.api.utils.ProtoUtils.createFlattenRowTypeCoderInfoDescriptorProto; +import static org.apache.flink.streaming.api.utils.ProtoUtils.createRowTypeCoderInfoDescriptorProto; + /** The Python {@link ScalarFunction} operator. */ @Internal public class RowDataPythonScalarFunctionOperator @@ -51,15 +54,7 @@ public RowDataPythonScalarFunctionOperator( RowType outputType, int[] udfInputOffsets, int[] forwardedFields) { - super( - config, - scalarFunctions, - inputType, - outputType, - udfInputOffsets, - forwardedFields, - toCoderParam(scalarFunctions), - FlinkFnApi.CoderParam.DataType.FLATTEN_ROW); + super(config, scalarFunctions, inputType, outputType, udfInputOffsets, forwardedFields); } @Override @@ -71,6 +66,24 @@ public void open() throws Exception { PythonTypeUtils.toInternalSerializer(userDefinedFunctionOutputType); } + @Override + public FlinkFnApi.CoderInfoDescriptor createInputCoderInfoDescriptor(RowType runnerInputType) { + for (PythonFunctionInfo pythonFunctionInfo : scalarFunctions) { + if (pythonFunctionInfo.getPythonFunction().takesRowAsInput()) { + return createRowTypeCoderInfoDescriptorProto( + runnerInputType, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, false); + } + } + return createFlattenRowTypeCoderInfoDescriptorProto( + runnerInputType, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, false); + } + + @Override + public FlinkFnApi.CoderInfoDescriptor createOutputCoderInfoDescriptor(RowType runnerOutType) { + return createFlattenRowTypeCoderInfoDescriptorProto( + runnerOutType, FlinkFnApi.CoderInfoDescriptor.Mode.SINGLE, false); + } + @Override public void processElementInternal(RowData value) throws Exception { udfInputTypeSerializer.serialize(getFunctionInput(value), baosWrapper); @@ -89,14 +102,4 @@ public void emitResult(Tuple2 resultTuple) throws IOException { RowData udfResult = udfOutputTypeSerializer.deserialize(baisWrapper); rowDataWrapper.collect(reuseJoinedRow.replace(input, udfResult)); } - - private static FlinkFnApi.CoderParam.DataType toCoderParam( - PythonFunctionInfo[] pythonFunctionInfos) { - for (PythonFunctionInfo pythonFunctionInfo : pythonFunctionInfos) { - if (pythonFunctionInfo.getPythonFunction().takesRowAsInput()) { - return FlinkFnApi.CoderParam.DataType.ROW; - } - } - return FlinkFnApi.CoderParam.DataType.FLATTEN_ROW; - } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/RowDataArrowPythonScalarFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/RowDataArrowPythonScalarFunctionOperator.java index a63d8fd09f2ac..ca485fe776b86 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/RowDataArrowPythonScalarFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/RowDataArrowPythonScalarFunctionOperator.java @@ -30,6 +30,8 @@ import org.apache.flink.table.runtime.operators.python.scalar.AbstractRowDataPythonScalarFunctionOperator; import org.apache.flink.table.types.logical.RowType; +import static org.apache.flink.streaming.api.utils.ProtoUtils.createArrowTypeCoderInfoDescriptorProto; + /** Arrow Python {@link ScalarFunction} operator. */ @Internal public class RowDataArrowPythonScalarFunctionOperator @@ -52,15 +54,7 @@ public RowDataArrowPythonScalarFunctionOperator( RowType outputType, int[] udfInputOffsets, int[] forwardedFields) { - super( - config, - scalarFunctions, - inputType, - outputType, - udfInputOffsets, - forwardedFields, - FlinkFnApi.CoderParam.DataType.ARROW, - FlinkFnApi.CoderParam.DataType.ARROW); + super(config, scalarFunctions, inputType, outputType, udfInputOffsets, forwardedFields); } @Override @@ -74,6 +68,19 @@ public void open() throws Exception { currentBatchCount = 0; } + @Override + public FlinkFnApi.CoderInfoDescriptor createInputCoderInfoDescriptor(RowType runnerInputType) { + return createArrowTypeCoderInfoDescriptorProto( + runnerInputType, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, false); + } + + @Override + public FlinkFnApi.CoderInfoDescriptor createOutputCoderInfoDescriptor( + RowType runnerOutputType) { + return createArrowTypeCoderInfoDescriptorProto( + runnerOutputType, FlinkFnApi.CoderInfoDescriptor.Mode.SINGLE, false); + } + @Override protected void invokeFinishBundle() throws Exception { invokeCurrentBatch(); diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/table/AbstractPythonTableFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/table/AbstractPythonTableFunctionOperator.java index 6536f1114ff85..e8057344318a3 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/table/AbstractPythonTableFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/table/AbstractPythonTableFunctionOperator.java @@ -21,7 +21,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.configuration.Configuration; import org.apache.flink.fnexecution.v1.FlinkFnApi; -import org.apache.flink.streaming.api.utils.PythonOperatorUtils; +import org.apache.flink.streaming.api.utils.ProtoUtils; import org.apache.flink.table.functions.TableFunction; import org.apache.flink.table.functions.python.PythonEnv; import org.apache.flink.table.functions.python.PythonFunctionInfo; @@ -33,6 +33,9 @@ import java.util.ArrayList; import java.util.List; +import static org.apache.flink.streaming.api.utils.ProtoUtils.createFlattenRowTypeCoderInfoDescriptorProto; +import static org.apache.flink.streaming.api.utils.ProtoUtils.createRowTypeCoderInfoDescriptorProto; + /** * @param Type of the input elements. * @param Type of the output elements. @@ -59,14 +62,7 @@ public AbstractPythonTableFunctionOperator( RowType outputType, int[] udtfInputOffsets, FlinkJoinType joinType) { - super( - config, - inputType, - outputType, - udtfInputOffsets, - toCoderParam(tableFunction), - FlinkFnApi.CoderParam.DataType.FLATTEN_ROW, - FlinkFnApi.CoderParam.OutputMode.MULTIPLE_WITH_END); + super(config, inputType, outputType, udtfInputOffsets); this.tableFunction = Preconditions.checkNotNull(tableFunction); Preconditions.checkArgument( joinType == FlinkJoinType.INNER || joinType == FlinkJoinType.LEFT, @@ -95,24 +91,33 @@ public String getFunctionUrn() { return TABLE_FUNCTION_URN; } + @Override + public FlinkFnApi.CoderInfoDescriptor createInputCoderInfoDescriptor(RowType runnerInputType) { + if (tableFunction.getPythonFunction().takesRowAsInput()) { + // need the field names in case of row-based operations + return createRowTypeCoderInfoDescriptorProto( + runnerInputType, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, true); + } else { + return createFlattenRowTypeCoderInfoDescriptorProto( + runnerInputType, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, true); + } + } + + @Override + public FlinkFnApi.CoderInfoDescriptor createOutputCoderInfoDescriptor(RowType runnerOutType) { + return createFlattenRowTypeCoderInfoDescriptorProto( + runnerOutType, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, true); + } + @Override public FlinkFnApi.UserDefinedFunctions getUserDefinedFunctionsProto() { FlinkFnApi.UserDefinedFunctions.Builder builder = FlinkFnApi.UserDefinedFunctions.newBuilder(); - builder.addUdfs(PythonOperatorUtils.getUserDefinedFunctionProto(tableFunction)); + builder.addUdfs(ProtoUtils.getUserDefinedFunctionProto(tableFunction)); builder.setMetricEnabled(getPythonConfig().isMetricEnabled()); return builder.build(); } - private static FlinkFnApi.CoderParam.DataType toCoderParam( - PythonFunctionInfo pythonFunctionInfo) { - if (pythonFunctionInfo.getPythonFunction().takesRowAsInput()) { - return FlinkFnApi.CoderParam.DataType.ROW; - } else { - return FlinkFnApi.CoderParam.DataType.FLATTEN_ROW; - } - } - /** The received udtf execution result is a finish message when it is a byte with value 0x00. */ boolean isFinishResult(byte[] rawUdtfResult, int length) { return length == 1 && rawUdtfResult[0] == 0x00; diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/beam/BeamTablePythonFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/beam/BeamTablePythonFunctionRunner.java index 1f5dde0bcc97a..c4943c57e8a7a 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/beam/BeamTablePythonFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/beam/BeamTablePythonFunctionRunner.java @@ -26,30 +26,21 @@ import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.runtime.state.KeyedStateBackend; import org.apache.flink.streaming.api.runners.python.beam.BeamPythonFunctionRunner; -import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.Preconditions; import com.google.protobuf.GeneratedMessageV3; -import org.apache.beam.model.pipeline.v1.RunnerApi; import java.util.Map; -import static org.apache.flink.python.Constants.FLINK_CODER_URN; -import static org.apache.flink.table.runtime.typeutils.PythonTypeUtils.toProtoType; - /** A {@link BeamTablePythonFunctionRunner} used to execute Python functions in Table API. */ @Internal public class BeamTablePythonFunctionRunner extends BeamPythonFunctionRunner { - private final RowType inputType; - private final RowType outputType; private final GeneratedMessageV3 userDefinedFunctionProto; public BeamTablePythonFunctionRunner( String taskName, PythonEnvironmentManager environmentManager, - RowType inputType, - RowType outputType, String functionUrn, GeneratedMessageV3 userDefinedFunctionProto, Map jobOptions, @@ -59,9 +50,8 @@ public BeamTablePythonFunctionRunner( TypeSerializer namespaceSerializer, MemoryManager memoryManager, double managedMemoryFraction, - FlinkFnApi.CoderParam.DataType inputDataType, - FlinkFnApi.CoderParam.DataType outputDataType, - FlinkFnApi.CoderParam.OutputMode outputMode) { + FlinkFnApi.CoderInfoDescriptor inputCoderDescriptor, + FlinkFnApi.CoderInfoDescriptor outputCoderDescriptor) { super( taskName, environmentManager, @@ -73,11 +63,8 @@ public BeamTablePythonFunctionRunner( namespaceSerializer, memoryManager, managedMemoryFraction, - inputDataType, - outputDataType, - outputMode); - this.inputType = Preconditions.checkNotNull(inputType); - this.outputType = Preconditions.checkNotNull(outputType); + inputCoderDescriptor, + outputCoderDescriptor); this.userDefinedFunctionProto = Preconditions.checkNotNull(userDefinedFunctionProto); } @@ -85,35 +72,4 @@ public BeamTablePythonFunctionRunner( protected byte[] getUserDefinedFunctionsProtoBytes() { return userDefinedFunctionProto.toByteArray(); } - - @Override - protected RunnerApi.Coder getInputCoderProto() { - return getRowCoderProto(inputType, inputDataType, outputMode); - } - - @Override - protected RunnerApi.Coder getOutputCoderProto() { - return getRowCoderProto(outputType, outputDataType, outputMode); - } - - private static RunnerApi.Coder getRowCoderProto( - RowType rowType, - FlinkFnApi.CoderParam.DataType dataType, - FlinkFnApi.CoderParam.OutputMode outputMode) { - FlinkFnApi.Schema rowSchema = toProtoType(rowType).getRowSchema(); - FlinkFnApi.CoderParam.Builder coderParamBuilder = FlinkFnApi.CoderParam.newBuilder(); - coderParamBuilder.setDataType(dataType); - coderParamBuilder.setSchema(rowSchema); - coderParamBuilder.setOutputMode(outputMode); - return RunnerApi.Coder.newBuilder() - .setSpec( - RunnerApi.FunctionSpec.newBuilder() - .setUrn(FLINK_CODER_URN) - .setPayload( - org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf - .ByteString.copyFrom( - coderParamBuilder.build().toByteArray())) - .build()) - .build(); - } } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonAggregateFunctionRunner.java b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonAggregateFunctionRunner.java index 76cb9ab39d1f4..48fcbee5a840c 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonAggregateFunctionRunner.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonAggregateFunctionRunner.java @@ -39,6 +39,8 @@ import java.util.List; import java.util.Map; +import static org.apache.flink.streaming.api.utils.ProtoUtils.createArrowTypeCoderInfoDescriptorProto; + /** * A {@link PassThroughPythonAggregateFunctionRunner} runner that just return the first input * element with the same key as the execution results. @@ -76,8 +78,6 @@ public PassThroughPythonAggregateFunctionRunner( super( taskName, environmentManager, - inputType, - outputType, functionUrn, userDefinedFunctions, jobOptions, @@ -87,9 +87,10 @@ public PassThroughPythonAggregateFunctionRunner( null, null, 0.0, - FlinkFnApi.CoderParam.DataType.ARROW, - FlinkFnApi.CoderParam.DataType.ARROW, - FlinkFnApi.CoderParam.OutputMode.SINGLE); + createArrowTypeCoderInfoDescriptorProto( + inputType, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, false), + createArrowTypeCoderInfoDescriptorProto( + outputType, FlinkFnApi.CoderInfoDescriptor.Mode.SINGLE, false)); this.buffer = new LinkedList<>(); this.isBatchOverWindow = isBatchOverWindow; arrowSerializer = new RowDataArrowSerializer(inputType, outputType); diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonScalarFunctionRunner.java b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonScalarFunctionRunner.java index 04e5c57028ae1..7638e569ea1da 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonScalarFunctionRunner.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonScalarFunctionRunner.java @@ -31,6 +31,8 @@ import java.util.List; import java.util.Map; +import static org.apache.flink.streaming.api.utils.ProtoUtils.createFlattenRowTypeCoderInfoDescriptorProto; + /** * A {@link PassThroughPythonScalarFunctionRunner} runner that just return the input elements as the * execution results. @@ -51,8 +53,6 @@ public PassThroughPythonScalarFunctionRunner( super( taskName, environmentManager, - inputType, - outputType, functionUrn, userDefinedFunctions, jobOptions, @@ -62,9 +62,10 @@ public PassThroughPythonScalarFunctionRunner( null, null, 0.0, - FlinkFnApi.CoderParam.DataType.FLATTEN_ROW, - FlinkFnApi.CoderParam.DataType.FLATTEN_ROW, - FlinkFnApi.CoderParam.OutputMode.SINGLE); + createFlattenRowTypeCoderInfoDescriptorProto( + inputType, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, false), + createFlattenRowTypeCoderInfoDescriptorProto( + outputType, FlinkFnApi.CoderInfoDescriptor.Mode.SINGLE, false)); this.buffer = new LinkedList<>(); } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonTableFunctionRunner.java b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonTableFunctionRunner.java index b2165f7302776..e4cd02d9514f5 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonTableFunctionRunner.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonTableFunctionRunner.java @@ -31,6 +31,8 @@ import java.util.List; import java.util.Map; +import static org.apache.flink.streaming.api.utils.ProtoUtils.createFlattenRowTypeCoderInfoDescriptorProto; + /** * A {@link BeamTablePythonFunctionRunner} that emit each input element in inner join and emit null * in left join when certain test conditions are met. @@ -53,8 +55,6 @@ public PassThroughPythonTableFunctionRunner( super( taskName, environmentManager, - inputType, - outputType, functionUrn, userDefinedFunctions, jobOptions, @@ -64,9 +64,10 @@ public PassThroughPythonTableFunctionRunner( null, null, 0.0, - FlinkFnApi.CoderParam.DataType.FLATTEN_ROW, - FlinkFnApi.CoderParam.DataType.FLATTEN_ROW, - FlinkFnApi.CoderParam.OutputMode.MULTIPLE_WITH_END); + createFlattenRowTypeCoderInfoDescriptorProto( + inputType, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, true), + createFlattenRowTypeCoderInfoDescriptorProto( + outputType, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, true)); this.buffer = new LinkedList<>(); } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamAggregatePythonFunctionRunner.java b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamAggregatePythonFunctionRunner.java index 4fe2f0c91a8bc..14e147f1c6f99 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamAggregatePythonFunctionRunner.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamAggregatePythonFunctionRunner.java @@ -34,6 +34,8 @@ import java.util.Map; import java.util.function.Function; +import static org.apache.flink.streaming.api.utils.ProtoUtils.createRowTypeCoderInfoDescriptorProto; + /** * A {@link PassThroughStreamAggregatePythonFunctionRunner} runner that help to test the Python * stream group aggregate operators. It will process the input data with the provided @@ -60,8 +62,6 @@ public PassThroughStreamAggregatePythonFunctionRunner( super( taskName, environmentManager, - inputType, - outputType, functionUrn, userDefinedFunctions, jobOptions, @@ -71,9 +71,10 @@ public PassThroughStreamAggregatePythonFunctionRunner( null, null, 0.0, - FlinkFnApi.CoderParam.DataType.ROW, - FlinkFnApi.CoderParam.DataType.ROW, - FlinkFnApi.CoderParam.OutputMode.MULTIPLE); + createRowTypeCoderInfoDescriptorProto( + inputType, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, false), + createRowTypeCoderInfoDescriptorProto( + outputType, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, false)); this.buffer = new LinkedList<>(); this.processFunction = processFunction; } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamGroupWindowAggregatePythonFunctionRunner.java b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamGroupWindowAggregatePythonFunctionRunner.java index 0126d4b45f205..12e0e2f56055c 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamGroupWindowAggregatePythonFunctionRunner.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamGroupWindowAggregatePythonFunctionRunner.java @@ -32,6 +32,8 @@ import java.util.Map; +import static org.apache.flink.streaming.api.utils.ProtoUtils.createFlattenRowTypeCoderInfoDescriptorProto; + /** * A {@link PassThroughStreamGroupWindowAggregatePythonFunctionRunner} runner that help to test the * Python stream group window aggregate operators. @@ -56,8 +58,6 @@ public PassThroughStreamGroupWindowAggregatePythonFunctionRunner( super( taskName, environmentManager, - inputType, - outputType, functionUrn, userDefinedFunctions, jobOptions, @@ -67,9 +67,10 @@ public PassThroughStreamGroupWindowAggregatePythonFunctionRunner( null, null, 0.0, - FlinkFnApi.CoderParam.DataType.FLATTEN_ROW, - FlinkFnApi.CoderParam.DataType.FLATTEN_ROW, - FlinkFnApi.CoderParam.OutputMode.MULTIPLE); + createFlattenRowTypeCoderInfoDescriptorProto( + inputType, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, false), + createFlattenRowTypeCoderInfoDescriptorProto( + outputType, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, false)); this.operator = operator; } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamTableAggregatePythonFunctionRunner.java b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamTableAggregatePythonFunctionRunner.java index 77934cb96b40c..b84b63af46e0f 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamTableAggregatePythonFunctionRunner.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamTableAggregatePythonFunctionRunner.java @@ -35,6 +35,8 @@ import java.util.Map; import java.util.function.Function; +import static org.apache.flink.streaming.api.utils.ProtoUtils.createRowTypeCoderInfoDescriptorProto; + /** * A {@link PassThroughStreamTableAggregatePythonFunctionRunner} runner that help to test the Python * stream group table aggregate operators. It will process the input data with the provided @@ -62,8 +64,6 @@ public PassThroughStreamTableAggregatePythonFunctionRunner( super( taskName, environmentManager, - inputType, - outputType, functionUrn, userDefinedFunctions, jobOptions, @@ -73,9 +73,10 @@ public PassThroughStreamTableAggregatePythonFunctionRunner( null, null, 0.0, - FlinkFnApi.CoderParam.DataType.ROW, - FlinkFnApi.CoderParam.DataType.ROW, - FlinkFnApi.CoderParam.OutputMode.MULTIPLE); + createRowTypeCoderInfoDescriptorProto( + inputType, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, false), + createRowTypeCoderInfoDescriptorProto( + outputType, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, false)); this.buffer = new LinkedList<>(); this.processFunction = processFunction; }