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


##########
flink-python/pyflink/fn_execution/datastream/process/async_function/operation.py:
##########
@@ -0,0 +1,289 @@
+################################################################################
+#  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, Callable
+
+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
+
+IN = TypeVar('IN')
+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[IN, OUT]):
+
+    def __init__(self,
+                 classname: str,
+                 timeout_func: Callable[[IN, ResultFuture[[OUT]]], None],
+                 exception_handler: Callable[[Exception], None],
+                 record: IN,
+                 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)

Review Comment:
   It looks like we can move 'RuntimeError' int the previous "else" instead of 
doing another `if not isinstance(result, Iterable)` again, Like
   ```
           if not isinstance(result, Iterable):
               # complete with empty result, so that we remove timer and move 
ahead processing
               self._process_results([])
               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)
           else:
               self._process_results(result)
   
   ```



##########
flink-python/pyflink/fn_execution/datastream/process/async_function/queue.py:
##########
@@ -0,0 +1,345 @@
+################################################################################
+#  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 collections
+import threading
+from abc import ABC
+from typing import Generic, TypeVar, List
+
+from pyflink.datastream import ResultFuture
+from pyflink.fn_execution.datastream.process.async_function import 
LONG_MIN_VALUE
+from pyflink.fn_execution.datastream.process.input_handler import _emit_results
+
+OUT = TypeVar('OUT')
+
+
+class StreamElementQueueEntry(ABC, ResultFuture, Generic[OUT]):
+    """
+    An entry for the StreamElementQueue. The stream element queue entry stores 
the
+    StreamElement for which the stream element queue entry has been 
instantiated. Furthermore, it
+    allows to set the result of a completed entry through ResultFuture.
+    """
+
+    def is_done(self) -> bool:
+        """
+        True if the stream element queue entry has been completed; otherwise 
false.
+        """
+        pass
+
+    def emit_result(self, output_processor) -> int:
+        """
+        Emits the results associated with this queue entry.
+
+        :return: The number of popped input elements.
+        """
+        pass
+
+    def complete_exceptionally(self, error: Exception):
+        """
+        Exceptions should be handled in the ResultHandler.
+        """
+        raise Exception("This result future should only be used to set 
completed results.")
+
+
+class StreamRecordQueueEntry(StreamElementQueueEntry):
+    """
+    StreamElementQueueEntry implementation for StreamRecord. This class also 
acts as
+    the ResultFuture implementation which is given to the AsyncFunction. The 
async
+    function completes this class with a collection of results.
+    """
+
+    def __init__(self, windowed_value, timestamp, watermark, record):
+        self._windowed_value = windowed_value
+        self._record = record
+        self._timestamp = timestamp
+        self._watermark = watermark
+        self._completed_results = None
+        self._on_complete_handler = None
+
+    def is_done(self) -> bool:
+        return self._completed_results is not None
+
+    def emit_result(self, output_processor):
+        output_processor.process_outputs(
+            self._windowed_value,
+            _emit_results(self._timestamp, self._watermark, 
self._completed_results, False))
+        return 1
+
+    def on_complete(self, handler):
+        self._on_complete_handler = handler
+
+    def complete(self, result: List[OUT]):
+        self._completed_results = result
+        if self._on_complete_handler is not None:
+            self._on_complete_handler(self)
+
+
+class WatermarkQueueEntry(StreamElementQueueEntry):
+    """
+    StreamElementQueueEntry implementation for Watermark.
+    """
+
+    def __init__(self, watermark):
+        self._watermark = watermark
+
+    def is_done(self) -> bool:
+        return True
+
+    def emit_result(self, output_processor):
+        # watermark will be passed together with the record
+        return 0
+
+    def complete(self, result: List[OUT]):
+        raise Exception("Cannot complete a watermark.")
+
+
+class StreamElementQueue(ABC, Generic[OUT]):
+
+    def put(self, windowed_value, timestamp, watermark, record) -> 
ResultFuture[OUT]:
+        """
+        Put the given record in the queue. This operation blocks until the 
queue has
+        capacity left.
+
+        This method returns a handle to the inserted element that allows to 
set the result of the
+        computation.
+
+        :param windowed_value: The windowed value for the record to be 
inserted.
+        :param timestamp: The timestamp of the record to be inserted.
+        :param watermark: The watermark of the record to be inserted.
+        :param record: The actual record to be inserted.
+        :return: A handle to the element.
+        """
+        pass
+
+    def advance_watermark(self, watermark):
+        """
+        Tries to put the given watermark in the queue. This operation succeeds 
if the queue has
+        capacity left and fails if the queue is full.
+
+        :param watermark: The watermark to be inserted.
+        """
+        pass
+
+    def emit_completed_element(self, output_processor):
+        """
+        Emits one completed element from the head of this queue into the given 
output.
+
+        Will not emit any element if no element has been completed.
+        """
+        pass
+
+    def has_completed_elements(self) -> bool:
+        """
+        Checks if there is at least one completed head element.
+        """
+        pass
+
+    def wait_for_completed_elements(self):
+        """
+        Waits until there is completed elements.
+        """
+        pass
+
+    def wait_for_in_flight_elements_processed(self):
+        """
+        Waits until any inflight elements have been processed.
+        """
+        pass
+
+    def is_empty(self) -> bool:
+        """
+        True if the queue is empty; otherwise false.
+        """
+        pass
+
+    def size(self) -> int:
+        """
+        Return the size of the queue.
+        """
+        pass
+
+
+class UnorderedStreamElementQueue(StreamElementQueue):
+
+    class Segment(object):
+
+        def __init__(self, capacity):
+            self._incomplete_elements = set()
+            self._complete_elements = collections.deque(maxlen=capacity)

Review Comment:
   maybe what we mean here is "completed" (i.e. something is finished) instead 
of "complete"(i.e. entire)



##########
flink-python/pyflink/fn_execution/datastream/process/async_function/operation.py:
##########
@@ -0,0 +1,289 @@
+################################################################################
+#  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, Callable
+
+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
+
+IN = TypeVar('IN')
+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[IN, OUT]):
+
+    def __init__(self,
+                 classname: str,
+                 timeout_func: Callable[[IN, ResultFuture[[OUT]]], None],
+                 exception_handler: Callable[[Exception], None],
+                 record: IN,
+                 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: Callable[[Exception], None],
+                 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: Callable[[Exception], None]):
+        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()

