charlesdong1991 commented on code in PR #27145:
URL: https://github.com/apache/flink/pull/27145#discussion_r2462795983


##########
flink-python/pyflink/datastream/async_data_stream.py:
##########
@@ -0,0 +1,77 @@
+################################################################################
+#  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.
+################################################################################
+import inspect
+
+from pyflink.common import Time, TypeInformation
+from pyflink.datastream.data_stream import DataStream, 
_get_one_input_stream_operator
+from pyflink.datastream.functions import AsyncFunctionDescriptor, AsyncFunction
+from pyflink.java_gateway import get_gateway
+from pyflink.util.java_utils import get_j_env_configuration
+
+
+class AsyncDataStream(object):
+    """
+    A helper class to apply :class:`~AsyncFunction` to a data stream.
+    """
+
+    @staticmethod
+    def unordered_wait(
+            data_stream: DataStream,
+            async_function: AsyncFunction,
+            timeout: Time,
+            capacity: int = 100,
+            output_type: TypeInformation = None) -> 'DataStream':
+        """
+        Adds an async function to the data stream. The order of output stream 
records may be
+        reordered.
+
+        :param data_stream: The input data stream.
+        :param async_function: The async function.
+        :param timeout: The timeout for the asynchronous operation to complete.
+        :param capacity: The max number of async i/o operation that can be 
triggered.
+        :param output_type: The output data type.
+        :return: The transformed DataStream.
+        """
+        AsyncDataStream._validate(data_stream, async_function)
+
+        from pyflink.fn_execution import flink_fn_execution_pb2
+        j_python_data_stream_function_operator, j_output_type_info = \
+            _get_one_input_stream_operator(
+                data_stream,
+                AsyncFunctionDescriptor(
+                    async_function, timeout, capacity,
+                    AsyncFunctionDescriptor.OutputMode.UNORDERED),
+                flink_fn_execution_pb2.UserDefinedDataStreamFunction.PROCESS,  
# type: ignore
+                output_type)
+        return DataStream(data_stream._j_data_stream.transform(
+            "async wait operator",
+            j_output_type_info,
+            j_python_data_stream_function_operator))
+
+    @staticmethod
+    def _validate(data_stream: DataStream, async_function: AsyncFunction):
+        if not inspect.iscoroutinefunction(async_function.async_invoke):
+            raise Exception("Method 'async_invoke' of class '%s' should be 
declared as 'async def'."
+                            % type(async_function))

Review Comment:
   ```suggestion
                               % type(async_function).__name__)
   ```
   i think `type(function)` will return a verbose result, maybe this change 
gives actual name



