gaoyunhaii commented on code in PR #21546: URL: https://github.com/apache/flink/pull/21546#discussion_r1090468404
########## flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java: ########## @@ -442,36 +456,50 @@ public RetryableResultHandlerDelegator( this.processingTimeService = processingTimeService; } - public void registerTimeout(long timeout) { - resultHandler.registerTimeout(processingTimeService, timeout); + private void registerTimeout(long timeout) { + resultHandler.timeoutTimer = + registerTimer(processingTimeService, timeout, t -> timerTriggered()); + } + + private void cancelRetryTimer() { + if (delayedRetryTimer != null) { + delayedRetryTimer.cancel(true); Review Comment: We may use false here to avoid interrupt the timer thread unexpectically and cause possible failures. ########## flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java: ########## @@ -1260,6 +1263,79 @@ private void testProcessingTimeWithRetry( } } + /** + * Test the AsyncWaitOperator with an always-timeout async function under unordered mode and + * processing time. + */ + @Test + public void testProcessingTimeWithTimeoutFunctionUnorderedWithRetry() throws Exception { + testProcessingTimeAlwaysTimeoutFunctionWithRetry(AsyncDataStream.OutputMode.UNORDERED); + } + + /** + * Test the AsyncWaitOperator with an always-timeout async function under ordered mode and + * processing time. + */ + @Test + public void testProcessingTimeWithTimeoutFunctionOrderedWithRetry() throws Exception { + testProcessingTimeAlwaysTimeoutFunctionWithRetry(AsyncDataStream.OutputMode.ORDERED); + } + + private void testProcessingTimeAlwaysTimeoutFunctionWithRetry(AsyncDataStream.OutputMode mode) + throws Exception { + + StreamTaskMailboxTestHarnessBuilder<Integer> builder = + new StreamTaskMailboxTestHarnessBuilder<>( + OneInputStreamTask::new, BasicTypeInfo.INT_TYPE_INFO) + .addInput(BasicTypeInfo.INT_TYPE_INFO); + + AsyncRetryStrategy exceptionRetryStrategy = + new AsyncRetryStrategies.FixedDelayRetryStrategyBuilder(5, 100L) + .ifException(RetryPredicates.HAS_EXCEPTION_PREDICATE) + .build(); + AlwaysTimeoutWithDefaultValueAsyncFunction asyncFunction = + new AlwaysTimeoutWithDefaultValueAsyncFunction(); + + try (StreamTaskMailboxTestHarness<Integer> testHarness = + builder.setupOutputForSingletonOperatorChain( + new AsyncWaitOperatorFactory<>( + asyncFunction, TIMEOUT, 10, mode, exceptionRetryStrategy)) + .build()) { + + final long initialTime = 0L; + final Queue<Object> expectedOutput = new ArrayDeque<>(); + + testHarness.processElement(new StreamRecord<>(1, initialTime + 1)); + testHarness.processElement(new StreamRecord<>(2, initialTime + 2)); + + expectedOutput.add(new StreamRecord<>(-1, initialTime + 1)); + expectedOutput.add(new StreamRecord<>(-1, initialTime + 2)); + + Deadline deadline = Deadline.fromNow(Duration.ofSeconds(10)); + while (testHarness.getOutput().size() < expectedOutput.size() + && deadline.hasTimeLeft()) { + testHarness.processAll(); + //noinspection BusyWait + Thread.sleep(100); + } + + if (mode == AsyncDataStream.OutputMode.ORDERED) { + TestHarnessUtil.assertOutputEquals( + "ORDERED Output was not correct.", expectedOutput, testHarness.getOutput()); + } else { + TestHarnessUtil.assertOutputEqualsSorted( + "UNORDERED Output was not correct.", + expectedOutput, + testHarness.getOutput(), + new StreamRecordComparator()); + } + + // verify the elements' try count + assertTrue(asyncFunction.getTryCount(1) == 2); Review Comment: Here we relies on TIMEOUT and the `sleep(500)` in the testing async function. In consideration of the status of the testing machine, there might be drift for the timers, which might cause the counts to be other values. For now we may first make it <= 2, and see if the 500 slack is large enough for the other side. If there are failures, we may need to change to some kind of more deterministic logic. -- 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: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org