AHeise commented on a change in pull request #14573:
URL: https://github.com/apache/flink/pull/14573#discussion_r553252820
##########
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:
Okay makes sense, we could still try to deprecate
`suspendDefaultAction`. I don't see it being used outside of legacy sources and
I kinda consider it a hack there.
##########
File path:
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
##########
@@ -1241,9 +1241,60 @@ public void testProcessWithAvailableOutput() throws
Exception {
}
}
+ /**
+ * In this weird construct, we are:
+ *
+ * <ul>
+ * <li>1. We start a thread, which will...
+ * <li>2. ... sleep for X ms, and enqueue another mail, that will...
+ * <li>3. ... sleep for Y ms, and make the output available again
+ * </ul>
+ *
+ * <p>2nd step is to check that back pressure or idle counter is at least
X. In the last 3rd
+ * step, we test whether this counter was paused for the duration of
processing mails.
+ */
+ private static class WaitingThread extends Thread {
+ private final MailboxExecutor executor;
+ private final RunnableWithException resumeTask;
+ private final long sleepTimeInsideMail;
+ private final long sleepTimeOutsideMail;
+
+ public WaitingThread(
+ MailboxExecutor executor,
+ RunnableWithException resumeTask,
+ long sleepTimeInsideMail,
+ long sleepTimeOutsideMail) {
+ this.executor = executor;
+ this.resumeTask = resumeTask;
+ this.sleepTimeInsideMail = sleepTimeInsideMail;
+ this.sleepTimeOutsideMail = sleepTimeOutsideMail;
+ }
+
+ @Override
+ public void run() {
+ try {
+ Thread.sleep(sleepTimeOutsideMail);
+ } catch (InterruptedException e) {
+ e.printStackTrace();
Review comment:
Wouldn't the test fail on a spurious wake-up? Might be hard to debug if
you swallow the exception here. Would it be better to LOG?
Same inside the execute.
##########
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:
Okay, I just wanted to double check (after recent test instabilities).
##########
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:
I was thinking if it's safer to also join the `WaitingThread` after
`task.invoke();` but I guess that's more or less done through the
`completeFutureTask` (should that be called in a `finally`?).
----------------------------------------------------------------
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:
[email protected]