##########
flink-python/pyflink/fn_execution/datastream/process/async_function/operation.py:
##########
@@ -0,0 +1,276 @@
+################################################################################
+#  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.
+################################################################################
+import asyncio
+import pickle
+import threading
+from typing import TypeVar, Generic, List, Iterable
+
+from pyflink.datastream import RuntimeContext, ResultFuture
+from pyflink.datastream.functions import AsyncFunctionDescriptor
+from pyflink.fn_execution.datastream.process.async_function.queue import \
+    UnorderedStreamElementQueue, StreamElementQueue
+from pyflink.fn_execution.datastream.process.operations import Operation
+from pyflink.fn_execution.datastream.process.runtime_context import 
StreamingRuntimeContext
+
+OUT = TypeVar('OUT')
+
+
+class AtomicBoolean(object):
+    def __init__(self, initial_value=False):
+        self._value = initial_value
+        self._lock = threading.Lock()
+
+    def get(self):
+        with self._lock:
+            return self._value
+
+    def set(self, new_value):
+        with self._lock:
+            self._value = new_value
+
+    def get_and_set(self, new_value):
+        with self._lock:
+            old_value = self._value
+            self._value = new_value
+            return old_value
+
+    def compare_and_set(self, expected, new_value):
+        with self._lock:
+            if self._value == expected:
+                self._value = new_value
+                return True
+            return False
+
+
+class ResultHandler(ResultFuture, Generic[OUT]):
+
+    def __init__(self, classname, timeout_func, exception_handler, record,
+                 result_future: ResultFuture[OUT]):
+        self._classname = classname
+        self._timeout_func = timeout_func
+        self._exception_handler = exception_handler
+        self._record = record
+        self._result_future = result_future
+        self._timer = None
+        self._completed = AtomicBoolean(False)
+
+    def register_timeout(self, timeout: int):
+        self._timer = threading.Timer(timeout, self._timer_triggered)
+        self._timer.start()
+
+    def complete(self, result: List[OUT]):
+        # already completed (exceptionally or with previous complete call from 
ill-written
+        # AsyncFunction), so ignore additional result
+        if not self._completed.compare_and_set(False, True):
+            return
+
+        if isinstance(result, Iterable):
+            self._process_results(result)
+        else:
+            # complete with empty result, so that we remove timer and move 
ahead processing
+            self._process_results([])
+
+        if not isinstance(result, Iterable):
+            raise RuntimeError("The 'result_future' of AsyncFunction should be 
completed with "
+                               "data of list type, please check the methods 
'async_invoke' and "
+                               "'timeout' of class '%s'." % self._classname)
+
+    def complete_exceptionally(self, error: Exception):
+        # already completed, so ignore exception
+        if not self._completed.compare_and_set(False, True):
+            return
+
+        self._exception_handler(
+            Exception("Could not complete the element:" + str(self._record), 
error))
+
+        #  complete with empty result, so that we remove timer and move ahead 
processing
+        self._process_results([])
+
+    def _process_results(self, result: List[OUT]):
+        if self._timer is not None:
+            self._timer.cancel()
+            self._timer = None
+
+        self._result_future.complete(result)
+
+    def _timer_triggered(self):
+        if not self._completed.get():
+            self._timeout_func(self._record, self)
+
+class Emitter(threading.Thread):
+
+    def __init__(self, exception_handler, output_processor, queue: 
StreamElementQueue):
+        super().__init__()
+        self._exception_handler = exception_handler
+        self._output_processor = output_processor
+        self._queue = queue
+        self._running = True
+
+    def run(self):
+        while self._running:
+            try:
+                if self._queue.has_completed_elements():
+                    self._queue.emit_completed_element(self._output_processor)
+                else:
+                    self._queue.wait_for_completed_elements()
+            except Exception as e:
+                self._running = False
+                self._exception_handler(e)
+
+    def stop(self):
+        self._running = False
+
+class AsyncFunctionRunner(threading.Thread):
+    def __init__(self, exception_handler):
+        super().__init__()
+        self._exception_handler = exception_handler
+        self._loop = None
+
+    def run(self):
+        self._loop = asyncio.new_event_loop()
+        asyncio.set_event_loop(self._loop)
+        self._loop.run_forever()
+
+    def stop(self):
+        if self._loop is not None:
+            self._loop.stop()
+            self._loop = None
+
+    async def exception_handler_wrapper(self, async_function, *arg):
+        try:
+            await async_function(*arg)
+        except Exception as e:
+            self._exception_handler(e)
+
+    def run_async(self, async_function, *arg):
+        wrapped_function = self.exception_handler_wrapper(async_function, *arg)
+        asyncio.run_coroutine_threadsafe(wrapped_function, self._loop)
+
+class AsyncOperation(Operation):
+    def __init__(self, serialized_fn, operator_state_backend):
+        super(AsyncOperation, self).__init__(serialized_fn, 
operator_state_backend)
+        (
+            self.class_name,
+            self.open_func,
+            self.close_func,
+            self.async_invoke_func,
+            self.timeout_func,
+            self._timeout,
+            capacity,
+            output_mode
+        ) = extract_async_function(
+            user_defined_function_proto=serialized_fn,
+            runtime_context=StreamingRuntimeContext.of(
+                serialized_fn.runtime_context, self.base_metric_group
+            )
+        )
+        self._retry_enabled = False
+        if output_mode == AsyncFunctionDescriptor.OutputMode.UNORDERED:
+            self._queue = UnorderedStreamElementQueue(capacity, 
self._raise_exception_if_exists)
+        else:
+            raise NotImplementedError()
+        self._emitter = None
+        self._async_function_runner = None
+        self._exception = None
+
+    def set_output_processor(self, output_processor):
+        self._output_processor = output_processor
+
+    def open(self):
+        self.open_func()
+        self._emitter = Emitter(self._mark_exception, self._output_processor, 
self._queue)
+        self._emitter.daemon = True
+        self._emitter.start()
+
+        self._async_function_runner = AsyncFunctionRunner(self._mark_exception)
+        self._async_function_runner.daemon = True
+        self._async_function_runner.start()
+
+    def close(self):
+        self.close_func()
+        if self._emitter is not None:
+            self._emitter.stop()
+            self._emitter = None
+
+        if self._async_function_runner is not None:
+            self._async_function_runner.stop()
+            self._async_function_runner = None
+
+    def process_element(self, windowed_value, element):
+        self._raise_exception_if_exists()
+
+        # VALUE[CURRENT_TIMESTAMP, CURRENT_WATERMARK, NORMAL_DATA]
+        timestamp = element[0]
+        watermark = element[1]
+        record = element[2]
+
+        self._queue.advance_watermark(watermark)

