AHeise commented on a change in pull request #14573:
URL: https://github.com/apache/flink/pull/14573#discussion_r553159576



##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxDefaultAction.java
##########
@@ -59,6 +60,16 @@
          * Calling this method signals that the mailbox-thread should 
(temporarily) stop invoking
          * the default action, e.g. because there is currently no input 
available. This method must
          * be invoked from the mailbox-thread only!
+         *
+         * @param suspensionIdleTimer started (ticking) {@link TimerGauge} 
that measures how long
+         *     the default action was suspended/idling. If mailbox loop is 
busy processing mails,
+         *     this timer should be paused for the time required to process 
the mails.
+         */
+        Suspension suspendDefaultAction(TimerGauge suspensionIdleTimer);
+
+        /**
+         * Same as {@link #suspendDefaultAction(TimerGauge)} but without any 
associated timer
+         * measuring the idle time.
          */
         Suspension suspendDefaultAction();

Review comment:
       Why do we need to retain this signature? Just for legacy sources?
   
   Since this is internal API, we could also directly expose `@Nullable 
TimerGauge` (the implementation is relaying anyways). You could also transform 
this method into a default method with `suspendDefaultAction(null)`.

##########
File path: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
##########
@@ -1256,49 +1307,82 @@ public void testProcessWithUnAvailableOutput() throws 
Exception {
 
             final RunnableWithException completeFutureTask =
                     () -> {
-                        Thread.sleep(sleepTime + 1);
                         assertEquals(1, inputProcessor.currentNumProcessCalls);
                         
assertTrue(task.mailboxProcessor.isDefaultActionUnavailable());
                         
environment.getWriter(1).getAvailableFuture().complete(null);
                     };
 
+            // Make sure WaitingThread is started after Task starts processing.
             executor.submit(
-                    () -> {
-                        executor.submit(
-                                completeFutureTask,
-                                "This task will complete the future to resume 
process input action.");
-                    },
+                    () ->
+                            new WaitingThread(
+                                            executor,
+                                            completeFutureTask,
+                                            sleepTimeInsideMail,
+                                            sleepTimeOutsideMail)
+                                    .start(),
                     "This task will submit another task to execute after 
processing input once.");
 
+            long startTs = System.currentTimeMillis();
             TaskIOMetricGroup ioMetricGroup =
                     task.getEnvironment().getMetricGroup().getIOMetricGroup();
             task.invoke();
+            long totalDuration = System.currentTimeMillis() - startTs;
             assertThat(
                     ioMetricGroup.getBackPressuredTimePerSecond().getCount(),
-                    Matchers.greaterThanOrEqualTo(sleepTime));
+                    Matchers.greaterThanOrEqualTo(sleepTimeOutsideMail));
+            assertThat(
+                    ioMetricGroup.getBackPressuredTimePerSecond().getCount(),
+                    Matchers.lessThanOrEqualTo(totalDuration - 
sleepTimeInsideMail));

Review comment:
       How reliable is this test?
   I guess since we compare to the actually measured `totalDuration`, it's 
stable also in situations like GC pauses.
   
   But is it also effectively testing? What happens if `totalDuration` is much 
larger than any of the sleep times?
   (I just want to avoid any test instabilities by explicitly going over the 
cases)

##########
File path: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
##########
@@ -1256,49 +1307,82 @@ public void testProcessWithUnAvailableOutput() throws 
Exception {
 
             final RunnableWithException completeFutureTask =
                     () -> {
-                        Thread.sleep(sleepTime + 1);
                         assertEquals(1, inputProcessor.currentNumProcessCalls);
                         
assertTrue(task.mailboxProcessor.isDefaultActionUnavailable());
                         
environment.getWriter(1).getAvailableFuture().complete(null);
                     };
 
+            // Make sure WaitingThread is started after Task starts processing.
             executor.submit(
-                    () -> {
-                        executor.submit(
-                                completeFutureTask,
-                                "This task will complete the future to resume 
process input action.");
-                    },
+                    () ->
+                            new WaitingThread(
+                                            executor,
+                                            completeFutureTask,
+                                            sleepTimeInsideMail,
+                                            sleepTimeOutsideMail)
+                                    .start(),
                     "This task will submit another task to execute after 
processing input once.");
 
+            long startTs = System.currentTimeMillis();
             TaskIOMetricGroup ioMetricGroup =
                     task.getEnvironment().getMetricGroup().getIOMetricGroup();
             task.invoke();
+            long totalDuration = System.currentTimeMillis() - startTs;
             assertThat(
                     ioMetricGroup.getBackPressuredTimePerSecond().getCount(),
-                    Matchers.greaterThanOrEqualTo(sleepTime));
+                    Matchers.greaterThanOrEqualTo(sleepTimeOutsideMail));
+            assertThat(
+                    ioMetricGroup.getBackPressuredTimePerSecond().getCount(),
+                    Matchers.lessThanOrEqualTo(totalDuration - 
sleepTimeInsideMail));
             assertThat(ioMetricGroup.getIdleTimeMsPerSecond().getCount(), 
is(0L));
             assertEquals(numberOfProcessCalls, 
inputProcessor.currentNumProcessCalls);
         }
     }
 
     @Test
     public void testProcessWithUnAvailableInput() throws Exception {
-        final long unAvailableTime = 42;
+        final long sleepTimeOutsideMail = 42;
+        final long sleepTimeInsideMail = 44;
         try (final MockEnvironment environment = setupEnvironment(new 
boolean[] {true, true})) {
             final UnAvailableTestInputProcessor inputProcessor =
-                    new UnAvailableTestInputProcessor(unAvailableTime);
+                    new UnAvailableTestInputProcessor();
             final StreamTask task =
                     new MockStreamTaskBuilder(environment)
                             .setStreamInputProcessor(inputProcessor)
                             .build();
 
+            final MailboxExecutor executor = 
task.mailboxProcessor.getMainMailboxExecutor();
+            final RunnableWithException completeFutureTask =
+                    () -> {
+                        inputProcessor
+                                .availabilityProvider
+                                .getUnavailableToResetAvailable()
+                                .complete(null);
+                    };
+
+            // Make sure WaitingThread is started after Task starts processing.
+            executor.submit(
+                    () ->
+                            new WaitingThread(

Review comment:
       Why do we need a thread instead of directly executing the content of the 
mail? Is it because we wouldn't be able to measure any idle time?

##########
File path: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
##########
@@ -1256,49 +1307,82 @@ public void testProcessWithUnAvailableOutput() throws 
Exception {
 
             final RunnableWithException completeFutureTask =
                     () -> {
-                        Thread.sleep(sleepTime + 1);
                         assertEquals(1, inputProcessor.currentNumProcessCalls);
                         
assertTrue(task.mailboxProcessor.isDefaultActionUnavailable());
                         
environment.getWriter(1).getAvailableFuture().complete(null);
                     };
 
+            // Make sure WaitingThread is started after Task starts processing.
             executor.submit(
-                    () -> {
-                        executor.submit(
-                                completeFutureTask,
-                                "This task will complete the future to resume 
process input action.");
-                    },
+                    () ->
+                            new WaitingThread(
+                                            executor,
+                                            completeFutureTask,
+                                            sleepTimeInsideMail,
+                                            sleepTimeOutsideMail)
+                                    .start(),
                     "This task will submit another task to execute after 
processing input once.");
 
+            long startTs = System.currentTimeMillis();
             TaskIOMetricGroup ioMetricGroup =
                     task.getEnvironment().getMetricGroup().getIOMetricGroup();
             task.invoke();
+            long totalDuration = System.currentTimeMillis() - startTs;
             assertThat(
                     ioMetricGroup.getBackPressuredTimePerSecond().getCount(),
-                    Matchers.greaterThanOrEqualTo(sleepTime));
+                    Matchers.greaterThanOrEqualTo(sleepTimeOutsideMail));
+            assertThat(
+                    ioMetricGroup.getBackPressuredTimePerSecond().getCount(),
+                    Matchers.lessThanOrEqualTo(totalDuration - 
sleepTimeInsideMail));
             assertThat(ioMetricGroup.getIdleTimeMsPerSecond().getCount(), 
is(0L));
             assertEquals(numberOfProcessCalls, 
inputProcessor.currentNumProcessCalls);
         }
     }
 
     @Test
     public void testProcessWithUnAvailableInput() throws Exception {
-        final long unAvailableTime = 42;
+        final long sleepTimeOutsideMail = 42;
+        final long sleepTimeInsideMail = 44;
         try (final MockEnvironment environment = setupEnvironment(new 
boolean[] {true, true})) {

Review comment:
       nit: replace with vararg (I know it's not your code, but I happened to 
be confused now).




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

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to