Skip to content

Commit

Permalink
Fix a race condition in WatcherCleanerTest.testDeadWatcherMetrics
Browse files Browse the repository at this point in the history
Because the metrics were updated _after_ the listener is invoked, the listener does not always see
the fresh metric value. This fixes it so that the updated metric value is always visible to the
listener.
  • Loading branch information
PapaCharlie committed Mar 8, 2023
1 parent f46b8fb commit 4f84cf3
Showing 1 changed file with 20 additions and 3 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -24,13 +24,15 @@
import static org.junit.jupiter.api.Assertions.assertTrue;
import java.util.HashSet;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import org.apache.zookeeper.ZKTestCase;
import org.apache.zookeeper.common.Time;
import org.apache.zookeeper.metrics.MetricsUtils;
import org.apache.zookeeper.server.ServerMetrics;
import org.junit.jupiter.api.RepeatedTest;
import org.junit.jupiter.api.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -139,8 +141,10 @@ public void testMaxInProcessingDeadWatchers() {
assertTrue(listener.wait(5000));
}

@Test
public void testDeadWatcherMetrics() {
// There used to be a race condition surrounding this test which was reproducible by running the test multiple
// times. This test is kept as repeated to flag if the race condition reappears.
@RepeatedTest(5)
public void testDeadWatcherMetrics() throws InterruptedException {
ServerMetrics.getMetrics().resetAll();
MyDeadWatcherListener listener = new MyDeadWatcherListener();
WatcherCleaner cleaner = new WatcherCleaner(listener, 1, 1, 1, 1);
Expand All @@ -158,7 +162,9 @@ public void testDeadWatcherMetrics() {
Map<String, Object> values = MetricsUtils.currentServerMetrics();
assertThat("Adding dead watcher should be stalled twice", (Long) values.get("add_dead_watcher_stall_time"), greaterThan(0L));
assertEquals(3L, values.get("dead_watchers_queued"), "Total dead watchers added to the queue should be 3");
assertEquals(3L, values.get("dead_watchers_cleared"), "Total dead watchers cleared should be 3");
// This metric is updated _after_ the dead watcher listener is invoked, so it is not always immediately visible,
// hence the wait.
waitForMetricValue("dead_watchers_cleared", 3L, 5_000);

assertEquals(3L, values.get("cnt_dead_watchers_cleaner_latency"));

Expand All @@ -171,4 +177,15 @@ public void testDeadWatcherMetrics() {
assertEquals(20D, ((Long) values.get("p99_dead_watchers_cleaner_latency")).doubleValue(), 20);
}

/**
* Waits in a loop for the given metric to have the required value. If the given timeout is reached, the test fails.
*/
private static void waitForMetricValue(String metricName, Object expected, long timeoutMs) throws InterruptedException {
long start = Time.currentElapsedTime();
while (!Objects.equals(MetricsUtils.currentServerMetrics().get(metricName), expected)) {
Thread.sleep(100);
assertFalse(Time.currentElapsedTime() - start > timeoutMs,
"Metric value was not updated in " + timeoutMs + "ms!");
}
}
}

0 comments on commit 4f84cf3

Please sign in to comment.