Review Comment:
   Should we do `self._loop.close()` to free underlying resources?
   ```suggestion
               self._loop.stop()
               self._loop.close()
   ```



##########
flink-python/pyflink/fn_execution/datastream/process/async_function/operation.py:
##########
@@ -0,0 +1,289 @@
+################################################################################
+#  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, Callable
+
+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
+
+IN = TypeVar('IN')
+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[IN, OUT]):
+
+    def __init__(self,
+                 classname: str,
+                 timeout_func: Callable[[IN, ResultFuture[[OUT]]], None],
+                 exception_handler: Callable[[Exception], None],
+                 record: IN,
+                 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: Callable[[Exception], None],
+                 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: Callable[[Exception], None]):
+        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)

Review Comment:
   I want to make sure I understand the error handling here correctly so I try 
to summarize the error handling process with a simple case:
   1. there are 3 records that should be processed by 
`AsyncOperation`'s`process_element`.
   2. `self._async_function_runner.run_async` is called for element1 in main 
thread.
   3. `self._async_function_runner.run_async` is called for element2 in main 
thread but the task has not been submitted to the`loop` thread(i.e. line 172 
has not been executed).
   4. the`loop` thread executes the processing of element1, then exception 
happens, `AsyncOperation` marks exception.
   5. `AsyncOperation` in main threads submits the element2 to `loop` thread.
   6. `AsyncOperation` in main thread finds exception when processing element3 
and raises. After some population, the main thread calls `AsyncOperation`'s 
close() and stops the `loop` thread.
   According to the doc of async io:
   <img width="2078" height="672" alt="image" 
src="https://github.com/user-attachments/assets/287947cc-8a32-4ac2-bddf-785c17811f37";
 />
   if the `loop` thread has not process element2's task, then the thread just 
stops and no extra processing for element2. 
   If the `loop` thread is processing element2, the `loop` thread will finish 
the processing. But  as AsyncOperation has been closed, is it possible that the 
`loop` thread runs into some bad cases(e.g. if the async function uses some 
freed resources setup in advance)?



##########
flink-python/pyflink/fn_execution/datastream/process/async_function/operation.py:
##########
@@ -0,0 +1,289 @@
+################################################################################
+#  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, Callable
+
+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
+
+IN = TypeVar('IN')
+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[IN, OUT]):
+
+    def __init__(self,
+                 classname: str,
+                 timeout_func: Callable[[IN, ResultFuture[[OUT]]], None],
+                 exception_handler: Callable[[Exception], None],
+                 record: IN,
+                 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: Callable[[Exception], None],
+                 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: Callable[[Exception], None]):
+        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

Review Comment:
   do we need `self._exception = None` here?
   In other words, when failover happens, will we reuse `AsyncOperation` object 
or just create a new one?



##########
flink-python/pyflink/fn_execution/datastream/process/async_function/operation.py:
##########
@@ -0,0 +1,289 @@
+################################################################################
+#  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, Callable
+
+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
+
+IN = TypeVar('IN')
+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[IN, OUT]):
+
+    def __init__(self,
+                 classname: str,
+                 timeout_func: Callable[[IN, ResultFuture[[OUT]]], None],
+                 exception_handler: Callable[[Exception], None],
+                 record: IN,
+                 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: Callable[[Exception], None],
+                 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: Callable[[Exception], None]):
+        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()

Review Comment:
   ```suggestion
               raise NotImplementedError("ORDERED mode is supported.")
   ```



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