You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by nk...@apache.org on 2021/08/27 08:49:43 UTC

[zookeeper] branch branch-3.7 updated: ZOOKEEPER-4327: RequestThrottlerTest: Flakiness remediations

This is an automated email from the ASF dual-hosted git repository.

nkalmar pushed a commit to branch branch-3.7
in repository https://gitbox.apache.org/repos/asf/zookeeper.git


The following commit(s) were added to refs/heads/branch-3.7 by this push:
     new 3491781  ZOOKEEPER-4327: RequestThrottlerTest: Flakiness remediations
3491781 is described below

commit 34917819b38377723fcc875ec52b1e09f2a1ec62
Author: Damien Diederen <dd...@apache.org>
AuthorDate: Fri Aug 27 10:48:40 2021 +0200

    ZOOKEEPER-4327: RequestThrottlerTest: Flakiness remediations
    
      * `testGlobalOutstandingRequestThrottlingWithRequestThrottlerDisabled`: Only resume after looking at metrics
    
        Resuming the pipeline before looking at the metrics triggers a race between the test and other server threads, causing us to sometimes observe a larger `prep_processor_request_queued` than expected.
    
        The corresponding failures look like this:
    
            RequestThrottlerTest.testGlobalOutstandingRequestThrottlingWithRequestThrottlerDisabled:340 expected: <3> but was: <7>
    
        The metric we look at is updated at the front of the pipeline, before the portion blocked by `resumeProcess`; we don't have to wait for the latter to observe the effect of the global outstanding limit.
    
      * `testLargeRequestThrottling`: Allow more time for request ingestion
    
        The 5s timeout used before this patch was sometimes not sufficient for the server to process the five requests of interest. The corresponding failures look like this:
    
            RequestThrottlerTest.testLargeRequestThrottling:297 expected: <2> but was: <0>
    
        or this:
    
            RequestThrottlerTest.testLargeRequestThrottling:299 expected: <5> but was: <4>
    
        We can use a higher timeout value as the `await` normally quickly terminates with the countdown reaching zero (as opposed to timing out); the increase in duration is only paid on loaded (and slow) executors.
    
    Author: Damien Diederen <dd...@apache.org>
    
    Reviewers: Enrico Olivelli <eo...@apache.org>, Norbert Kalmar <nk...@apache.org>
    
    Closes #1739 from ztzg/ZOOKEEPER-4327-flaky-request-throttler-test
    
    (cherry picked from commit 1f5e4fe258707f9b82448a5f97c931c392d011ca)
    Signed-off-by: Norbert Kalmar <nk...@apache.org>
---
 .../test/java/org/apache/zookeeper/server/RequestThrottlerTest.java | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)

diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/RequestThrottlerTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/RequestThrottlerTest.java
index 58e138a..62d371d 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/RequestThrottlerTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/RequestThrottlerTest.java
@@ -289,7 +289,7 @@ public class RequestThrottlerTest extends ZKTestCase {
         }
 
         // make sure the server received all 5 requests
-        disconnected.await(5, TimeUnit.SECONDS);
+        disconnected.await(30, TimeUnit.SECONDS);
         Map<String, Object> metrics = MetricsUtils.currentServerMetrics();
 
         // but only two requests can get into the pipeline because they are large requests
@@ -329,8 +329,6 @@ public class RequestThrottlerTest extends ZKTestCase {
 
             submitted.await(5, TimeUnit.SECONDS);
 
-            resumeProcess.countDown();
-
             // We should start throttling instead of queuing more requests.
             //
             // We always allow up to GLOBAL_OUTSTANDING_LIMIT + 1 number of requests coming in request processing pipeline
@@ -339,6 +337,8 @@ public class RequestThrottlerTest extends ZKTestCase {
             // be GLOBAL_OUTSTANDING_LIMIT + 2.
             assertEquals(Integer.parseInt(GLOBAL_OUTSTANDING_LIMIT) + 2,
                     (long) MetricsUtils.currentServerMetrics().get("prep_processor_request_queued"));
+
+            resumeProcess.countDown();
         } catch (Exception e) {
             throw e;
         } finally {