Skip to content

Commit

Permalink
[FLINK-27042][metrics] Fix instability of StreamTaskTest#testMailboxM…
Browse files Browse the repository at this point in the history
…etricsScheduling

Remove assertion for latency measurement from StreamTaskTest#testMailboxMetricsScheduling as it
causes instability and is already covered in StreamTaskTest#testMailboxMetricsMeasurement.
  • Loading branch information
Sebastian Mattheis committed Apr 5, 2022
1 parent 3eb762b commit 450a142
Showing 1 changed file with 3 additions and 8 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -1769,9 +1769,8 @@ public long calculateThroughput() {

/**
* Tests mailbox metrics latency and queue size and verifies that (1) latency measurement is
* executed initially once and at least once triggered by timer, (2) latency max value is
* greater than zero and (3) mailbox size is greater than zero for some time and eventually
* equals to zero.
* executed once initially and at least once triggered by timer and (2) mailbox size is greater
* than zero for some time and eventually equals to zero.
*
* @throws Exception on {@link MockEnvironmentBuilder#build()} failure.
*/
Expand Down Expand Up @@ -1800,10 +1799,7 @@ protected void processInput(
mailboxProcessor
.getMainMailboxExecutor()
.execute(() -> {}, "mail");
// The actual delay here is irrelevant for the test but
// delay should be at least once 10 ms to reach a measurable
// delay >~ 8 ms.
Thread.sleep(mailboxLatencyMetric.getCount() == 0 ? 10 : 1);
Thread.sleep(1);
} else {
controller.suspendDefaultAction();
mailboxProcessor.suspend();
Expand All @@ -1820,7 +1816,6 @@ protected void processInput(
assertThat(
mailboxLatencyMetric.getCount(),
greaterThanOrEqualTo(new Long(minMeasurements)));
assertThat(mailboxLatencyMetric.getStatistics().getMax(), greaterThan(0L));
assertThat(maxMailboxSize.get(), greaterThan(0));
assertThat(mailboxSizeMetric.getValue(), equalTo(0));
}
Expand Down

0 comments on commit 450a142

Please sign in to comment.