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


##########
flink-python/pyflink/datastream/async_data_stream.py:
##########
@@ -47,14 +48,39 @@ def unordered_wait(
         :param output_type: The output data type.
         :return: The transformed DataStream.
         """
-        AsyncDataStream._validate(data_stream, async_function)
+        return AsyncDataStream.unordered_wait_with_retry(
+            data_stream, async_function, timeout, 
async_retry_strategies.NO_RETRY_STRATEGY,
+            capacity, output_type)
+
+    @staticmethod
+    def unordered_wait_with_retry(

Review Comment:
   why do we prefer introducing `unordered_wait_with_retry` instead of adding a 
new parameter(AsyncRetryStrategy, default is None) to `unordered_wait`



##########
flink-python/pyflink/fn_execution/datastream/process/async_function/operation.py:
##########
@@ -117,6 +118,83 @@ def _timer_triggered(self):
             self._timeout_func(self._record, self)
 
 
+class RetryableResultHandler(ResultFuture, Generic[IN, OUT]):
+
+    def __init__(self,
+                 result_handler: ResultHandler[IN, OUT],
+                 async_invoke_func_runner: Callable[[IN, ResultFuture[[OUT]]], 
None],
+                 retry_strategy: AsyncRetryStrategy[OUT]):
+        self._result_handler = result_handler
+        self._async_invoke_func_runner = async_invoke_func_runner
+        self._retry_strategy = retry_strategy
+        self._retry_result_predicate = \
+            retry_strategy.get_retry_predicate().result_predicate() or (lambda 
_: False)
+        self._retry_exception_predicate = \
+            retry_strategy.get_retry_predicate().exception_predicate() or 
(lambda _: False)
+        self._retry_awaiting = AtomicBoolean(False)
+        self._current_attempts = 1
+
+    def register_timeout(self, timeout):
+        timer = threading.Timer(timeout, self._timer_triggered)
+        timer.start()
+        self._result_handler._timer = timer
+        self._start_ts = datetime.now()
+        self._timeout = timeout
+
+    def complete(self, result: List[OUT]):
+        self._process_retry(result, None)
+
+    def complete_exceptionally(self, error: Exception):
+        self._process_retry(None, error)
+
+    def _process_retry(self, result: Optional[List[OUT]], error: 
Optional[Exception]):
+        if not self._retry_awaiting.compare_and_set(False, True):
+            return
+
+        satisfy = ((result is not None and 
self._retry_result_predicate(result)) or
+                   (error is not None and 
self._retry_exception_predicate(error)))
+
+        if (not self._is_timeout() and satisfy and
+                self._retry_strategy.can_retry(self._current_attempts)):
+

Review Comment:
   extra blank line



##########
flink-python/pyflink/datastream/tests/test_async_function.py:
##########
@@ -226,6 +226,46 @@ def timeout(self, value: Row, result_future: 
ResultFuture[int]):
         expected = ['0', '0', '0', '0', '0']
         self.assert_equals_sorted(expected, results)
 
+    def test_async_with_retry(self):
+        self.env.set_parallelism(1)
+        ds = self.env.from_collection(
+            [(1, 1), (2, 2), (3, 3), (4, 4), (5, 5)],
+            type_info=Types.ROW_NAMED(["v1", "v2"], [Types.INT(), Types.INT()])
+        )
+
+        class MyAsyncFunction(AsyncFunction):
+
+            def __init__(self):
+                self.retries = {}
+
+            async def async_invoke(self, value: Row, result_future: 
ResultFuture[int]):
+                await asyncio.sleep(1)
+                if value in self.retries:
+                    result_future.complete([value[0] + value[1]])
+                else:
+                    self.retries[value] = True
+                    result_future.complete_exceptionally(ValueError("failed 
the first time"))
+
+            def timeout(self, value: Row, result_future: ResultFuture[int]):
+                result_future.complete([value[0] + value[1]])
+
+        def exception_predicate(exception: Exception):
+            return "failed the first time" in str(exception)
+
+        async_retry_strategy = AsyncRetryStrategy.fixed_delay(
+            max_attempts=5,
+            backoff_time_millis=1000,
+            result_predicate=None,

Review Comment:
   It could be nicer if we add a test for result_predicate



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