Skip to content

Commit

Permalink
ZOOKEEPER-4327: Fix flaky RequestThrottlerTest.testLargeRequestThrott…
Browse files Browse the repository at this point in the history
…ling

This test failed following assertions in ci:
1. `RequestThrottlerTest.testRequestThrottler:206 expected: <5> but was: <4>`

   This is caused by no happens-before relationship between
   `connectionLossCount` and `disconnected.await`. Places
   `disconnected.countDown()` after `connectionLossCount++` to solve
   this.

2. `RequestThrottlerTest.testLargeRequestThrottling:297 expected: <2> but was: <0>`

   Large request throttling is handled in io thread, while
   `prep_processor_request_queued` metric is updated in processor
   thread. Places metric assertion after `finished.await` to solve this.

Additionally, I find one more potential flaky case. After connection
closed due to throttling third request, reconnecting could fail this
test in slow sending environment. It is easy to reproduce by adding
`Thread.sleep(i * 100)` in sending loop.
  • Loading branch information
kezhuw committed Apr 2, 2022
1 parent 2173c92 commit 3df34b8
Showing 1 changed file with 25 additions and 18 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -261,13 +261,22 @@ public void testLargeRequestThrottling() throws Exception {

AsyncCallback.StringCallback createCallback = (rc, path, ctx, name) -> {
if (KeeperException.Code.get(rc) == KeeperException.Code.CONNECTIONLOSS) {
disconnected.countDown();
connectionLossCount++;
disconnected.countDown();
}
};

// we allow five requests in the pipeline
RequestThrottler.setMaxRequests(5);
// the total length of the request is about 170-180 bytes, so only two requests are allowed
byte[] data = new byte[100];
// the third request will incur throttle. We don't send more requests to avoid reconnecting
// due to unstable test environment(e.g. slow sending).
int number_requests = 3;

// we allow more requests in the pipeline
RequestThrottler.setMaxRequests(number_requests + 2);

// request could become stale in processor threads due to throttle in io thread
RequestThrottler.setDropStaleRequests(false);

// enable large request throttling
zks.setLargeRequestThreshold(150);
Expand All @@ -277,34 +286,32 @@ public void testLargeRequestThrottling() throws Exception {
resumeProcess = new CountDownLatch(1);
// the connection will be close when large requests exceed the limit
// we can't use the submitted latch because requests after close won't be submitted
disconnected = new CountDownLatch(TOTAL_REQUESTS);

// the total length of the request is about 170-180 bytes, so only two requests are allowed
byte[] data = new byte[100];
disconnected = new CountDownLatch(number_requests);

// send 5 requests asynchronously
for (int i = 0; i < TOTAL_REQUESTS; i++) {
// send requests asynchronously
for (int i = 0; i < number_requests; i++) {
zk.create("/request_throttle_test- " + i , data,
ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT, createCallback, null);
}

// make sure the server received all 5 requests
disconnected.await(30, TimeUnit.SECONDS);
// make sure the server received all requests
assertTrue(disconnected.await(30, TimeUnit.SECONDS));

finished = new CountDownLatch(2);
// let the requests go through the pipeline
resumeProcess.countDown();
assertTrue(finished.await(5, TimeUnit.SECONDS));

// assert metrics after finished so metrics in no io threads are set also.
Map<String, Object> metrics = MetricsUtils.currentServerMetrics();

// but only two requests can get into the pipeline because they are large requests
// the connection will be closed
assertEquals(2L, (long) metrics.get("prep_processor_request_queued"));
assertEquals(1L, (long) metrics.get("large_requests_rejected"));
assertEquals(5, connectionLossCount);

finished = new CountDownLatch(2);
// let the requests go through the pipeline
resumeProcess.countDown();
finished.await(5, TimeUnit.SECONDS);
assertEquals(number_requests, connectionLossCount);

// when the two requests finish, they are stale because the connection is closed already
metrics = MetricsUtils.currentServerMetrics();
assertEquals(2, (long) metrics.get("stale_replies"));
}

Expand Down

0 comments on commit 3df34b8

Please sign in to comment.