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


##########
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:
   Added `self._loop.close()` in method `run`.



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