Review Comment:
   for my own curiosity, what is the reason to advance watermark before the 
entry (the line below)?



##########
flink-python/pyflink/fn_execution/datastream/process/async_function/operation.py:
##########
@@ -0,0 +1,276 @@
+################################################################################
+#  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.
+################################################################################
+import asyncio
+import pickle
+import threading
+from typing import TypeVar, Generic, List, Iterable
+
+from pyflink.datastream import RuntimeContext, ResultFuture
+from pyflink.datastream.functions import AsyncFunctionDescriptor
+from pyflink.fn_execution.datastream.process.async_function.queue import \
+    UnorderedStreamElementQueue, StreamElementQueue
+from pyflink.fn_execution.datastream.process.operations import Operation
+from pyflink.fn_execution.datastream.process.runtime_context import 
StreamingRuntimeContext
+
+OUT = TypeVar('OUT')
+
+
+class AtomicBoolean(object):
+    def __init__(self, initial_value=False):
+        self._value = initial_value
+        self._lock = threading.Lock()
+
+    def get(self):
+        with self._lock:
+            return self._value
+
+    def set(self, new_value):
+        with self._lock:
+            self._value = new_value
+
+    def get_and_set(self, new_value):
+        with self._lock:
+            old_value = self._value
+            self._value = new_value
+            return old_value
+
+    def compare_and_set(self, expected, new_value):
+        with self._lock:
+            if self._value == expected:
+                self._value = new_value
+                return True
+            return False
+
+
+class ResultHandler(ResultFuture, Generic[OUT]):
+
+    def __init__(self, classname, timeout_func, exception_handler, record,

Review Comment:
   would be nice to add type hints here too?



##########
flink-python/pyflink/fn_execution/datastream/process/async_function/operation.py:
##########
@@ -0,0 +1,276 @@
+################################################################################
+#  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.
+################################################################################
+import asyncio
+import pickle
+import threading
+from typing import TypeVar, Generic, List, Iterable
+
+from pyflink.datastream import RuntimeContext, ResultFuture
+from pyflink.datastream.functions import AsyncFunctionDescriptor
+from pyflink.fn_execution.datastream.process.async_function.queue import \
+    UnorderedStreamElementQueue, StreamElementQueue
+from pyflink.fn_execution.datastream.process.operations import Operation
+from pyflink.fn_execution.datastream.process.runtime_context import 
StreamingRuntimeContext
+
+OUT = TypeVar('OUT')
+
+
+class AtomicBoolean(object):
+    def __init__(self, initial_value=False):
+        self._value = initial_value
+        self._lock = threading.Lock()
+
+    def get(self):
+        with self._lock:
+            return self._value
+
+    def set(self, new_value):
+        with self._lock:
+            self._value = new_value
+
+    def get_and_set(self, new_value):
+        with self._lock:
+            old_value = self._value
+            self._value = new_value
+            return old_value
+
+    def compare_and_set(self, expected, new_value):
+        with self._lock:
+            if self._value == expected:
+                self._value = new_value
+                return True
+            return False
+
+
+class ResultHandler(ResultFuture, Generic[OUT]):
+
+    def __init__(self, classname, timeout_func, exception_handler, record,
+                 result_future: ResultFuture[OUT]):
+        self._classname = classname
+        self._timeout_func = timeout_func
+        self._exception_handler = exception_handler
+        self._record = record
+        self._result_future = result_future
+        self._timer = None
+        self._completed = AtomicBoolean(False)
+
+    def register_timeout(self, timeout: int):
+        self._timer = threading.Timer(timeout, self._timer_triggered)
+        self._timer.start()
+
+    def complete(self, result: List[OUT]):
+        # already completed (exceptionally or with previous complete call from 
ill-written
+        # AsyncFunction), so ignore additional result
+        if not self._completed.compare_and_set(False, True):
+            return
+
+        if isinstance(result, Iterable):
+            self._process_results(result)
+        else:
+            # complete with empty result, so that we remove timer and move 
ahead processing
+            self._process_results([])
+
+        if not isinstance(result, Iterable):
+            raise RuntimeError("The 'result_future' of AsyncFunction should be 
completed with "
+                               "data of list type, please check the methods 
'async_invoke' and "
+                               "'timeout' of class '%s'." % self._classname)
+
+    def complete_exceptionally(self, error: Exception):
+        # already completed, so ignore exception
+        if not self._completed.compare_and_set(False, True):
+            return
+
+        self._exception_handler(
+            Exception("Could not complete the element:" + str(self._record), 
error))

Review Comment:
   i wonder if it is possible to have a test case to trigger this exception?



##########
flink-python/pyflink/datastream/async_data_stream.py:
##########
@@ -0,0 +1,77 @@
+################################################################################
+#  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.
+################################################################################
+import inspect
+
+from pyflink.common import Time, TypeInformation
+from pyflink.datastream.data_stream import DataStream, 
_get_one_input_stream_operator
+from pyflink.datastream.functions import AsyncFunctionDescriptor, AsyncFunction
+from pyflink.java_gateway import get_gateway
+from pyflink.util.java_utils import get_j_env_configuration
+
+
+class AsyncDataStream(object):
+    """
+    A helper class to apply :class:`~AsyncFunction` to a data stream.
+    """
+
+    @staticmethod
+    def unordered_wait(
+            data_stream: DataStream,
+            async_function: AsyncFunction,
+            timeout: Time,
+            capacity: int = 100,
+            output_type: TypeInformation = None) -> 'DataStream':
+        """
+        Adds an async function to the data stream. The order of output stream 
records may be
+        reordered.
+
+        :param data_stream: The input data stream.
+        :param async_function: The async function.
+        :param timeout: The timeout for the asynchronous operation to complete.
+        :param capacity: The max number of async i/o operation that can be 
triggered.
+        :param output_type: The output data type.
+        :return: The transformed DataStream.
+        """
+        AsyncDataStream._validate(data_stream, async_function)
+
+        from pyflink.fn_execution import flink_fn_execution_pb2
+        j_python_data_stream_function_operator, j_output_type_info = \
+            _get_one_input_stream_operator(
+                data_stream,
+                AsyncFunctionDescriptor(
+                    async_function, timeout, capacity,
+                    AsyncFunctionDescriptor.OutputMode.UNORDERED),
+                flink_fn_execution_pb2.UserDefinedDataStreamFunction.PROCESS,  
# type: ignore
+                output_type)
+        return DataStream(data_stream._j_data_stream.transform(
+            "async wait operator",
+            j_output_type_info,
+            j_python_data_stream_function_operator))
+
+    @staticmethod
+    def _validate(data_stream: DataStream, async_function: AsyncFunction):

Review Comment:
   ```suggestion
       def _validate(data_stream: DataStream, async_function: AsyncFunction) -> 
None:
   ```
   



##########
flink-python/pyflink/fn_execution/datastream/process/async_function/operation.py:
##########
@@ -0,0 +1,276 @@
+################################################################################
+#  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.
+################################################################################
+import asyncio
+import pickle
+import threading
+from typing import TypeVar, Generic, List, Iterable
+
+from pyflink.datastream import RuntimeContext, ResultFuture
+from pyflink.datastream.functions import AsyncFunctionDescriptor
+from pyflink.fn_execution.datastream.process.async_function.queue import \
+    UnorderedStreamElementQueue, StreamElementQueue
+from pyflink.fn_execution.datastream.process.operations import Operation
+from pyflink.fn_execution.datastream.process.runtime_context import 
StreamingRuntimeContext
+
+OUT = TypeVar('OUT')
+
+
+class AtomicBoolean(object):
+    def __init__(self, initial_value=False):
+        self._value = initial_value
+        self._lock = threading.Lock()
+
+    def get(self):
+        with self._lock:
+            return self._value
+
+    def set(self, new_value):
+        with self._lock:
+            self._value = new_value
+
+    def get_and_set(self, new_value):
+        with self._lock:
+            old_value = self._value
+            self._value = new_value
+            return old_value
+
+    def compare_and_set(self, expected, new_value):
+        with self._lock:
+            if self._value == expected:
+                self._value = new_value
+                return True
+            return False
+
+
+class ResultHandler(ResultFuture, Generic[OUT]):

Review Comment:
   do i understand correctly that retry is not supported?



##########
flink-python/pyflink/fn_execution/datastream/process/async_function/operation.py:
##########
@@ -0,0 +1,276 @@
+################################################################################
+#  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.
+################################################################################
+import asyncio
+import pickle
+import threading
+from typing import TypeVar, Generic, List, Iterable
+
+from pyflink.datastream import RuntimeContext, ResultFuture
+from pyflink.datastream.functions import AsyncFunctionDescriptor
+from pyflink.fn_execution.datastream.process.async_function.queue import \
+    UnorderedStreamElementQueue, StreamElementQueue
+from pyflink.fn_execution.datastream.process.operations import Operation
+from pyflink.fn_execution.datastream.process.runtime_context import 
StreamingRuntimeContext
+
+OUT = TypeVar('OUT')
+
+
+class AtomicBoolean(object):
+    def __init__(self, initial_value=False):
+        self._value = initial_value
+        self._lock = threading.Lock()
+
+    def get(self):
+        with self._lock:
+            return self._value
+
+    def set(self, new_value):
+        with self._lock:
+            self._value = new_value
+
+    def get_and_set(self, new_value):
+        with self._lock:
+            old_value = self._value
+            self._value = new_value
+            return old_value
+
+    def compare_and_set(self, expected, new_value):
+        with self._lock:
+            if self._value == expected:
+                self._value = new_value
+                return True
+            return False
+
+
+class ResultHandler(ResultFuture, Generic[OUT]):
+
+    def __init__(self, classname, timeout_func, exception_handler, record,
+                 result_future: ResultFuture[OUT]):
+        self._classname = classname
+        self._timeout_func = timeout_func
+        self._exception_handler = exception_handler
+        self._record = record
+        self._result_future = result_future
+        self._timer = None
+        self._completed = AtomicBoolean(False)
+
+    def register_timeout(self, timeout: int):
+        self._timer = threading.Timer(timeout, self._timer_triggered)
+        self._timer.start()
+
+    def complete(self, result: List[OUT]):
+        # already completed (exceptionally or with previous complete call from 
ill-written
+        # AsyncFunction), so ignore additional result
+        if not self._completed.compare_and_set(False, True):
+            return
+
+        if isinstance(result, Iterable):
+            self._process_results(result)
+        else:
+            # complete with empty result, so that we remove timer and move 
ahead processing
+            self._process_results([])
+
+        if not isinstance(result, Iterable):
+            raise RuntimeError("The 'result_future' of AsyncFunction should be 
completed with "
+                               "data of list type, please check the methods 
'async_invoke' and "
+                               "'timeout' of class '%s'." % self._classname)
+
+    def complete_exceptionally(self, error: Exception):
+        # already completed, so ignore exception
+        if not self._completed.compare_and_set(False, True):
+            return
+
+        self._exception_handler(
+            Exception("Could not complete the element:" + str(self._record), 
error))
+
+        #  complete with empty result, so that we remove timer and move ahead 
processing
+        self._process_results([])
+
+    def _process_results(self, result: List[OUT]):
+        if self._timer is not None:
+            self._timer.cancel()
+            self._timer = None
+
+        self._result_future.complete(result)
+
+    def _timer_triggered(self):
+        if not self._completed.get():
+            self._timeout_func(self._record, self)
+
+class Emitter(threading.Thread):
+
+    def __init__(self, exception_handler, output_processor, queue: 
StreamElementQueue):
+        super().__init__()
+        self._exception_handler = exception_handler
+        self._output_processor = output_processor
+        self._queue = queue
+        self._running = True
+
+    def run(self):
+        while self._running:
+            try:
+                if self._queue.has_completed_elements():
+                    self._queue.emit_completed_element(self._output_processor)
+                else:
+                    self._queue.wait_for_completed_elements()
+            except Exception as e:
+                self._running = False
+                self._exception_handler(e)
+
+    def stop(self):
+        self._running = False
+
+class AsyncFunctionRunner(threading.Thread):
+    def __init__(self, exception_handler):
+        super().__init__()
+        self._exception_handler = exception_handler
+        self._loop = None
+
+    def run(self):
+        self._loop = asyncio.new_event_loop()
+        asyncio.set_event_loop(self._loop)
+        self._loop.run_forever()
+
+    def stop(self):
+        if self._loop is not None:
+            self._loop.stop()
+            self._loop = None
+
+    async def exception_handler_wrapper(self, async_function, *arg):
+        try:
+            await async_function(*arg)
+        except Exception as e:
+            self._exception_handler(e)
+
+    def run_async(self, async_function, *arg):
+        wrapped_function = self.exception_handler_wrapper(async_function, *arg)
+        asyncio.run_coroutine_threadsafe(wrapped_function, self._loop)
+
+class AsyncOperation(Operation):
+    def __init__(self, serialized_fn, operator_state_backend):
+        super(AsyncOperation, self).__init__(serialized_fn, 
operator_state_backend)
+        (
+            self.class_name,
+            self.open_func,
+            self.close_func,
+            self.async_invoke_func,
+            self.timeout_func,
+            self._timeout,
+            capacity,
+            output_mode
+        ) = extract_async_function(
+            user_defined_function_proto=serialized_fn,
+            runtime_context=StreamingRuntimeContext.of(
+                serialized_fn.runtime_context, self.base_metric_group
+            )
+        )
+        self._retry_enabled = False
+        if output_mode == AsyncFunctionDescriptor.OutputMode.UNORDERED:
+            self._queue = UnorderedStreamElementQueue(capacity, 
self._raise_exception_if_exists)
+        else:
+            raise NotImplementedError()
+        self._emitter = None
+        self._async_function_runner = None
+        self._exception = None
+
+    def set_output_processor(self, output_processor):
+        self._output_processor = output_processor
+
+    def open(self):
+        self.open_func()
+        self._emitter = Emitter(self._mark_exception, self._output_processor, 
self._queue)
+        self._emitter.daemon = True
+        self._emitter.start()
+
+        self._async_function_runner = AsyncFunctionRunner(self._mark_exception)
+        self._async_function_runner.daemon = True
+        self._async_function_runner.start()
+
+    def close(self):
+        self.close_func()
+        if self._emitter is not None:
+            self._emitter.stop()
+            self._emitter = None
+
+        if self._async_function_runner is not None:
+            self._async_function_runner.stop()
+            self._async_function_runner = None
+
+    def process_element(self, windowed_value, element):

Review Comment:
   i wonder what will happen if the element is a watermark record?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to