You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@zookeeper.apache.org by karanmehta93 <gi...@git.apache.org> on 2017/07/12 04:16:47 UTC

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

GitHub user karanmehta93 opened a pull request:

    https://github.com/apache/zookeeper/pull/307

    ZOOKEEPER-2770 ZooKeeper slow operation log

    The patch sets a default threshold of 10 sec for request. If the value of warn.responseTime is configured as -1 then logging is ignored.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/karanmehta93/zookeeper ZOOKEEPER-2770

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/zookeeper/pull/307.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #307
    
----
commit d09f72a8fca1764e2ed4c1861746af9a7fd5d15a
Author: Karan Mehta <ka...@gmail.com>
Date:   2017-05-02T21:20:39Z

    ZOOKEEPER-2770 ZooKeeper slow operation log

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r127048435
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java ---
    @@ -292,6 +293,8 @@ public void parseProperties(Properties zkProp)
                     }
                 } else if ((key.startsWith("server.") || key.startsWith("group") || key.startsWith("weight")) && zkProp.containsKey("dynamicConfigFile")) {
                     throw new ConfigException("parameter: " + key + " must be in a separate dynamic config file");
    +            } else if(key.equals("warn.responseTime")) {
    --- End diff --
    
    I will document the new property and change the name to the suggested one.
    I have tested this feature manually. 
    I am currently trying to add unit test case in NettyServerCnxnTest.java or NIOServerCnxnTest.java, but seems too high level approach.
    Can you suggest any locations for adding test case? @hanm 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r129431091
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java ---
    @@ -61,6 +61,7 @@
     
         private static boolean standaloneEnabled = true;
         private static boolean reconfigEnabled = false;
    +    private static int requestWarnThresholdMs = 10000;
    --- End diff --
    
    Is 2 or 3 seconds reasonable? I have seen 2.3 seconds as max latency sometimes, however I don't have much experience. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r126872020
  
    --- Diff: src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java ---
    @@ -430,6 +432,7 @@ public void processRequest(Request request) {
                 // the client and leader disagree on where the client is most
                 // recently attached (and therefore invalid SESSION MOVED generated)
                 cnxn.sendCloseSession();
    +            request.checkLatency();
    --- End diff --
    
    It was intended to measure end to end latency of a request. Hence latency check after the request is completed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on the issue:

    https://github.com/apache/zookeeper/pull/307
  
    From my understanding, `QuorumPeer` class will be the candidate class to maintain this data structure. Let me know alternatives. 
    I will look into the data-structure ideas you provided till then.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r147047400
  
    --- Diff: src/java/main/org/apache/zookeeper/server/ServerStats.java ---
    @@ -148,9 +174,46 @@ synchronized public void resetRequestCounters(){
             packetsReceived = 0;
             packetsSent = 0;
         }
    +    synchronized public void resetNumRequestsAboveThresholdTime() {
    +        numRequestsAboveThresholdTime = 0;
    +    }
         synchronized public void reset() {
             resetLatency();
             resetRequestCounters();
    +        resetNumRequestsAboveThresholdTime();
    +    }
    +
    +    public void checkLatency(final ZooKeeperServer zks, Request request) {
    +        long requestLatency = Time.currentElapsedTime() - request.createTime;
    +        boolean enabledAndAboveThreshold = (requestWarnThresholdMs == 0) ||
    +                (requestWarnThresholdMs > -1 && requestLatency > requestWarnThresholdMs);
    +        if (enabledAndAboveThreshold) {
    +            zks.serverStats().incNumRequestsAboveThresholdTime();
    +
    +            // Try acquiring lock only if not waiting
    +            boolean success = waitForLoggingWarnThresholdMsg.compareAndSet(Boolean.FALSE, Boolean.TRUE);
    +            if(success) {
    --- End diff --
    
    Fixed it in the next commit.


---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r127109651
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java ---
    @@ -292,6 +293,8 @@ public void parseProperties(Properties zkProp)
                     }
                 } else if ((key.startsWith("server.") || key.startsWith("group") || key.startsWith("weight")) && zkProp.containsKey("dynamicConfigFile")) {
                     throw new ConfigException("parameter: " + key + " must be in a separate dynamic config file");
    +            } else if(key.equals("warn.responseTime")) {
    --- End diff --
    
    +1 about @hanm suggestions. 
    
    nit: a space between `if` and `(` at line 296.
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by tdunning <gi...@git.apache.org>.
Github user tdunning commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r127877506
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ZooKeeperServerMainTest.java ---
    @@ -138,14 +145,56 @@ void delete(File f) throws IOException {
             ServerCnxnFactory getCnxnFactory() {
                 return main.getCnxnFactory();
             }
    +
         }
     
    -    public static  class TestZKSMain extends ZooKeeperServerMain {
    +    public static class TestZKSMain extends ZooKeeperServerMain {
    +
    +        private ServerStats serverStats;
    +
    +        @Override
    +        public ZooKeeperServer getZooKeeperServer(FileTxnSnapLog txnLog, ServerConfig config, ZKDatabase zkDb) {
    +            ZooKeeperServer zooKeeperServer = super.getZooKeeperServer(txnLog, config, zkDb);
    +            serverStats = zooKeeperServer.serverStats();
    +            return zooKeeperServer;
    +        }
    +
    +        @Override
             public void shutdown() {
                 super.shutdown();
             }
         }
     
    +    // Test for ZOOKEEPER-2770 ZooKeeper slow operation log
    +    @Test
    +    public void testRequestWarningThreshold() throws IOException, KeeperException, InterruptedException {
    +        ClientBase.setupTestEnv();
    +
    +        final int CLIENT_PORT = PortAssignment.unique();
    +
    +        MainThread main = new MainThread(CLIENT_PORT, true, null, 0);
    +        main.start();
    +
    +        Assert.assertTrue("waiting for server being up",
    +                ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT,
    +                        CONNECTION_TIMEOUT));
    +        // Get the stats object from the ZooKeeperServer to keep track of high latency requests.
    +        ServerStats stats = main.main.serverStats;
    +
    +        ZooKeeper zk = new ZooKeeper("127.0.0.1:" + CLIENT_PORT,
    +                ClientBase.CONNECTION_TIMEOUT, this);
    +
    +        zk.create("/foo1", "foobar".getBytes(), Ids.OPEN_ACL_UNSAFE,
    +                CreateMode.PERSISTENT);
    +
    +        Assert.assertEquals(new String(zk.getData("/foo1", null, null)), "foobar");
    +        // It takes a while for the counter to get updated sometimes, this is added to reduce flakyness
    +        Thread.sleep(1000);
    --- End diff --
    
    There is a sync() call that ZK supports on the client side. It waits until the replica that the client is connected to catches up with the leader. 
    
    This solves lots of timing issues and is typically the way to implement read-what-you-wrote.
    
    On the other hand, if your test is running with server internals, this isn't likely to work as such.
    
    The point is that if you wait until all pending items in the queue have been committed to all followers, you will be guaranteed to see all writes you caused before starting to wait.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on the issue:

    https://github.com/apache/zookeeper/pull/307
  
    General comment: please add more description on the pull request on what's the motivation of change and how it impacts performance / improve debugging etc. Or add those to the JIRA and post a digest on the pull request. That will make others who does not have a lot of knowledge on your use cases understand the motivation / impact of the change.
    
    Here is a good example w.r.t. pull request description: https://github.com/apache/zookeeper/pull/306, for reference.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r126866521
  
    --- Diff: src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java ---
    @@ -430,6 +432,7 @@ public void processRequest(Request request) {
                 // the client and leader disagree on where the client is most
                 // recently attached (and therefore invalid SESSION MOVED generated)
                 cnxn.sendCloseSession();
    +            request.checkLatency();
    --- End diff --
    
    close session takes time. Is it intended to put the latency check after closing the session instead of before the session closing? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by tdunning <gi...@git.apache.org>.
Github user tdunning commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r147040873
  
    --- Diff: src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java ---
    @@ -81,9 +81,11 @@
         private static final Logger LOG = LoggerFactory.getLogger(FinalRequestProcessor.class);
     
         ZooKeeperServer zks;
    +    ServerStats serverStats;
    --- End diff --
    
    I think that caching this reference is a bad practice. Getting a reference every time costs essentially nothing and keeping a reference to a data structure that we don't own is asking for trouble if anybody every reallocates it.


---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r127892837
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ZooKeeperServerMainTest.java ---
    @@ -138,14 +145,56 @@ void delete(File f) throws IOException {
             ServerCnxnFactory getCnxnFactory() {
                 return main.getCnxnFactory();
             }
    +
         }
     
    -    public static  class TestZKSMain extends ZooKeeperServerMain {
    +    public static class TestZKSMain extends ZooKeeperServerMain {
    +
    +        private ServerStats serverStats;
    +
    +        @Override
    +        public ZooKeeperServer getZooKeeperServer(FileTxnSnapLog txnLog, ServerConfig config, ZKDatabase zkDb) {
    +            ZooKeeperServer zooKeeperServer = super.getZooKeeperServer(txnLog, config, zkDb);
    +            serverStats = zooKeeperServer.serverStats();
    +            return zooKeeperServer;
    +        }
    +
    +        @Override
             public void shutdown() {
                 super.shutdown();
             }
         }
     
    +    // Test for ZOOKEEPER-2770 ZooKeeper slow operation log
    +    @Test
    +    public void testRequestWarningThreshold() throws IOException, KeeperException, InterruptedException {
    +        ClientBase.setupTestEnv();
    +
    +        final int CLIENT_PORT = PortAssignment.unique();
    +
    +        MainThread main = new MainThread(CLIENT_PORT, true, null, 0);
    +        main.start();
    +
    +        Assert.assertTrue("waiting for server being up",
    +                ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT,
    +                        CONNECTION_TIMEOUT));
    +        // Get the stats object from the ZooKeeperServer to keep track of high latency requests.
    +        ServerStats stats = main.main.serverStats;
    +
    +        ZooKeeper zk = new ZooKeeper("127.0.0.1:" + CLIENT_PORT,
    +                ClientBase.CONNECTION_TIMEOUT, this);
    +
    +        zk.create("/foo1", "foobar".getBytes(), Ids.OPEN_ACL_UNSAFE,
    +                CreateMode.PERSISTENT);
    +
    +        Assert.assertEquals(new String(zk.getData("/foo1", null, null)), "foobar");
    +        // It takes a while for the counter to get updated sometimes, this is added to reduce flakyness
    +        Thread.sleep(1000);
    --- End diff --
    
    > There is a sync() call that ZK supports on the client side. It waits until the replica that the client is connected to catches up with the leader.
    
    I am not sure if this is the case here since this test has a single instance of ZooKeeper Server and not a quorum. IMO, the conditional timeout should be good enough to make this test completely deterministic. Suggest if you feel otherwise.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r127110615
  
    --- Diff: src/java/main/org/apache/zookeeper/server/Request.java ---
    @@ -21,20 +21,27 @@
     import java.nio.ByteBuffer;
     import java.util.List;
     
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
     import org.apache.jute.Record;
     import org.apache.zookeeper.KeeperException;
     import org.apache.zookeeper.ZooDefs.OpCode;
     import org.apache.zookeeper.common.Time;
     import org.apache.zookeeper.data.Id;
    +import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
     import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier;
     import org.apache.zookeeper.txn.TxnHeader;
     
    +
     /**
      * This is the structure that represents a request moving through a chain of
      * RequestProcessors. There are various pieces of information that is tacked
      * onto the request as it is processed.
      */
     public class Request {
    +    private static final Logger LOG = LoggerFactory
    +            .getLogger(Request.class);
    --- End diff --
    
    No need to break line here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r184910721
  
    --- Diff: src/java/main/org/apache/zookeeper/server/ServerStats.java ---
    @@ -148,9 +174,46 @@ synchronized public void resetRequestCounters(){
             packetsReceived = 0;
             packetsSent = 0;
         }
    +    synchronized public void resetNumRequestsAboveThresholdTime() {
    +        numRequestsAboveThresholdTime = 0;
    +    }
         synchronized public void reset() {
             resetLatency();
             resetRequestCounters();
    +        resetNumRequestsAboveThresholdTime();
    +    }
    +
    +    public void checkLatency(final ZooKeeperServer zks, Request request) {
    +        long requestLatency = Time.currentElapsedTime() - request.createTime;
    +        boolean enabledAndAboveThreshold = (requestWarnThresholdMs == 0) ||
    +                (requestWarnThresholdMs > -1 && requestLatency > requestWarnThresholdMs);
    +        if (enabledAndAboveThreshold) {
    +            zks.serverStats().incNumRequestsAboveThresholdTime();
    +
    +            // Try acquiring lock only if not waiting
    +            boolean success = waitForLoggingWarnThresholdMsg.compareAndSet(Boolean.FALSE, Boolean.TRUE);
    +            if (success) {
    +                LOG.warn("Request {} exceeded threshold. Took {} ms", request, requestLatency);
    +                startCount = zks.serverStats().getNumRequestsAboveThresholdTime();
    +                timer.schedule(new TimerTask() {
    +                    @Override
    +                    public void run() {
    +                        long count = zks.serverStats().getNumRequestsAboveThresholdTime() - startCount;
    --- End diff --
    
    If there is no slow requests coming in between the time the task is scheduled and the time the task is executed, this count will be 0. It will not reflect the actual number of requests we want to log. Maybe log the startCount instead and reset it and leave the task only to reset the barrier?
    
    Also please use ScheduledService instead of Timer task.


---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r129450258
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java ---
    @@ -61,6 +61,7 @@
     
         private static boolean standaloneEnabled = true;
         private static boolean reconfigEnabled = false;
    +    private static int requestWarnThresholdMs = 10000;
    --- End diff --
    
    To be frank, I am newbie and haven't debugged this in detail. This value is purely seen based on the 'stat' command on our test cluster. @apurtell might be able to tell more practical values.
    
    @skamille I would prefer turning this on by default, although the default value needs to be discussed. In my understanding, this helps in situations when we see timeouts at application level, such a log can might help narrow down towards the cause.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r191204982
  
    --- Diff: src/java/main/org/apache/zookeeper/server/ServerStats.java ---
    @@ -148,9 +174,46 @@ synchronized public void resetRequestCounters(){
             packetsReceived = 0;
             packetsSent = 0;
         }
    +    synchronized public void resetNumRequestsAboveThresholdTime() {
    +        numRequestsAboveThresholdTime = 0;
    +    }
         synchronized public void reset() {
             resetLatency();
             resetRequestCounters();
    +        resetNumRequestsAboveThresholdTime();
    +    }
    +
    +    public void checkLatency(final ZooKeeperServer zks, Request request) {
    +        long requestLatency = Time.currentElapsedTime() - request.createTime;
    +        boolean enabledAndAboveThreshold = (requestWarnThresholdMs == 0) ||
    +                (requestWarnThresholdMs > -1 && requestLatency > requestWarnThresholdMs);
    +        if (enabledAndAboveThreshold) {
    +            zks.serverStats().incNumRequestsAboveThresholdTime();
    +
    +            // Try acquiring lock only if not waiting
    +            boolean success = waitForLoggingWarnThresholdMsg.compareAndSet(Boolean.FALSE, Boolean.TRUE);
    +            if (success) {
    +                LOG.warn("Request {} exceeded threshold. Took {} ms", request, requestLatency);
    +                startCount = zks.serverStats().getNumRequestsAboveThresholdTime();
    +                timer.schedule(new TimerTask() {
    +                    @Override
    +                    public void run() {
    +                        long count = zks.serverStats().getNumRequestsAboveThresholdTime() - startCount;
    +                        LOG.warn("Number of requests that exceeded {} ms in past {} ms: {}",
    +                                requestWarnThresholdMs, delayTimeForLoggingWarnThresholdMsg, count);
    +                        waitForLoggingWarnThresholdMsg.set(Boolean.FALSE);
    +                    }
    +                }, delayTimeForLoggingWarnThresholdMsg);
    +            }
    +        }
    +    }
    +
    +    public void setDelayTimeForLoggingWarnThresholdMsg(int delay) {
    +        this.delayTimeForLoggingWarnThresholdMsg = delay;
    +    }
    +
    +    public void setWaitForLoggingWarnThresholdMsgToFalse() {
    --- End diff --
    
    This one should be `package-private` too. Same comment applies here too.


---

[GitHub] zookeeper issue #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on the issue:

    https://github.com/apache/zookeeper/pull/307
  
    >  Can we at least make it so that the process of "push a metric to a buffer, have a thread that wakes up periodically and flushes information out of that buffer" is usable by multiple parts of the system, instead of coupling it to the one metric of request time?
    
    I am ready to put in the effort of making this framework more generic, if you feel its worth the time and effort to put into it. I would also like to hear others opinions on this. 
    The inspiration for this JIRA came from HBase, where it has been used to log slow requests at the basic level. I feel that it can be useful to keep track of latency spikes that are seen in production. @apurtell can suggest use-cases.


---

[GitHub] zookeeper issue #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by zodvik <gi...@git.apache.org>.
Github user zodvik commented on the issue:

    https://github.com/apache/zookeeper/pull/307
  
    > reasons why any given read or write operation may become slow: a software bug, a protocol problem, a hardware issue with the commit log(s), a network issue. If the problem is constant it is trivial to come to an understanding of the cause. However in order to diagnose intermittent problems we often don't know where, or when, to begin looking. We need some sort of timestamped indication of the problem
    
    IMO, slow query log is useful for those systems were the response times can vary due to type of query (long scans, improper index, poor query plan, etc.). For the type of issues mentioned, we can expect all operations to be impacted and fill up the log. 
    
    If the intent is to get an time stamp indication of the problem, a metric reporting mechanism will be more useful. If we have rolling interval percentiles (per command?) published either to `stat` or `jmx`, that would give a good visibility.



---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r191215183
  
    --- Diff: src/java/test/org/apache/zookeeper/server/HighLatencyRequestLoggingTest.java ---
    @@ -0,0 +1,285 @@
    +package org.apache.zookeeper.server;
    +
    +import org.apache.log4j.AppenderSkeleton;
    +import org.apache.log4j.LogManager;
    +import org.apache.log4j.spi.LoggingEvent;
    +import org.apache.zookeeper.CreateMode;
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.PortAssignment;
    +import org.apache.zookeeper.WatchedEvent;
    +import org.apache.zookeeper.Watcher;
    +import org.apache.zookeeper.ZKTestCase;
    +import org.apache.zookeeper.ZooDefs;
    +import org.apache.zookeeper.ZooKeeper;
    +import org.apache.zookeeper.test.ClientBase;
    +import org.junit.After;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.mockito.runners.MockitoJUnitRunner;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +import static org.apache.zookeeper.test.ClientBase.CONNECTION_TIMEOUT;
    +
    +@RunWith(MockitoJUnitRunner.class)
    +public class HighLatencyRequestLoggingTest extends ZKTestCase implements Watcher {
    +
    +    private TestAppender testAppender = new TestAppender();
    +
    +    @Before
    +    public void setup() {
    +        LogManager.getLogger(ServerStats.class).addAppender(testAppender);
    +    }
    +
    +    @After
    +    public void tearDown() {
    +        LogManager.getLogger(ServerStats.class).removeAppender(testAppender);
    +    }
    +
    +    class TestAppender extends AppenderSkeleton {
    +        private List<String> messages = new ArrayList<String>();
    +
    +        @Override
    +        protected void append(LoggingEvent loggingEvent) {
    +
    +        }
    +
    +        @Override
    +        public void close() {
    +
    +        }
    +
    +        public void doAppend(LoggingEvent event) {
    +            synchronized (messages) {
    +                messages.add(event.getMessage().toString());
    +            }
    +        }
    +
    +        @Override
    +        public boolean requiresLayout() {
    +            return false;
    +        }
    +
    +        public List<String> getMessages() {
    +            return messages;
    +        }
    +    }
    +
    +    // Test Class to create ZNodes at the specified path and check them
    +    class ZNodeCreator implements Runnable {
    +
    +        ZooKeeper zk;
    +        int startNum;
    +        int totalNum;
    +
    +        ZNodeCreator(ZooKeeper zk, int startNum, int totalNum) {
    +            this.zk = zk;
    +            this.startNum = startNum;
    +            this.totalNum = totalNum;
    +        }
    +
    +        @Override
    +        public void run() {
    +            for(int i = startNum; i < startNum + totalNum; i++) {
    +                try {
    +                    zk.create("/foo" + i, ("foobar" +  i).getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE,
    +                            CreateMode.PERSISTENT);
    +                    Assert.assertEquals(new String(zk.getData("/foo" + i, null, null)), "foobar" + i);
    +                } catch (Exception e) {
    +                    Assert.fail("Failed to create ZNode. Exiting test.");
    +                }
    +            }
    +        }
    +    }
    +
    +    // Basic test that verifies total number of requests above threshold time
    +    @Test
    +    public void testRequestWarningThreshold() throws IOException, KeeperException, InterruptedException {
    +        ClientBase.setupTestEnv();
    +
    +        final int CLIENT_PORT = PortAssignment.unique();
    +
    +        ZooKeeperServerMainTest.MainThread main =
    +                new ZooKeeperServerMainTest.MainThread(CLIENT_PORT, true, null, 0);
    +        main.start();
    +
    +        Assert.assertTrue("waiting for server being up",
    +                ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT,
    +                        CONNECTION_TIMEOUT));
    +        // Get the stats object from the ZooKeeperServer to keep track of high latency requests.
    +        ServerStats stats = main.main.getServerStats();
    +        stats.setWaitForLoggingWarnThresholdMsgToFalse();
    +        stats.setDelayTimeForLoggingWarnThresholdMsg(1000);
    +
    +        ZooKeeper zk = new ZooKeeper("127.0.0.1:" + CLIENT_PORT,
    +                ClientBase.CONNECTION_TIMEOUT, (Watcher) this);
    +
    +        zk.create("/foo1", "fb".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE,
    +                CreateMode.PERSISTENT);
    +
    +        Assert.assertEquals(new String(zk.getData("/foo1", null, null)), "fb");
    +
    +        zk.close();
    +        main.shutdown();
    +        main.join();
    +        main.deleteDirs();
    +
    +        // Total Requests: 4
    +        // 1 Thread, each writing and verifying 1 znode1 = (1 threads * 1 znodes * 2 req = 2)
    +        // CreateSession, CloseSession
    +        verifyLogMessages(4, 5);
    +
    +        Assert.assertTrue("waiting for server down",
    +                ClientBase.waitForServerDown("127.0.0.1:" + CLIENT_PORT,
    +                        ClientBase.CONNECTION_TIMEOUT));
    +
    +
    +    }
    +
    +    // Test to verify count of requests that exceeded threshold and logger rate limiting with a single thread
    +    @Test
    +    public void testSingleThreadFrequentRequestWarningThresholdLogging() throws IOException, KeeperException, InterruptedException {
    +        ClientBase.setupTestEnv();
    +
    +        final int CLIENT_PORT = PortAssignment.unique();
    +
    +        ZooKeeperServerMainTest.MainThread main = new
    +                ZooKeeperServerMainTest.MainThread(CLIENT_PORT, true, null, 0);
    +        main.start();
    +
    +        Assert.assertTrue("waiting for server being up",
    +                ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT,
    +                        CONNECTION_TIMEOUT));
    +        // Get the stats object from the ZooKeeperServer to keep track of high latency requests.
    +        ServerStats stats = main.main.getServerStats();
    +        stats.setWaitForLoggingWarnThresholdMsgToFalse();
    +        stats.setDelayTimeForLoggingWarnThresholdMsg(1000);
    +
    +        ZooKeeper zk = new ZooKeeper("127.0.0.1:" + CLIENT_PORT,
    +                ClientBase.CONNECTION_TIMEOUT, this);
    +
    +        Thread thread = new Thread(new ZNodeCreator(zk, 0, 5));
    +        thread.start();
    +        thread.join();
    +
    +        zk.close();
    +        main.shutdown();
    +        main.join();
    +        main.deleteDirs();
    +
    +        // Total Requests: 12
    +        // 1 Thread, each writing and verifying 5 znodes = (1 threads * 5 znodes * 2 req = 10)
    +        // CreateSession, CloseSession
    +        verifyLogMessages(12, 5);
    +
    +        Assert.assertTrue("waiting for server down",
    +                ClientBase.waitForServerDown("127.0.0.1:" + CLIENT_PORT,
    +                        ClientBase.CONNECTION_TIMEOUT));
    +    }
    +
    +
    +    // Test to verify count of requests that exceeded threshold and logger rate limiting with a multiple threads and multiple times
    +    @Test
    +    public void testMultipleThreadsFrequentRequestWarningThresholdLogging()
    +            throws IOException, KeeperException, InterruptedException {
    +        ClientBase.setupTestEnv();
    +
    +        final int CLIENT_PORT = PortAssignment.unique();
    +
    +        ZooKeeperServerMainTest.MainThread main =
    +                new ZooKeeperServerMainTest.MainThread(CLIENT_PORT, true, null, 0);
    +        main.start();
    +
    +        Assert.assertTrue("waiting for server being up",
    +                ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT,
    +                        CONNECTION_TIMEOUT));
    +        // Get the stats object from the ZooKeeperServer to keep track of high latency requests.
    +        ServerStats stats = main.main.getServerStats();
    +        stats.setWaitForLoggingWarnThresholdMsgToFalse();
    +        stats.setDelayTimeForLoggingWarnThresholdMsg(1000);
    +
    +        final ZooKeeper zk = new ZooKeeper("127.0.0.1:" + CLIENT_PORT,
    +                ClientBase.CONNECTION_TIMEOUT, this);
    +
    +        Thread thread1 = new Thread(new ZNodeCreator(zk,0,3));
    --- End diff --
    
    Using a for loop to housekeep these threads would have the additional benefit of testing with more clients (like 100).


---

[GitHub] zookeeper issue #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on the issue:

    https://github.com/apache/zookeeper/pull/307
  
    Due to some reason, Jenkins didn't build it automatically. I triggered a manual build with my latest commit ID SHA1 and the results are at https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/1123/
    
    All the Findbugs warnings have been resolved, and the single failing test seems to be unrelated to my change (or possibly a flapper since it is passing on my local machine).
    
    This commit is a squashed commit based on the earlier inputs provided by everyone. Thank you!
    
    As of now, the default value for `requestWarnThresholdMs` is 10 seconds, which is too high. Users can set it according to the average latency values they typically see from the serverstats. The patch also limits logging rate, so if the value is set too low, it will print out a message every 60 seconds showing the number of high latency requests in past minute. The motivation of this JIRA is specified in description at  https://issues.apache.org/jira/browse/ZOOKEEPER-2770.
    
    If you can review this PR, it would be great.
    @hanm @tdunning @eribeiro @skamille 
    
    If we can get in this JIRA, the future plan is to work on @tdunning t-digest and use it to determine high latency requests specifically based on its type. Thanks!


---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r191204646
  
    --- Diff: src/java/main/org/apache/zookeeper/server/ServerStats.java ---
    @@ -148,9 +174,46 @@ synchronized public void resetRequestCounters(){
             packetsReceived = 0;
             packetsSent = 0;
         }
    +    synchronized public void resetNumRequestsAboveThresholdTime() {
    +        numRequestsAboveThresholdTime = 0;
    +    }
         synchronized public void reset() {
             resetLatency();
             resetRequestCounters();
    +        resetNumRequestsAboveThresholdTime();
    +    }
    +
    +    public void checkLatency(final ZooKeeperServer zks, Request request) {
    +        long requestLatency = Time.currentElapsedTime() - request.createTime;
    +        boolean enabledAndAboveThreshold = (requestWarnThresholdMs == 0) ||
    +                (requestWarnThresholdMs > -1 && requestLatency > requestWarnThresholdMs);
    +        if (enabledAndAboveThreshold) {
    +            zks.serverStats().incNumRequestsAboveThresholdTime();
    +
    +            // Try acquiring lock only if not waiting
    +            boolean success = waitForLoggingWarnThresholdMsg.compareAndSet(Boolean.FALSE, Boolean.TRUE);
    +            if (success) {
    +                LOG.warn("Request {} exceeded threshold. Took {} ms", request, requestLatency);
    +                startCount = zks.serverStats().getNumRequestsAboveThresholdTime();
    +                timer.schedule(new TimerTask() {
    +                    @Override
    +                    public void run() {
    +                        long count = zks.serverStats().getNumRequestsAboveThresholdTime() - startCount;
    --- End diff --
    
    > it is fine to say that 0 requests had longer times since the last bad request was logged. The total count can be seen using stat at any point of time. What do you suggest?
    
    Makes sense to me. What do you think of _not_ logging anything in the task if actual counter equals to `startCount`?
    
    As I stated above, please use `ScheduledExecutorService` here.


---

[GitHub] zookeeper issue #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on the issue:

    https://github.com/apache/zookeeper/pull/307
  
    @hanm @eribeiro @tdunning @skamille 
    Please review.
    Now that I have added rate limiting to logging, can we also turn this on by default?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r187815108
  
    --- Diff: src/java/main/org/apache/zookeeper/server/ServerStats.java ---
    @@ -148,9 +174,46 @@ synchronized public void resetRequestCounters(){
             packetsReceived = 0;
             packetsSent = 0;
         }
    +    synchronized public void resetNumRequestsAboveThresholdTime() {
    +        numRequestsAboveThresholdTime = 0;
    +    }
         synchronized public void reset() {
             resetLatency();
             resetRequestCounters();
    +        resetNumRequestsAboveThresholdTime();
    +    }
    +
    +    public void checkLatency(final ZooKeeperServer zks, Request request) {
    +        long requestLatency = Time.currentElapsedTime() - request.createTime;
    +        boolean enabledAndAboveThreshold = (requestWarnThresholdMs == 0) ||
    +                (requestWarnThresholdMs > -1 && requestLatency > requestWarnThresholdMs);
    +        if (enabledAndAboveThreshold) {
    +            zks.serverStats().incNumRequestsAboveThresholdTime();
    +
    +            // Try acquiring lock only if not waiting
    +            boolean success = waitForLoggingWarnThresholdMsg.compareAndSet(Boolean.FALSE, Boolean.TRUE);
    +            if (success) {
    +                LOG.warn("Request {} exceeded threshold. Took {} ms", request, requestLatency);
    +                startCount = zks.serverStats().getNumRequestsAboveThresholdTime();
    +                timer.schedule(new TimerTask() {
    +                    @Override
    +                    public void run() {
    +                        long count = zks.serverStats().getNumRequestsAboveThresholdTime() - startCount;
    +                        LOG.warn("Number of requests that exceeded {} ms in past {} ms: {}",
    +                                requestWarnThresholdMs, delayTimeForLoggingWarnThresholdMsg, count);
    +                        waitForLoggingWarnThresholdMsg.set(Boolean.FALSE);
    +                    }
    +                }, delayTimeForLoggingWarnThresholdMsg);
    +            }
    +        }
    +    }
    +
    +    public void setDelayTimeForLoggingWarnThresholdMsg(int delay) {
    +        this.delayTimeForLoggingWarnThresholdMsg = delay;
    --- End diff --
    
    We are not really planning to expose this method to end user. A rolling window of 60 seconds is desirable and hence only a single parameter `request.warningthresholdms` will be provided for end user to configure.


---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r127627805
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ZooKeeperServerMainTest.java ---
    @@ -138,14 +145,56 @@ void delete(File f) throws IOException {
             ServerCnxnFactory getCnxnFactory() {
                 return main.getCnxnFactory();
             }
    +
         }
     
    -    public static  class TestZKSMain extends ZooKeeperServerMain {
    +    public static class TestZKSMain extends ZooKeeperServerMain {
    +
    +        private ServerStats serverStats;
    +
    +        @Override
    +        public ZooKeeperServer getZooKeeperServer(FileTxnSnapLog txnLog, ServerConfig config, ZKDatabase zkDb) {
    +            ZooKeeperServer zooKeeperServer = super.getZooKeeperServer(txnLog, config, zkDb);
    +            serverStats = zooKeeperServer.serverStats();
    +            return zooKeeperServer;
    +        }
    +
    +        @Override
             public void shutdown() {
                 super.shutdown();
             }
         }
     
    +    // Test for ZOOKEEPER-2770 ZooKeeper slow operation log
    +    @Test
    +    public void testRequestWarningThreshold() throws IOException, KeeperException, InterruptedException {
    +        ClientBase.setupTestEnv();
    +
    +        final int CLIENT_PORT = PortAssignment.unique();
    +
    +        MainThread main = new MainThread(CLIENT_PORT, true, null, 0);
    +        main.start();
    +
    +        Assert.assertTrue("waiting for server being up",
    +                ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT,
    +                        CONNECTION_TIMEOUT));
    +        // Get the stats object from the ZooKeeperServer to keep track of high latency requests.
    +        ServerStats stats = main.main.serverStats;
    +
    +        ZooKeeper zk = new ZooKeeper("127.0.0.1:" + CLIENT_PORT,
    +                ClientBase.CONNECTION_TIMEOUT, this);
    +
    +        zk.create("/foo1", "foobar".getBytes(), Ids.OPEN_ACL_UNSAFE,
    +                CreateMode.PERSISTENT);
    +
    +        Assert.assertEquals(new String(zk.getData("/foo1", null, null)), "foobar");
    +        // It takes a while for the counter to get updated sometimes, this is added to reduce flakyness
    +        Thread.sleep(1000);
    --- End diff --
    
    wondering why... 🤔 I have run this test case a couple of times, but was unable to see this lag in counter updated manifested as assertion failure. Did you see this consistently?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on the issue:

    https://github.com/apache/zookeeper/pull/307
  
    @hanm 
    > I think at some point after merge this you need create another pull request targeting stable branch (branch-3.4) where the version number should be 3.4.11 (hopefully, if this can get in before next release)
    
    I will create a new PR for that branch as well. I am hoping this to get this out in next release if possible.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r127172773
  
    --- Diff: src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java ---
    @@ -430,6 +432,7 @@ public void processRequest(Request request) {
                 // the client and leader disagree on where the client is most
                 // recently attached (and therefore invalid SESSION MOVED generated)
                 cnxn.sendCloseSession();
    +            request.checkLatency();
    --- End diff --
    
    Yes. The percentiles and whatsoever should be a matter of another issue. No doubt about this, IMO.
    
    The meat of my comment was in fact if checkLatency isn't in the responsibility of Stats class, even it remaining as an proper method (i.e, no inclusion into updateLatency). 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r128153360
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ZooKeeperServerMainTest.java ---
    @@ -138,14 +145,56 @@ void delete(File f) throws IOException {
             ServerCnxnFactory getCnxnFactory() {
                 return main.getCnxnFactory();
             }
    +
         }
     
    -    public static  class TestZKSMain extends ZooKeeperServerMain {
    +    public static class TestZKSMain extends ZooKeeperServerMain {
    +
    +        private ServerStats serverStats;
    +
    +        @Override
    +        public ZooKeeperServer getZooKeeperServer(FileTxnSnapLog txnLog, ServerConfig config, ZKDatabase zkDb) {
    +            ZooKeeperServer zooKeeperServer = super.getZooKeeperServer(txnLog, config, zkDb);
    +            serverStats = zooKeeperServer.serverStats();
    +            return zooKeeperServer;
    +        }
    +
    +        @Override
             public void shutdown() {
                 super.shutdown();
             }
         }
     
    +    // Test for ZOOKEEPER-2770 ZooKeeper slow operation log
    +    @Test
    +    public void testRequestWarningThreshold() throws IOException, KeeperException, InterruptedException {
    +        ClientBase.setupTestEnv();
    +
    +        final int CLIENT_PORT = PortAssignment.unique();
    +
    +        MainThread main = new MainThread(CLIENT_PORT, true, null, 0);
    +        main.start();
    +
    +        Assert.assertTrue("waiting for server being up",
    +                ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT,
    +                        CONNECTION_TIMEOUT));
    +        // Get the stats object from the ZooKeeperServer to keep track of high latency requests.
    +        ServerStats stats = main.main.serverStats;
    +
    +        ZooKeeper zk = new ZooKeeper("127.0.0.1:" + CLIENT_PORT,
    +                ClientBase.CONNECTION_TIMEOUT, this);
    +
    +        zk.create("/foo1", "foobar".getBytes(), Ids.OPEN_ACL_UNSAFE,
    +                CreateMode.PERSISTENT);
    +
    +        Assert.assertEquals(new String(zk.getData("/foo1", null, null)), "foobar");
    +        // It takes a while for the counter to get updated sometimes, this is added to reduce flakyness
    +        Thread.sleep(1000);
    --- End diff --
    
    sync() will not help here because this specific counter is not part of any zab transactions. Two approaches to make test more robust:
    
    * We can always increase the counter before we send a response back to client in FinalRequestProcessor. This will make test deterministic, Or,
    
    * In test, implement retry polling with a timeout and a smaller sleep interval (e..g 10ms instead of 1000ms). We use this pattern a lot in our test. In general we'd like to make deterministic tests as much as possible (in other words, avoid sleep at all cost.) though sometimes that is hard to do due to various reasons.
    
    A side note on sync() - sync is not a quorum operation so it only works if the client connects to the leader. I think there is a jira trying to fix this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r191202289
  
    --- Diff: src/java/main/org/apache/zookeeper/server/ServerStats.java ---
    @@ -117,6 +138,11 @@ public String toString(){
             sb.append("Mode: " + getServerState() + "\n");
             return sb.toString();
         }
    +
    +    synchronized void incNumRequestsAboveThresholdTime() {
    +        numRequestsAboveThresholdTime++;
    --- End diff --
    
    Again, change the field to `AtomicLong` and you don't need to sync here.


---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r191202605
  
    --- Diff: src/java/main/org/apache/zookeeper/server/ServerStats.java ---
    @@ -148,9 +174,46 @@ synchronized public void resetRequestCounters(){
             packetsReceived = 0;
             packetsSent = 0;
         }
    +    synchronized public void resetNumRequestsAboveThresholdTime() {
    +        numRequestsAboveThresholdTime = 0;
    +    }
         synchronized public void reset() {
             resetLatency();
             resetRequestCounters();
    +        resetNumRequestsAboveThresholdTime();
    +    }
    +
    +    public void checkLatency(final ZooKeeperServer zks, Request request) {
    --- End diff --
    
    IMO, this method could be private and called from `updateLatency()`. Basically you don't want to miss any latency update calls, hence it makes sense to put the call there. Any objections?


---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r127116226
  
    --- Diff: src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java ---
    @@ -430,6 +432,7 @@ public void processRequest(Request request) {
                 // the client and leader disagree on where the client is most
                 // recently attached (and therefore invalid SESSION MOVED generated)
                 cnxn.sendCloseSession();
    +            request.checkLatency();
    --- End diff --
    
    I see that both `request.checkLatency()` and  `zks.serverStats().updateLatency(request.createTime);` (a few lines above) are dealing with the same matter (request latency) even though on radical different positions.
    
    IMHO, `zks.serverStats().updateLatency(request.createTime);` should be moved to the line position where `request.checkLatency()` currently is and to move the if condition from `request.checkLatency()` to `zks.serverStats().updateLatency(request.createTime);`, eventually getting rid of `request.checkLatency()`. Could this raise some unexpected bugs?
    
    @karanmehta93 @hanm, wdyt?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r191213617
  
    --- Diff: src/java/main/org/apache/zookeeper/server/ServerStats.java ---
    @@ -21,17 +21,33 @@
     
     
     import org.apache.zookeeper.common.Time;
    +import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.Timer;
    +import java.util.TimerTask;
    +import java.util.concurrent.atomic.AtomicReference;
     
     /**
      * Basic Server Statistics
      */
     public class ServerStats {
    +    private static final Logger LOG = LoggerFactory.getLogger(ServerStats.class);
    +
         private long packetsSent;
         private long packetsReceived;
         private long maxLatency;
         private long minLatency = Long.MAX_VALUE;
         private long totalLatency = 0;
         private long count = 0;
    +    private long numRequestsAboveThresholdTime = 0;
    +
    +    final static long requestWarnThresholdMs = QuorumPeerConfig.getRequestWarnResponseThresholdMs();
    +    final static Timer timer = new Timer();
    --- End diff --
    
    And this field should be `private` too.


---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r191205970
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java ---
    @@ -61,6 +61,7 @@
     
         private static boolean standaloneEnabled = true;
         private static boolean reconfigEnabled = false;
    +    private static int requestWarnThresholdMs = 10000;
    --- End diff --
    
    @karanmehta93 This feature sounds to me some kind of debugging / verbose monitoring feature which should generally be turned off by default. If @hanm feels the same, I'll second that.
    
    However, if you want to implement a feature switch to this - which I think would be awesome - use a dedicated bool config instead of some extraordinary threshold value. 


---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r127619032
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ZooKeeperServerMainTest.java ---
    @@ -138,14 +145,56 @@ void delete(File f) throws IOException {
             ServerCnxnFactory getCnxnFactory() {
                 return main.getCnxnFactory();
             }
    +
         }
     
    -    public static  class TestZKSMain extends ZooKeeperServerMain {
    +    public static class TestZKSMain extends ZooKeeperServerMain {
    +
    +        private ServerStats serverStats;
    +
    +        @Override
    +        public ZooKeeperServer getZooKeeperServer(FileTxnSnapLog txnLog, ServerConfig config, ZKDatabase zkDb) {
    +            ZooKeeperServer zooKeeperServer = super.getZooKeeperServer(txnLog, config, zkDb);
    +            serverStats = zooKeeperServer.serverStats();
    +            return zooKeeperServer;
    +        }
    +
    +        @Override
             public void shutdown() {
                 super.shutdown();
             }
         }
     
    +    // Test for ZOOKEEPER-2770 ZooKeeper slow operation log
    --- End diff --
    
    We don't put these kind of comments. The git log already has the patch name, description and files changed/added.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on the issue:

    https://github.com/apache/zookeeper/pull/307
  
    There are -1 on findbugs, release audits, and test failures reported by Jekins. Please investigate.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r146108343
  
    --- Diff: src/java/main/org/apache/zookeeper/server/ServerStats.java ---
    @@ -21,17 +21,32 @@
     
     
     import org.apache.zookeeper.common.Time;
    +import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.Timer;
    +import java.util.TimerTask;
     
     /**
      * Basic Server Statistics
      */
     public class ServerStats {
    +    private static final Logger LOG = LoggerFactory.getLogger(ServerStats.class);
    +
         private long packetsSent;
         private long packetsReceived;
         private long maxLatency;
         private long minLatency = Long.MAX_VALUE;
         private long totalLatency = 0;
         private long count = 0;
    +    private long numRequestsAboveThresholdTime = 0;
    +
    +    final static long requestWarnThresholdMs = QuorumPeerConfig.getRequestWarnResponseThresholdMs();
    +    final static Timer timer = new Timer();
    +    volatile Boolean waitForLoggingWarnThresholdMsg = false;
    --- End diff --
    
    Oh, excuse me, I got it. It's because you are synchronizing on it, right? Well, it's not recommended (anti-pattern) to synchronize on non-final fields. Didn't Findbugs raised a warning about this?


---

[GitHub] zookeeper issue #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on the issue:

    https://github.com/apache/zookeeper/pull/307
  
    I think we should consolidate the latency check in `zks.serverStats().updateLatency`. It's odd to have two (or in future even more) types of latency checks scattered around which creates fragmentation w.r.t. the definition of what a request latency means. The existing latency measurement in ServerStats measures the time between a request creation and a request landing at final request processor; the patch instead measures end to end time of a request from its start to finish processing. I am fine with the end to end processing time, though I'd like to double check with a few folks around to make sure the regression and impact of this change is limited.
    
    I think ServerStats is a good place to put the DS Ted recommended. 
    
    I think it's a good idea to scope the JIRA so it's easier to get it reviewed and committed. What this patch is doing is a positive improvement to the operational aspects of ZK so that can be the scope of this PR. On top of that future improvements could be what Edward and Ted suggested (JMX, distribution of latencies / histogram etc). These work can be tracked by making them sub tasks under current JIRA.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on the issue:

    https://github.com/apache/zookeeper/pull/307
  
    @karanmehta93 Thanks for update the pull request with more context information. Do you mind put the context description in the pull request description? 
    
    Regarding test cases, I think you can just add a new test file under zookeeper/test. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r127112584
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java ---
    @@ -735,6 +738,8 @@ public String getConfigFilename(){
         public Boolean getQuorumListenOnAllIPs() {
             return quorumListenOnAllIPs;
         }
    +
    +    public static int getWarnResponseTime() { return  warnThreshold; }
    --- End diff --
    
    What about rename this method to `getMaxRequestThreshold()` or `getWarnRequestThreshold()`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by tdunning <gi...@git.apache.org>.
Github user tdunning commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r128130857
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ZooKeeperServerMainTest.java ---
    @@ -138,14 +145,56 @@ void delete(File f) throws IOException {
             ServerCnxnFactory getCnxnFactory() {
                 return main.getCnxnFactory();
             }
    +
         }
     
    -    public static  class TestZKSMain extends ZooKeeperServerMain {
    +    public static class TestZKSMain extends ZooKeeperServerMain {
    +
    +        private ServerStats serverStats;
    +
    +        @Override
    +        public ZooKeeperServer getZooKeeperServer(FileTxnSnapLog txnLog, ServerConfig config, ZKDatabase zkDb) {
    +            ZooKeeperServer zooKeeperServer = super.getZooKeeperServer(txnLog, config, zkDb);
    +            serverStats = zooKeeperServer.serverStats();
    +            return zooKeeperServer;
    +        }
    +
    +        @Override
             public void shutdown() {
                 super.shutdown();
             }
         }
     
    +    // Test for ZOOKEEPER-2770 ZooKeeper slow operation log
    +    @Test
    +    public void testRequestWarningThreshold() throws IOException, KeeperException, InterruptedException {
    +        ClientBase.setupTestEnv();
    +
    +        final int CLIENT_PORT = PortAssignment.unique();
    +
    +        MainThread main = new MainThread(CLIENT_PORT, true, null, 0);
    +        main.start();
    +
    +        Assert.assertTrue("waiting for server being up",
    +                ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT,
    +                        CONNECTION_TIMEOUT));
    +        // Get the stats object from the ZooKeeperServer to keep track of high latency requests.
    +        ServerStats stats = main.main.serverStats;
    +
    +        ZooKeeper zk = new ZooKeeper("127.0.0.1:" + CLIENT_PORT,
    +                ClientBase.CONNECTION_TIMEOUT, this);
    +
    +        zk.create("/foo1", "foobar".getBytes(), Ids.OPEN_ACL_UNSAFE,
    +                CreateMode.PERSISTENT);
    +
    +        Assert.assertEquals(new String(zk.getData("/foo1", null, null)), "foobar");
    +        // It takes a while for the counter to get updated sometimes, this is added to reduce flakyness
    +        Thread.sleep(1000);
    --- End diff --
    
    So if the test does any kind of operation that involves the interaction of the `FinalRequestProcessor`, the completion of that operation will occur after the counter increment has completed. The client side can then query the counter with safety.
    
    You are correct that the counter isn't part of the state that ZK is protecting, but the effect will be the same since the `FRP` handles all requests in order.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by tdunning <gi...@git.apache.org>.
Github user tdunning commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r147040783
  
    --- Diff: src/java/main/org/apache/zookeeper/server/ServerStats.java ---
    @@ -148,9 +174,46 @@ synchronized public void resetRequestCounters(){
             packetsReceived = 0;
             packetsSent = 0;
         }
    +    synchronized public void resetNumRequestsAboveThresholdTime() {
    +        numRequestsAboveThresholdTime = 0;
    +    }
         synchronized public void reset() {
             resetLatency();
             resetRequestCounters();
    +        resetNumRequestsAboveThresholdTime();
    +    }
    +
    +    public void checkLatency(final ZooKeeperServer zks, Request request) {
    +        long requestLatency = Time.currentElapsedTime() - request.createTime;
    +        boolean enabledAndAboveThreshold = (requestWarnThresholdMs == 0) ||
    +                (requestWarnThresholdMs > -1 && requestLatency > requestWarnThresholdMs);
    +        if (enabledAndAboveThreshold) {
    +            zks.serverStats().incNumRequestsAboveThresholdTime();
    +
    +            // Try acquiring lock only if not waiting
    +            boolean success = waitForLoggingWarnThresholdMsg.compareAndSet(Boolean.FALSE, Boolean.TRUE);
    +            if(success) {
    --- End diff --
    
    Need a space between if and (


---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by skamille <gi...@git.apache.org>.
Github user skamille commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r129444784
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java ---
    @@ -61,6 +61,7 @@
     
         private static boolean standaloneEnabled = true;
         private static boolean reconfigEnabled = false;
    +    private static int requestWarnThresholdMs = 10000;
    --- End diff --
    
    You've seen 2.3 seconds latency within the ZK quorum operations? That seems worthy of posting to the mailing list along with some information about what was happening and why.
    I think it sounds like @hanm wants to turn this off by default, which makes this moot, and I'm supportive of that, so I'll let him make the call.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r131282046
  
    --- Diff: src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java ---
    @@ -460,11 +465,20 @@ public void processRequest(Request request) {
                 if (request.type == OpCode.closeSession) {
                     cnxn.sendCloseSession();
                 }
    +            checkLatency(request);
             } catch (IOException e) {
                 LOG.error("FIXMSG",e);
             }
         }
     
    +    private void checkLatency(Request request) {
    +        long requestLatency = Time.currentElapsedTime() - request.createTime;
    +        if((requestWarnThresholdMs > -1 && requestLatency > requestWarnThresholdMs) || requestWarnThresholdMs == 0) {
    +            LOG.warn("Request "  + request + " exceeded threshold. Took " + requestLatency + " ms.");
    --- End diff --
    
    You can leverage the use of interpolation of Strings in logging:
    ```
    LOG.warn("Request {} exceeded {}", obj1, obj2);
    ```
    
    Again, such a minor suggestion that it's up to you adopt it or not. Cheers!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on the issue:

    https://github.com/apache/zookeeper/pull/307
  
    ZooKeeper is a complex distributed application. There are many reasons why any given read or write operation may become slow: a software bug, a protocol problem, a hardware issue with the commit log(s), a network issue. If the problem is constant it is trivial to come to an understanding of the cause. However in order to diagnose intermittent problems we often don't know where, or when, to begin looking. We need some sort of timestamped indication of the problem. Although ZooKeeper is not a datastore, it does persist data, and can suffer intermittent performance degradation, and should consider implementing a 'slow query' log, a feature very common to services which persist information on behalf of clients which may be sensitive to latency while waiting for confirmation of successful persistence.
    
    @hanm FYI


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on the issue:

    https://github.com/apache/zookeeper/pull/307
  
    @hanm Got a few spare cycles to address my comments?


---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r191199674
  
    --- Diff: src/java/main/org/apache/zookeeper/server/ServerStats.java ---
    @@ -21,17 +21,33 @@
     
     
     import org.apache.zookeeper.common.Time;
    +import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.Timer;
    +import java.util.TimerTask;
    +import java.util.concurrent.atomic.AtomicReference;
     
     /**
      * Basic Server Statistics
      */
     public class ServerStats {
    +    private static final Logger LOG = LoggerFactory.getLogger(ServerStats.class);
    +
         private long packetsSent;
         private long packetsReceived;
         private long maxLatency;
         private long minLatency = Long.MAX_VALUE;
         private long totalLatency = 0;
         private long count = 0;
    +    private long numRequestsAboveThresholdTime = 0;
    +
    +    final static long requestWarnThresholdMs = QuorumPeerConfig.getRequestWarnResponseThresholdMs();
    +    final static Timer timer = new Timer();
    --- End diff --
    
    Have you considered using `ScheduledExecutorService` instead?
    https://docs.oracle.com/javase/7/docs/api/java/util/concurrent/ScheduledExecutorService.html


---

[GitHub] zookeeper issue #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on the issue:

    https://github.com/apache/zookeeper/pull/307
  
    @eribeiro Thank you for your comments. Will post an updated patch soon.
    A small concern regarding the `ServerStats.updateLatency()` method. 
    Should the updated patch have both the methods for now and file another JIRA to look into it?
    And for the documentation, this property is being introduced from `3.5.4` version, Is that correct?
    @hanm 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r128330873
  
    --- Diff: src/java/main/org/apache/zookeeper/server/Request.java ---
    @@ -29,12 +29,14 @@
     import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier;
     import org.apache.zookeeper.txn.TxnHeader;
     
    +
    --- End diff --
    
    Some whitespace changes have been introduced by mistake. Whats the best way to resolve them? I am not changing `Request.java` file at all in the final code.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r184910500
  
    --- Diff: src/java/main/org/apache/zookeeper/server/ServerStats.java ---
    @@ -148,9 +174,46 @@ synchronized public void resetRequestCounters(){
             packetsReceived = 0;
             packetsSent = 0;
         }
    +    synchronized public void resetNumRequestsAboveThresholdTime() {
    +        numRequestsAboveThresholdTime = 0;
    +    }
         synchronized public void reset() {
             resetLatency();
             resetRequestCounters();
    +        resetNumRequestsAboveThresholdTime();
    +    }
    +
    +    public void checkLatency(final ZooKeeperServer zks, Request request) {
    +        long requestLatency = Time.currentElapsedTime() - request.createTime;
    +        boolean enabledAndAboveThreshold = (requestWarnThresholdMs == 0) ||
    +                (requestWarnThresholdMs > -1 && requestLatency > requestWarnThresholdMs);
    --- End diff --
    
    This could be simplified: `boolean enabledAndAboveThreshold = requestWarnThresholdMs != -1 && requestLatency > requestWarnThresholdMs`


---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r127111196
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java ---
    @@ -61,6 +61,7 @@
     
         private static boolean standaloneEnabled = true;
         private static boolean reconfigEnabled = false;
    +    private static int warnThreshold = 10000;
    --- End diff --
    
    I would suggest to rename this field as `warnThresholdMs` or `maxRequestThresholdMs`. The important pare here is the `Ms` suffix.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by tdunning <gi...@git.apache.org>.
Github user tdunning commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r128045203
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ZooKeeperServerMainTest.java ---
    @@ -138,14 +145,56 @@ void delete(File f) throws IOException {
             ServerCnxnFactory getCnxnFactory() {
                 return main.getCnxnFactory();
             }
    +
         }
     
    -    public static  class TestZKSMain extends ZooKeeperServerMain {
    +    public static class TestZKSMain extends ZooKeeperServerMain {
    +
    +        private ServerStats serverStats;
    +
    +        @Override
    +        public ZooKeeperServer getZooKeeperServer(FileTxnSnapLog txnLog, ServerConfig config, ZKDatabase zkDb) {
    +            ZooKeeperServer zooKeeperServer = super.getZooKeeperServer(txnLog, config, zkDb);
    +            serverStats = zooKeeperServer.serverStats();
    +            return zooKeeperServer;
    +        }
    +
    +        @Override
             public void shutdown() {
                 super.shutdown();
             }
         }
     
    +    // Test for ZOOKEEPER-2770 ZooKeeper slow operation log
    +    @Test
    +    public void testRequestWarningThreshold() throws IOException, KeeperException, InterruptedException {
    +        ClientBase.setupTestEnv();
    +
    +        final int CLIENT_PORT = PortAssignment.unique();
    +
    +        MainThread main = new MainThread(CLIENT_PORT, true, null, 0);
    +        main.start();
    +
    +        Assert.assertTrue("waiting for server being up",
    +                ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT,
    +                        CONNECTION_TIMEOUT));
    +        // Get the stats object from the ZooKeeperServer to keep track of high latency requests.
    +        ServerStats stats = main.main.serverStats;
    +
    +        ZooKeeper zk = new ZooKeeper("127.0.0.1:" + CLIENT_PORT,
    +                ClientBase.CONNECTION_TIMEOUT, this);
    +
    +        zk.create("/foo1", "foobar".getBytes(), Ids.OPEN_ACL_UNSAFE,
    +                CreateMode.PERSISTENT);
    +
    +        Assert.assertEquals(new String(zk.getData("/foo1", null, null)), "foobar");
    +        // It takes a while for the counter to get updated sometimes, this is added to reduce flakyness
    +        Thread.sleep(1000);
    --- End diff --
    
    OK. The point of the timeout, however, is that an update has not been applied. The sync() is designed to wait for exactly this.
    
    Code that depends on timeouts is not deterministic in my book. It may just have a low probability of failure.
    
    If this isn't feasible, however, it may be something that has to be lived with.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r127757305
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ZooKeeperServerMainTest.java ---
    @@ -138,14 +145,56 @@ void delete(File f) throws IOException {
             ServerCnxnFactory getCnxnFactory() {
                 return main.getCnxnFactory();
             }
    +
         }
     
    -    public static  class TestZKSMain extends ZooKeeperServerMain {
    +    public static class TestZKSMain extends ZooKeeperServerMain {
    +
    +        private ServerStats serverStats;
    +
    +        @Override
    +        public ZooKeeperServer getZooKeeperServer(FileTxnSnapLog txnLog, ServerConfig config, ZKDatabase zkDb) {
    +            ZooKeeperServer zooKeeperServer = super.getZooKeeperServer(txnLog, config, zkDb);
    +            serverStats = zooKeeperServer.serverStats();
    +            return zooKeeperServer;
    +        }
    +
    +        @Override
             public void shutdown() {
                 super.shutdown();
             }
         }
     
    +    // Test for ZOOKEEPER-2770 ZooKeeper slow operation log
    +    @Test
    +    public void testRequestWarningThreshold() throws IOException, KeeperException, InterruptedException {
    +        ClientBase.setupTestEnv();
    +
    +        final int CLIENT_PORT = PortAssignment.unique();
    +
    +        MainThread main = new MainThread(CLIENT_PORT, true, null, 0);
    +        main.start();
    +
    +        Assert.assertTrue("waiting for server being up",
    +                ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT,
    +                        CONNECTION_TIMEOUT));
    +        // Get the stats object from the ZooKeeperServer to keep track of high latency requests.
    +        ServerStats stats = main.main.serverStats;
    +
    +        ZooKeeper zk = new ZooKeeper("127.0.0.1:" + CLIENT_PORT,
    +                ClientBase.CONNECTION_TIMEOUT, this);
    +
    +        zk.create("/foo1", "foobar".getBytes(), Ids.OPEN_ACL_UNSAFE,
    +                CreateMode.PERSISTENT);
    +
    +        Assert.assertEquals(new String(zk.getData("/foo1", null, null)), "foobar");
    +        // It takes a while for the counter to get updated sometimes, this is added to reduce flakyness
    +        Thread.sleep(1000);
    --- End diff --
    
    @tdunning Yes a conditional timeout is a better option.
    From what I understand, the failure can be cause in test-only scenario where there is no physical network between client and server, which sometimes results in client getting back the acknowledge and the server is just yet to complete incrementing the count. IMO, it is not possible in actual cluster. 
    > Would this be susceptible to a sync() to force the test to come up to date with the leader?
    
    I am not sure what you mean by this. Could you please explain?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r191204931
  
    --- Diff: src/java/main/org/apache/zookeeper/server/ServerStats.java ---
    @@ -148,9 +174,46 @@ synchronized public void resetRequestCounters(){
             packetsReceived = 0;
             packetsSent = 0;
         }
    +    synchronized public void resetNumRequestsAboveThresholdTime() {
    +        numRequestsAboveThresholdTime = 0;
    +    }
         synchronized public void reset() {
             resetLatency();
             resetRequestCounters();
    +        resetNumRequestsAboveThresholdTime();
    +    }
    +
    +    public void checkLatency(final ZooKeeperServer zks, Request request) {
    +        long requestLatency = Time.currentElapsedTime() - request.createTime;
    +        boolean enabledAndAboveThreshold = (requestWarnThresholdMs == 0) ||
    +                (requestWarnThresholdMs > -1 && requestLatency > requestWarnThresholdMs);
    +        if (enabledAndAboveThreshold) {
    +            zks.serverStats().incNumRequestsAboveThresholdTime();
    +
    +            // Try acquiring lock only if not waiting
    +            boolean success = waitForLoggingWarnThresholdMsg.compareAndSet(Boolean.FALSE, Boolean.TRUE);
    +            if (success) {
    +                LOG.warn("Request {} exceeded threshold. Took {} ms", request, requestLatency);
    +                startCount = zks.serverStats().getNumRequestsAboveThresholdTime();
    +                timer.schedule(new TimerTask() {
    +                    @Override
    +                    public void run() {
    +                        long count = zks.serverStats().getNumRequestsAboveThresholdTime() - startCount;
    +                        LOG.warn("Number of requests that exceeded {} ms in past {} ms: {}",
    +                                requestWarnThresholdMs, delayTimeForLoggingWarnThresholdMsg, count);
    +                        waitForLoggingWarnThresholdMsg.set(Boolean.FALSE);
    +                    }
    +                }, delayTimeForLoggingWarnThresholdMsg);
    +            }
    +        }
    +    }
    +
    +    public void setDelayTimeForLoggingWarnThresholdMsg(int delay) {
    +        this.delayTimeForLoggingWarnThresholdMsg = delay;
    --- End diff --
    
    Please make this method `package-private`.
    Also it would be great to have a javadoc comment that the method is for testing purposes only.


---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r128155180
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ZooKeeperServerMainTest.java ---
    @@ -138,14 +145,56 @@ void delete(File f) throws IOException {
             ServerCnxnFactory getCnxnFactory() {
                 return main.getCnxnFactory();
             }
    +
         }
     
    -    public static  class TestZKSMain extends ZooKeeperServerMain {
    +    public static class TestZKSMain extends ZooKeeperServerMain {
    +
    +        private ServerStats serverStats;
    +
    +        @Override
    +        public ZooKeeperServer getZooKeeperServer(FileTxnSnapLog txnLog, ServerConfig config, ZKDatabase zkDb) {
    +            ZooKeeperServer zooKeeperServer = super.getZooKeeperServer(txnLog, config, zkDb);
    +            serverStats = zooKeeperServer.serverStats();
    +            return zooKeeperServer;
    +        }
    +
    +        @Override
             public void shutdown() {
                 super.shutdown();
             }
         }
     
    +    // Test for ZOOKEEPER-2770 ZooKeeper slow operation log
    +    @Test
    +    public void testRequestWarningThreshold() throws IOException, KeeperException, InterruptedException {
    +        ClientBase.setupTestEnv();
    +
    +        final int CLIENT_PORT = PortAssignment.unique();
    +
    +        MainThread main = new MainThread(CLIENT_PORT, true, null, 0);
    +        main.start();
    +
    +        Assert.assertTrue("waiting for server being up",
    +                ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT,
    +                        CONNECTION_TIMEOUT));
    +        // Get the stats object from the ZooKeeperServer to keep track of high latency requests.
    +        ServerStats stats = main.main.serverStats;
    +
    +        ZooKeeper zk = new ZooKeeper("127.0.0.1:" + CLIENT_PORT,
    +                ClientBase.CONNECTION_TIMEOUT, this);
    +
    +        zk.create("/foo1", "foobar".getBytes(), Ids.OPEN_ACL_UNSAFE,
    +                CreateMode.PERSISTENT);
    +
    +        Assert.assertEquals(new String(zk.getData("/foo1", null, null)), "foobar");
    +        // It takes a while for the counter to get updated sometimes, this is added to reduce flakyness
    +        Thread.sleep(1000);
    --- End diff --
    
    > We can always increase the counter before we send a response back to client in FinalRequestProcessor. This will make test deterministic, Or,
    
    Not this approach. It will boil down to the same thing as `ServerStats` which we don't want in the first place. 
    
    > In test, implement retry polling with a timeout and a smaller sleep interval (e..g 10ms instead of 1000ms). We use this pattern a lot in our test. In general we'd like to make deterministic tests as much as possible (in other words, avoid sleep at all cost.) though sometimes that is hard to do due to various reasons.
    
    I will add the polling mechanism with a smaller sleep interval. I would also love to make the test deterministic, however I don't think this will be possible here because of race condition involved between client side and server side processing thread.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r127627537
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ZooKeeperServerMainTest.java ---
    @@ -138,14 +145,56 @@ void delete(File f) throws IOException {
             ServerCnxnFactory getCnxnFactory() {
                 return main.getCnxnFactory();
             }
    +
         }
     
    -    public static  class TestZKSMain extends ZooKeeperServerMain {
    +    public static class TestZKSMain extends ZooKeeperServerMain {
    +
    +        private ServerStats serverStats;
    +
    +        @Override
    +        public ZooKeeperServer getZooKeeperServer(FileTxnSnapLog txnLog, ServerConfig config, ZKDatabase zkDb) {
    +            ZooKeeperServer zooKeeperServer = super.getZooKeeperServer(txnLog, config, zkDb);
    +            serverStats = zooKeeperServer.serverStats();
    +            return zooKeeperServer;
    +        }
    +
    +        @Override
             public void shutdown() {
                 super.shutdown();
             }
         }
     
    +    // Test for ZOOKEEPER-2770 ZooKeeper slow operation log
    +    @Test
    +    public void testRequestWarningThreshold() throws IOException, KeeperException, InterruptedException {
    +        ClientBase.setupTestEnv();
    +
    +        final int CLIENT_PORT = PortAssignment.unique();
    +
    +        MainThread main = new MainThread(CLIENT_PORT, true, null, 0);
    +        main.start();
    +
    +        Assert.assertTrue("waiting for server being up",
    +                ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT,
    +                        CONNECTION_TIMEOUT));
    +        // Get the stats object from the ZooKeeperServer to keep track of high latency requests.
    +        ServerStats stats = main.main.serverStats;
    +
    +        ZooKeeper zk = new ZooKeeper("127.0.0.1:" + CLIENT_PORT,
    +                ClientBase.CONNECTION_TIMEOUT, this);
    +
    +        zk.create("/foo1", "foobar".getBytes(), Ids.OPEN_ACL_UNSAFE,
    +                CreateMode.PERSISTENT);
    +
    +        Assert.assertEquals(new String(zk.getData("/foo1", null, null)), "foobar");
    --- End diff --
    
    question: Is this assertion check really necessary? It's not enough just a `zk.getData("/foo1", null, null);`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on the issue:

    https://github.com/apache/zookeeper/pull/307
  
    Ping @hanm @tdunning @eribeiro @skamille for a review if you got some time. Thanks!


---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 closed the pull request at:

    https://github.com/apache/zookeeper/pull/307


---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by skamille <gi...@git.apache.org>.
Github user skamille commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r129424302
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java ---
    @@ -61,6 +61,7 @@
     
         private static boolean standaloneEnabled = true;
         private static boolean reconfigEnabled = false;
    +    private static int requestWarnThresholdMs = 10000;
    --- End diff --
    
    If we're going to implement this let's at least put some sort of realistic threshold. 10s is basically saying "don't enable this feature" is that what we want?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r127762613
  
    --- Diff: src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java ---
    @@ -460,11 +464,21 @@ public void processRequest(Request request) {
                 if (request.type == OpCode.closeSession) {
                     cnxn.sendCloseSession();
                 }
    +            checkLatency(request);
             } catch (IOException e) {
                 LOG.error("FIXMSG",e);
             }
         }
     
    +    private void checkLatency(Request request) {
    +        long requestWarnThresholdMs = QuorumPeerConfig.getRequestWarnResponseThresholdMs();
    --- End diff --
    
    I am thinking to cache the value of `requestWarnThresholdMs` inside the `FinalRequestProcessor.java`. Is that a good idea?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r187815063
  
    --- Diff: src/java/main/org/apache/zookeeper/server/ServerStats.java ---
    @@ -148,9 +174,46 @@ synchronized public void resetRequestCounters(){
             packetsReceived = 0;
             packetsSent = 0;
         }
    +    synchronized public void resetNumRequestsAboveThresholdTime() {
    +        numRequestsAboveThresholdTime = 0;
    +    }
         synchronized public void reset() {
             resetLatency();
             resetRequestCounters();
    +        resetNumRequestsAboveThresholdTime();
    +    }
    +
    +    public void checkLatency(final ZooKeeperServer zks, Request request) {
    +        long requestLatency = Time.currentElapsedTime() - request.createTime;
    +        boolean enabledAndAboveThreshold = (requestWarnThresholdMs == 0) ||
    +                (requestWarnThresholdMs > -1 && requestLatency > requestWarnThresholdMs);
    --- End diff --
    
    The reason for explicit check of condition `requestWarnThresholdMs == 0` is two fold. One is for debugging scenarios where you want to log time for each request and for test purposes. There were cases when request would finish in less than 1 ms and hence the timer would show `requestLatency` as 0. It is added to make my test `HighLatencyRequestLoggingTest` not flaky. What do you think?


---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r127633626
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ZooKeeperServerMainTest.java ---
    @@ -138,14 +145,56 @@ void delete(File f) throws IOException {
             ServerCnxnFactory getCnxnFactory() {
                 return main.getCnxnFactory();
             }
    +
         }
     
    -    public static  class TestZKSMain extends ZooKeeperServerMain {
    +    public static class TestZKSMain extends ZooKeeperServerMain {
    +
    +        private ServerStats serverStats;
    +
    +        @Override
    +        public ZooKeeperServer getZooKeeperServer(FileTxnSnapLog txnLog, ServerConfig config, ZKDatabase zkDb) {
    +            ZooKeeperServer zooKeeperServer = super.getZooKeeperServer(txnLog, config, zkDb);
    +            serverStats = zooKeeperServer.serverStats();
    +            return zooKeeperServer;
    +        }
    +
    +        @Override
             public void shutdown() {
                 super.shutdown();
             }
         }
     
    +    // Test for ZOOKEEPER-2770 ZooKeeper slow operation log
    +    @Test
    +    public void testRequestWarningThreshold() throws IOException, KeeperException, InterruptedException {
    +        ClientBase.setupTestEnv();
    +
    +        final int CLIENT_PORT = PortAssignment.unique();
    +
    +        MainThread main = new MainThread(CLIENT_PORT, true, null, 0);
    +        main.start();
    +
    +        Assert.assertTrue("waiting for server being up",
    +                ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT,
    +                        CONNECTION_TIMEOUT));
    +        // Get the stats object from the ZooKeeperServer to keep track of high latency requests.
    +        ServerStats stats = main.main.serverStats;
    +
    +        ZooKeeper zk = new ZooKeeper("127.0.0.1:" + CLIENT_PORT,
    +                ClientBase.CONNECTION_TIMEOUT, this);
    +
    +        zk.create("/foo1", "foobar".getBytes(), Ids.OPEN_ACL_UNSAFE,
    +                CreateMode.PERSISTENT);
    +
    +        Assert.assertEquals(new String(zk.getData("/foo1", null, null)), "foobar");
    --- End diff --
    
    Not necessary as such, but will be useful in debugging if some change to `getData()` method might cause this test to fail. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r127110211
  
    --- Diff: src/java/main/org/apache/zookeeper/server/Request.java ---
    @@ -121,6 +128,13 @@ public void setTxn(Record txn) {
             this.txn = txn;
         }
     
    +    public void checkLatency() {
    +        long requestLatency = Time.currentElapsedTime() - this.createTime;
    +        if(QuorumPeerConfig.getWarnResponseTime() > -1 && requestLatency > QuorumPeerConfig.getWarnResponseTime()) {
    --- End diff --
    
    nit: space between `if` and `(`.
    
    You can reduce the size of the `if` expression and avoid an unnecessary call to the same method in the same expression by doing:
    
    ``` java
    long warnRequestThreshold = QuorumPeerConfig.getWarnRequestTime()
    if (warnRequestThreshold > -1 && requestLatency > warnRequestThreshold) {
    ```
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r147047396
  
    --- Diff: src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java ---
    @@ -81,9 +81,11 @@
         private static final Logger LOG = LoggerFactory.getLogger(FinalRequestProcessor.class);
     
         ZooKeeperServer zks;
    +    ServerStats serverStats;
    --- End diff --
    
    Fixed it in the next commit.


---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r127627334
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java ---
    @@ -292,6 +293,14 @@ public void parseProperties(Properties zkProp)
                     }
                 } else if ((key.startsWith("server.") || key.startsWith("group") || key.startsWith("weight")) && zkProp.containsKey("dynamicConfigFile")) {
                     throw new ConfigException("parameter: " + key + " must be in a separate dynamic config file");
    +            } else if(key.equals("request.warningthresholdms")) {
    --- End diff --
    
    nit: space between `if` and `(`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r127628162
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ZooKeeperServerMainTest.java ---
    @@ -27,6 +27,7 @@
     import java.util.concurrent.CountDownLatch;
     import java.util.concurrent.TimeUnit;
     
    +import org.apache.log4j.BasicConfigurator;
    --- End diff --
    
    Unused import


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on the issue:

    https://github.com/apache/zookeeper/pull/307
  
    Hello everyone,
    Appreciate your efforts in reviewing this patch. @hanm @tdunning @eribeiro @skamille 
    Is there any possibility that the patch will get merged in (with minor changes if required) or shall we 'never' this JIRA and close this PR?
    Thanks!


---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r191213569
  
    --- Diff: src/java/main/org/apache/zookeeper/server/ServerStats.java ---
    @@ -21,17 +21,33 @@
     
     
     import org.apache.zookeeper.common.Time;
    +import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.Timer;
    +import java.util.TimerTask;
    +import java.util.concurrent.atomic.AtomicReference;
     
     /**
      * Basic Server Statistics
      */
     public class ServerStats {
    +    private static final Logger LOG = LoggerFactory.getLogger(ServerStats.class);
    +
         private long packetsSent;
         private long packetsReceived;
         private long maxLatency;
         private long minLatency = Long.MAX_VALUE;
         private long totalLatency = 0;
         private long count = 0;
    +    private long numRequestsAboveThresholdTime = 0;
    --- End diff --
    
    Also make it `private` please.


---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by tdunning <gi...@git.apache.org>.
Github user tdunning commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r127711561
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ZooKeeperServerMainTest.java ---
    @@ -138,14 +145,56 @@ void delete(File f) throws IOException {
             ServerCnxnFactory getCnxnFactory() {
                 return main.getCnxnFactory();
             }
    +
         }
     
    -    public static  class TestZKSMain extends ZooKeeperServerMain {
    +    public static class TestZKSMain extends ZooKeeperServerMain {
    +
    +        private ServerStats serverStats;
    +
    +        @Override
    +        public ZooKeeperServer getZooKeeperServer(FileTxnSnapLog txnLog, ServerConfig config, ZKDatabase zkDb) {
    +            ZooKeeperServer zooKeeperServer = super.getZooKeeperServer(txnLog, config, zkDb);
    +            serverStats = zooKeeperServer.serverStats();
    +            return zooKeeperServer;
    +        }
    +
    +        @Override
             public void shutdown() {
                 super.shutdown();
             }
         }
     
    +    // Test for ZOOKEEPER-2770 ZooKeeper slow operation log
    +    @Test
    +    public void testRequestWarningThreshold() throws IOException, KeeperException, InterruptedException {
    +        ClientBase.setupTestEnv();
    +
    +        final int CLIENT_PORT = PortAssignment.unique();
    +
    +        MainThread main = new MainThread(CLIENT_PORT, true, null, 0);
    +        main.start();
    +
    +        Assert.assertTrue("waiting for server being up",
    +                ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT,
    +                        CONNECTION_TIMEOUT));
    +        // Get the stats object from the ZooKeeperServer to keep track of high latency requests.
    +        ServerStats stats = main.main.serverStats;
    +
    +        ZooKeeper zk = new ZooKeeper("127.0.0.1:" + CLIENT_PORT,
    +                ClientBase.CONNECTION_TIMEOUT, this);
    +
    +        zk.create("/foo1", "foobar".getBytes(), Ids.OPEN_ACL_UNSAFE,
    +                CreateMode.PERSISTENT);
    +
    +        Assert.assertEquals(new String(zk.getData("/foo1", null, null)), "foobar");
    +        // It takes a while for the counter to get updated sometimes, this is added to reduce flakyness
    +        Thread.sleep(1000);
    --- End diff --
    
    At the very least, make the timeout conditional on getting the wrong answer. That avoids paying the penalty of a slow test every time.
    
    The mere presence of a timeout indicates a problem with concurrency, however. Would this be susceptible to a sync() to force the test to come up to date with the leader? If so, that can make the test deterministic.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r127363108
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java ---
    @@ -61,6 +61,7 @@
     
         private static boolean standaloneEnabled = true;
         private static boolean reconfigEnabled = false;
    +    private static int warnThreshold = 10000;
    --- End diff --
    
    I suggest having to turn on logging by default. Won't hurt much as such. People can opt out if they want. Its similar to usual stats collection IMO.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on the issue:

    https://github.com/apache/zookeeper/pull/307
  
    @karanmehta93 hi, mate.
    
    You don't need to touch `ZooKeeperAdmin.html` You change `ZooKeeperAdmin.xml` **only** and Apache Forrest generate html anf PDF. Also, this xml change should be part of this patch, okay?
    
    Cheers!



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r127618988
  
    --- Diff: src/java/main/org/apache/zookeeper/server/ServerStats.java ---
    @@ -32,6 +32,7 @@
         private long minLatency = Long.MAX_VALUE;
         private long totalLatency = 0;
         private long count = 0;
    +    private long numRequestsExceededThresholdTime = 0;
    --- End diff --
    
    The addition of this metric is missing a `resetRequestsExceededThresholdTime` method to zero the counter. ;)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on the issue:

    https://github.com/apache/zookeeper/pull/307
  
    @tdunning Sorry about this. I rebased my branch with the current master and pushed my patch along with it. Now it is showing up all those commits as my commits. Let me try to figure out a way to resolve this.


---

[GitHub] zookeeper issue #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by tdunning <gi...@git.apache.org>.
Github user tdunning commented on the issue:

    https://github.com/apache/zookeeper/pull/307
  
    What is this pull request?  45 commits? 95 changed files? Lots of commits without associated issues?
    
    (danger, danger)


---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r191201829
  
    --- Diff: src/java/main/org/apache/zookeeper/server/ServerStats.java ---
    @@ -106,6 +126,7 @@ public String toString(){
             StringBuilder sb = new StringBuilder();
             sb.append("Latency min/avg/max: " + getMinLatency() + "/"
                     + getAvgLatency() + "/" + getMaxLatency() + "\n");
    +        sb.append("Num Requests that exceeded threshold latency: " + getNumRequestsAboveThresholdTime() + "\n");
    --- End diff --
    
    This is going to be printed in `stat` 4lw command which makes things a little bit complicated.
    
    - what's the target branch/version of this patch? As we're in the middle of 3.5 stabilisation, I would suggest to target this trunk/3.6-only. Please let me know if you've concerns and please update the Jira fix version accordingly.
    - Output of 4lw commands are parsed by machines occasionally, so adding new fields in the middle of the ouput would potentially brake backward compatibility. Adding them to the end of the output could be cumbersome in this case, but:
    - 4LWs have some security flaws and we should decommission them in the upcoming ZooKeeper versions in favour of JMX and Jetty interfaces,
    - I recommend not adding the new metric here, but expose it via JMX and Jetty `stat` and `monitor` commands.



---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r184910779
  
    --- Diff: src/java/main/org/apache/zookeeper/server/ServerStats.java ---
    @@ -148,9 +174,46 @@ synchronized public void resetRequestCounters(){
             packetsReceived = 0;
             packetsSent = 0;
         }
    +    synchronized public void resetNumRequestsAboveThresholdTime() {
    +        numRequestsAboveThresholdTime = 0;
    +    }
         synchronized public void reset() {
             resetLatency();
             resetRequestCounters();
    +        resetNumRequestsAboveThresholdTime();
    +    }
    +
    +    public void checkLatency(final ZooKeeperServer zks, Request request) {
    +        long requestLatency = Time.currentElapsedTime() - request.createTime;
    +        boolean enabledAndAboveThreshold = (requestWarnThresholdMs == 0) ||
    +                (requestWarnThresholdMs > -1 && requestLatency > requestWarnThresholdMs);
    +        if (enabledAndAboveThreshold) {
    +            zks.serverStats().incNumRequestsAboveThresholdTime();
    +
    +            // Try acquiring lock only if not waiting
    +            boolean success = waitForLoggingWarnThresholdMsg.compareAndSet(Boolean.FALSE, Boolean.TRUE);
    +            if (success) {
    +                LOG.warn("Request {} exceeded threshold. Took {} ms", request, requestLatency);
    +                startCount = zks.serverStats().getNumRequestsAboveThresholdTime();
    +                timer.schedule(new TimerTask() {
    +                    @Override
    +                    public void run() {
    +                        long count = zks.serverStats().getNumRequestsAboveThresholdTime() - startCount;
    +                        LOG.warn("Number of requests that exceeded {} ms in past {} ms: {}",
    +                                requestWarnThresholdMs, delayTimeForLoggingWarnThresholdMsg, count);
    +                        waitForLoggingWarnThresholdMsg.set(Boolean.FALSE);
    +                    }
    +                }, delayTimeForLoggingWarnThresholdMsg);
    +            }
    +        }
    +    }
    +
    +    public void setDelayTimeForLoggingWarnThresholdMsg(int delay) {
    +        this.delayTimeForLoggingWarnThresholdMsg = delay;
    --- End diff --
    
    I think it is worth to add some comment about this method. Probably also add comments in the document as well about the behavior of the log and how to control the rate using this method.


---

[GitHub] zookeeper issue #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on the issue:

    https://github.com/apache/zookeeper/pull/307
  
    @hanm @eribeiro @tdunning 
    I have fixed the whitespace issues and final patch is ready, Please review whenever convenient.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r127117526
  
    --- Diff: src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java ---
    @@ -430,6 +432,7 @@ public void processRequest(Request request) {
                 // the client and leader disagree on where the client is most
                 // recently attached (and therefore invalid SESSION MOVED generated)
                 cnxn.sendCloseSession();
    +            request.checkLatency();
    --- End diff --
    
    On a related note, besides logging those slow requests, wouldn't be nice to expose a counter of how many requests were above the latency threshold and store it in `zks.serverStats()` so that it could be exposed as metric via JMX?
    
    Going even further, we could expose other metrics related to slow requests like average time & 99% percentiles as suggested by @tdunning


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on the issue:

    https://github.com/apache/zookeeper/pull/307
  
    @hanm @eribeiro 
    Thank you for your comments. I have added a rebased patch with a test in `ZooKeeperServerMainTest` class. Please review whenever convenient. 
    
    > It's odd to have two (or in future even more) types of latency checks scattered around which creates fragmentation w.r.t. the definition of what a request latency means. 
    
    We can decide based on opinion from other people and I can update the patch accordingly. I will create new JIRA's based on @tdunning suggestions, after this one gets approved. I am eager to take them up.
    
    Documentation is still pending. Will add a patch for it soon. I am not sure if I have to modify the `ZooKeeperAdmin.xml` file or `ZooKeeperAdmin.html` file. Can you point me to any relevant stuff?



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r146108372
  
    --- Diff: src/java/main/org/apache/zookeeper/server/ServerStats.java ---
    @@ -148,9 +173,47 @@ synchronized public void resetRequestCounters(){
             packetsReceived = 0;
             packetsSent = 0;
         }
    +    synchronized public void resetNumRequestsAboveThresholdTime() {
    +        numRequestsAboveThresholdTime = 0;
    +    }
         synchronized public void reset() {
             resetLatency();
             resetRequestCounters();
    +        resetNumRequestsAboveThresholdTime();
    +    }
    +
    +    public void checkLatency(final ZooKeeperServer zks, Request request) {
    +        long requestLatency = Time.currentElapsedTime() - request.createTime;
    +        boolean enabledAndAboveThreshold = (requestWarnThresholdMs == 0) ||
    +                (requestWarnThresholdMs > -1 && requestLatency > requestWarnThresholdMs);
    +        if (enabledAndAboveThreshold) {
    +            zks.serverStats().incNumRequestsAboveThresholdTime();
    +            // Try acquiring lock only if not waiting
    +            if(!waitForLoggingWarnThresholdMsg) {
    --- End diff --
    
    nit: space between `if` and `(`


---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
GitHub user karanmehta93 reopened a pull request:

    https://github.com/apache/zookeeper/pull/307

    ZOOKEEPER-2770 ZooKeeper slow operation log

    ZooKeeper is a complex distributed application. There are many reasons why any given read or write operation may become slow: a software bug, a protocol problem, a hardware issue with the commit log(s), a network issue. If the problem is constant it is trivial to come to an understanding of the cause. However in order to diagnose intermittent problems we often don't know where, or when, to begin looking. We need some sort of timestamped indication of the problem. Although ZooKeeper is not a datastore, it does persist data, and can suffer intermittent performance degradation, and should consider implementing a 'slow query' log, a feature very common to services which persist information on behalf of clients which may be sensitive to latency while waiting for confirmation of successful persistence.
    
    The patch sets a default threshold of 10 sec for request. If the value of warn.responseTime is configured as -1 then logging is ignored.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/karanmehta93/zookeeper ZOOKEEPER-2770

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/zookeeper/pull/307.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #307
    
----
commit bf0ed9eca106faebca6548122108e3ff123b04d3
Author: Karan Mehta <ka...@gmail.com>
Date:   2017-10-26T01:48:56Z

    ZOOKEEPER-2770 ZooKeeper slow operation log

----


---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r131281803
  
    --- Diff: src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java ---
    @@ -460,11 +465,20 @@ public void processRequest(Request request) {
                 if (request.type == OpCode.closeSession) {
                     cnxn.sendCloseSession();
                 }
    +            checkLatency(request);
             } catch (IOException e) {
                 LOG.error("FIXMSG",e);
             }
         }
     
    +    private void checkLatency(Request request) {
    +        long requestLatency = Time.currentElapsedTime() - request.createTime;
    +        if((requestWarnThresholdMs > -1 && requestLatency > requestWarnThresholdMs) || requestWarnThresholdMs == 0) {
    --- End diff --
    
    @karanmehta93 really *really* minor nit: 
    ```
    boolean enabledAndAboveThreshold = requestWarnThresholdMs == 0 || (requestWarnThresholdMs > -1 && requestLatency > requestWarnThresholdMs);
    
    if (enabledAndAboveThreshold) {
    ```
    
    I moved `requestWarnThresholdMs == 0` to the beginning of the expression so that the whole expression can be short circuited if it is equals zero. ;-)
    
    But this is a so trivial recommendation that it's up to you use it or not.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r146108249
  
    --- Diff: src/java/main/org/apache/zookeeper/server/ServerStats.java ---
    @@ -21,17 +21,32 @@
     
     
     import org.apache.zookeeper.common.Time;
    +import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.Timer;
    +import java.util.TimerTask;
     
     /**
      * Basic Server Statistics
      */
     public class ServerStats {
    +    private static final Logger LOG = LoggerFactory.getLogger(ServerStats.class);
    +
         private long packetsSent;
         private long packetsReceived;
         private long maxLatency;
         private long minLatency = Long.MAX_VALUE;
         private long totalLatency = 0;
         private long count = 0;
    +    private long numRequestsAboveThresholdTime = 0;
    +
    +    final static long requestWarnThresholdMs = QuorumPeerConfig.getRequestWarnResponseThresholdMs();
    +    final static Timer timer = new Timer();
    +    volatile Boolean waitForLoggingWarnThresholdMsg = false;
    --- End diff --
    
    What is the reason to make `waitForLoggingWarnThresholdMsg` a boxed type instead of a primitive `boolean`?


---

[GitHub] zookeeper issue #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on the issue:

    https://github.com/apache/zookeeper/pull/307
  
    looks like the branch has merge conflicts. do you mind update your patch with latest master and I'll review @karanmehta93 


---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r128122188
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ZooKeeperServerMainTest.java ---
    @@ -138,14 +145,56 @@ void delete(File f) throws IOException {
             ServerCnxnFactory getCnxnFactory() {
                 return main.getCnxnFactory();
             }
    +
         }
     
    -    public static  class TestZKSMain extends ZooKeeperServerMain {
    +    public static class TestZKSMain extends ZooKeeperServerMain {
    +
    +        private ServerStats serverStats;
    +
    +        @Override
    +        public ZooKeeperServer getZooKeeperServer(FileTxnSnapLog txnLog, ServerConfig config, ZKDatabase zkDb) {
    +            ZooKeeperServer zooKeeperServer = super.getZooKeeperServer(txnLog, config, zkDb);
    +            serverStats = zooKeeperServer.serverStats();
    +            return zooKeeperServer;
    +        }
    +
    +        @Override
             public void shutdown() {
                 super.shutdown();
             }
         }
     
    +    // Test for ZOOKEEPER-2770 ZooKeeper slow operation log
    +    @Test
    +    public void testRequestWarningThreshold() throws IOException, KeeperException, InterruptedException {
    +        ClientBase.setupTestEnv();
    +
    +        final int CLIENT_PORT = PortAssignment.unique();
    +
    +        MainThread main = new MainThread(CLIENT_PORT, true, null, 0);
    +        main.start();
    +
    +        Assert.assertTrue("waiting for server being up",
    +                ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT,
    +                        CONNECTION_TIMEOUT));
    +        // Get the stats object from the ZooKeeperServer to keep track of high latency requests.
    +        ServerStats stats = main.main.serverStats;
    +
    +        ZooKeeper zk = new ZooKeeper("127.0.0.1:" + CLIENT_PORT,
    +                ClientBase.CONNECTION_TIMEOUT, this);
    +
    +        zk.create("/foo1", "foobar".getBytes(), Ids.OPEN_ACL_UNSAFE,
    +                CreateMode.PERSISTENT);
    +
    +        Assert.assertEquals(new String(zk.getData("/foo1", null, null)), "foobar");
    +        // It takes a while for the counter to get updated sometimes, this is added to reduce flakyness
    +        Thread.sleep(1000);
    --- End diff --
    
    @tdunning 
    
    > OK. The point of the timeout, however, is that an update has not been applied. The sync() is designed to wait for exactly this.
    
    Where does the update have to applied? To ZKDb or the counter? This is what my understanding is, correct me where I am getting it wrong. The call to `zk.getData()` is a blocking call. The `FinalRequestProcessor` completes all the relevant task related to this request and in the end sends out a reply packet to the client. It puts the packet on a queue which is handled by the `ServerCnxn` on a separate thread. In the meanwhile, the former thread goes and increments the `numRequestsAboveThresholdTime` counter. In a case when client gets acknowledged about the data being persisted at the node, the test proceeds to check the counter value. If the later thread got suspended due to any reason and didn't update the counter, this might cause the test the fail.
    
    > Code that depends on timeouts is not deterministic in my book. It may just have a low probability of failure.
    
    Agreed, using the term `completely deterministic` on my part was incorrect.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r191199501
  
    --- Diff: src/java/main/org/apache/zookeeper/server/ServerStats.java ---
    @@ -21,17 +21,33 @@
     
     
     import org.apache.zookeeper.common.Time;
    +import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.Timer;
    +import java.util.TimerTask;
    +import java.util.concurrent.atomic.AtomicReference;
     
     /**
      * Basic Server Statistics
      */
     public class ServerStats {
    +    private static final Logger LOG = LoggerFactory.getLogger(ServerStats.class);
    +
         private long packetsSent;
         private long packetsReceived;
         private long maxLatency;
         private long minLatency = Long.MAX_VALUE;
         private long totalLatency = 0;
         private long count = 0;
    +    private long numRequestsAboveThresholdTime = 0;
    --- End diff --
    
    You can use `AtomicLong` instead of acquiring class-level lock to access this.


---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r187815218
  
    --- Diff: src/java/main/org/apache/zookeeper/server/ServerStats.java ---
    @@ -148,9 +174,46 @@ synchronized public void resetRequestCounters(){
             packetsReceived = 0;
             packetsSent = 0;
         }
    +    synchronized public void resetNumRequestsAboveThresholdTime() {
    +        numRequestsAboveThresholdTime = 0;
    +    }
         synchronized public void reset() {
             resetLatency();
             resetRequestCounters();
    +        resetNumRequestsAboveThresholdTime();
    +    }
    +
    +    public void checkLatency(final ZooKeeperServer zks, Request request) {
    +        long requestLatency = Time.currentElapsedTime() - request.createTime;
    +        boolean enabledAndAboveThreshold = (requestWarnThresholdMs == 0) ||
    +                (requestWarnThresholdMs > -1 && requestLatency > requestWarnThresholdMs);
    +        if (enabledAndAboveThreshold) {
    +            zks.serverStats().incNumRequestsAboveThresholdTime();
    +
    +            // Try acquiring lock only if not waiting
    +            boolean success = waitForLoggingWarnThresholdMsg.compareAndSet(Boolean.FALSE, Boolean.TRUE);
    +            if (success) {
    +                LOG.warn("Request {} exceeded threshold. Took {} ms", request, requestLatency);
    +                startCount = zks.serverStats().getNumRequestsAboveThresholdTime();
    +                timer.schedule(new TimerTask() {
    +                    @Override
    +                    public void run() {
    +                        long count = zks.serverStats().getNumRequestsAboveThresholdTime() - startCount;
    --- End diff --
    
    > If there is no slow requests coming in between the time the task is scheduled and the time the task is executed, this count will be 0. It will not reflect the actual number of requests we want to log. Maybe log the startCount instead and reset it and leave the task only to reset the barrier?
    
    The whole idea behind the schedule timer is to see the requests that exceeded threshold in the past rolling window of 60 seconds. Since the task is only scheduled once a high latency request is logged, it is fine to say that 0 requests had longer times since the last bad request was logged. The total count can be seen using `stat` at any point of time. What do you suggest?
    
    > Also please use ScheduledService instead of Timer task
    
    Sure will update.



---

[GitHub] zookeeper issue #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on the issue:

    https://github.com/apache/zookeeper/pull/307
  
    @hanm Thank you! I have rebased the patch. The jenkins build has been failing for a while and my patch doesn't introduce new failure. Thanks once again!


---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r192604269
  
    --- Diff: src/java/main/org/apache/zookeeper/server/ServerStats.java ---
    @@ -148,9 +174,46 @@ synchronized public void resetRequestCounters(){
             packetsReceived = 0;
             packetsSent = 0;
         }
    +    synchronized public void resetNumRequestsAboveThresholdTime() {
    +        numRequestsAboveThresholdTime = 0;
    +    }
         synchronized public void reset() {
             resetLatency();
             resetRequestCounters();
    +        resetNumRequestsAboveThresholdTime();
    +    }
    +
    +    public void checkLatency(final ZooKeeperServer zks, Request request) {
    +        long requestLatency = Time.currentElapsedTime() - request.createTime;
    +        boolean enabledAndAboveThreshold = (requestWarnThresholdMs == 0) ||
    +                (requestWarnThresholdMs > -1 && requestLatency > requestWarnThresholdMs);
    +        if (enabledAndAboveThreshold) {
    +            zks.serverStats().incNumRequestsAboveThresholdTime();
    +
    +            // Try acquiring lock only if not waiting
    +            boolean success = waitForLoggingWarnThresholdMsg.compareAndSet(Boolean.FALSE, Boolean.TRUE);
    +            if (success) {
    +                LOG.warn("Request {} exceeded threshold. Took {} ms", request, requestLatency);
    +                startCount = zks.serverStats().getNumRequestsAboveThresholdTime();
    +                timer.schedule(new TimerTask() {
    +                    @Override
    +                    public void run() {
    +                        long count = zks.serverStats().getNumRequestsAboveThresholdTime() - startCount;
    --- End diff --
    
    Okay. I got it now. Will update it to not log anything.
    The whole idea behind `ScheduledExecutorService` is that you can regularly output such metrics to any relevant reader.


---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r127618910
  
    --- Diff: src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java ---
    @@ -460,11 +464,21 @@ public void processRequest(Request request) {
                 if (request.type == OpCode.closeSession) {
                     cnxn.sendCloseSession();
                 }
    +            checkLatency(request);
             } catch (IOException e) {
                 LOG.error("FIXMSG",e);
             }
         }
     
    +    private void checkLatency(Request request) {
    +        long requestWarnThresholdMs = QuorumPeerConfig.getRequestWarnResponseThresholdMs();
    +        long requestLatency = Time.currentElapsedTime() - request.createTime;
    +        if((requestWarnThresholdMs > -1 && requestLatency > requestWarnThresholdMs) || requestWarnThresholdMs == 0) {
    +            LOG.warn("Request "  + request + " exceeded threshold. Took " + requestLatency + " ms.");
    +            zks.serverStats().updateNumRequestsExceededThresholdTime();
    --- End diff --
    
    Cool! :) 
    
    nit: Two suggestions: to change the prefix from `update` to `inc` and use `numRequestsAboveThresholdTime()` (up to you both).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by skamille <gi...@git.apache.org>.
Github user skamille commented on the issue:

    https://github.com/apache/zookeeper/pull/307
  
    I appreciate the work that you are trying to do here, and this still feels like an incomplete approach to a problem that would be worth investing further into. It seems like we all agree that it would be good to have clearer metrics about the performance behavior of the internals of the system. This implements some basic infrastructure for a metric recording framework, coupled to the recording of one metric. Can we at least make it so that the process of "push a metric to a buffer, have a thread that wakes up periodically and flushes information out of that buffer" is usable by multiple parts of the system, instead of coupling it to the one metric of request time?


---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r191213705
  
    --- Diff: src/java/main/org/apache/zookeeper/server/ServerStats.java ---
    @@ -21,17 +21,33 @@
     
     
     import org.apache.zookeeper.common.Time;
    +import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.Timer;
    +import java.util.TimerTask;
    +import java.util.concurrent.atomic.AtomicReference;
     
     /**
      * Basic Server Statistics
      */
     public class ServerStats {
    +    private static final Logger LOG = LoggerFactory.getLogger(ServerStats.class);
    +
         private long packetsSent;
         private long packetsReceived;
         private long maxLatency;
         private long minLatency = Long.MAX_VALUE;
         private long totalLatency = 0;
         private long count = 0;
    +    private long numRequestsAboveThresholdTime = 0;
    +
    +    final static long requestWarnThresholdMs = QuorumPeerConfig.getRequestWarnResponseThresholdMs();
    --- End diff --
    
    Would you please make it `private`?


---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r191203530
  
    --- Diff: src/java/main/org/apache/zookeeper/server/ServerStats.java ---
    @@ -148,9 +174,46 @@ synchronized public void resetRequestCounters(){
             packetsReceived = 0;
             packetsSent = 0;
         }
    +    synchronized public void resetNumRequestsAboveThresholdTime() {
    +        numRequestsAboveThresholdTime = 0;
    +    }
         synchronized public void reset() {
             resetLatency();
             resetRequestCounters();
    +        resetNumRequestsAboveThresholdTime();
    +    }
    +
    +    public void checkLatency(final ZooKeeperServer zks, Request request) {
    +        long requestLatency = Time.currentElapsedTime() - request.createTime;
    +        boolean enabledAndAboveThreshold = (requestWarnThresholdMs == 0) ||
    +                (requestWarnThresholdMs > -1 && requestLatency > requestWarnThresholdMs);
    --- End diff --
    
    I think that makes sense.


---

[GitHub] zookeeper issue #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on the issue:

    https://github.com/apache/zookeeper/pull/307
  
    >> And for the documentation, this property is being introduced from 3.5.4 version, Is that correct?
    
    Yes, for this specific pull request targeting master. I think at some point after merge this you need create another pull request targeting stable branch (branch-3.4) where the version number should be 3.4.11 (hopefully, if this can get in before next release).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on the issue:

    https://github.com/apache/zookeeper/pull/307
  
    Agreed @zodvik . The scope of this Jira is to limit it to a generic slow query log and identify the outliers if any. For that reason, I am printing the actual request to the log as well. An improvement would be definitely to separate based on request types as others have also suggested. If this gets in, I would like to work on the improvement part of using t-digest(https://github.com/tdunning/t-digest) to give out detailed insight and not have the problem of setting a fixed threshold. 


---

[GitHub] zookeeper issue #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by tdunning <gi...@git.apache.org>.
Github user tdunning commented on the issue:

    https://github.com/apache/zookeeper/pull/307
  
    So here is an alternative suggestion.
    
    For each class of operation (GET, PUT, MULTIPUT, DELETE and so on) maintain a data structure (such as t-digest or Floathistogram see https://github.com/tdunning/t-digest) that keeps track of the distribution of latencies for that operation.
    
    Log any transactions that are above a settable percentile latency for operations of that class. Default could be 99.99%.
    
    Operations longer than 2x the 99.9%-ile latency should not be entered into the historical distribution.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r126866390
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java ---
    @@ -292,6 +293,8 @@ public void parseProperties(Properties zkProp)
                     }
                 } else if ((key.startsWith("server.") || key.startsWith("group") || key.startsWith("weight")) && zkProp.containsKey("dynamicConfigFile")) {
                     throw new ConfigException("parameter: " + key + " must be in a separate dynamic config file");
    +            } else if(key.equals("warn.responseTime")) {
    --- End diff --
    
    This is a new property, so it should be:
    
    * Documented - in particular mentions that this log can be disabled if set to -1. We don't want user to check code to figure out how to use this.
    * Add a test case to cover various code paths.
    
    Also, the name used here could be improved. I think something like request.warningthresholdms is more accurate. We already have fsync.warningthresholdms which in many ways has similar functionality of what's being added, so use that as a reference.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r127356689
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java ---
    @@ -61,6 +61,7 @@
     
         private static boolean standaloneEnabled = true;
         private static boolean reconfigEnabled = false;
    +    private static int warnThreshold = 10000;
    --- End diff --
    
    Should this have a default value of -1 instead? I think we'd like this to be an opt in feature and by default the logging should be disabled.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r127627096
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ZooKeeperServerMainTest.java ---
    @@ -64,10 +65,15 @@
     
             public MainThread(int clientPort, boolean preCreateDirs, String configs)
                     throws IOException {
    -            this(clientPort, preCreateDirs, ClientBase.createTmpDir(), configs);
    +            this(clientPort, preCreateDirs, ClientBase.createTmpDir(), configs, -1);
             }
     
    -        public MainThread(int clientPort, boolean preCreateDirs, File tmpDir, String configs)
    +        public MainThread(int clientPort, boolean preCreateDirs, String configs, int threshold)
    --- End diff --
    
    the `threshold` word in method signature doesn't say much so it requires digging the code to see what it's all about. It would be nice to rename to something closer to its intent (there are plenty of names in this patch already).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r127127351
  
    --- Diff: src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java ---
    @@ -430,6 +432,7 @@ public void processRequest(Request request) {
                 // the client and leader disagree on where the client is most
                 // recently attached (and therefore invalid SESSION MOVED generated)
                 cnxn.sendCloseSession();
    +            request.checkLatency();
    --- End diff --
    
    @eribeiro 
    That is exactly the reason that I created my own function after the `cnxn.sendCloseSession()` since I was interested in end to end latency, and didn't use the `zks.serverStats().updateLatency(request.createTime);` method. I would also like to know opinion from others on this. 
    Should we scope this JIRA for adding threshold as a general and create a new JIRA for extending the scope to include percentiles with new data-structure inside the `ServerStats` class?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #307: ZOOKEEPER-2770 ZooKeeper slow operation log

Posted by karanmehta93 <gi...@git.apache.org>.
Github user karanmehta93 commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/307#discussion_r127633278
  
    --- Diff: src/java/test/org/apache/zookeeper/server/ZooKeeperServerMainTest.java ---
    @@ -138,14 +145,56 @@ void delete(File f) throws IOException {
             ServerCnxnFactory getCnxnFactory() {
                 return main.getCnxnFactory();
             }
    +
         }
     
    -    public static  class TestZKSMain extends ZooKeeperServerMain {
    +    public static class TestZKSMain extends ZooKeeperServerMain {
    +
    +        private ServerStats serverStats;
    +
    +        @Override
    +        public ZooKeeperServer getZooKeeperServer(FileTxnSnapLog txnLog, ServerConfig config, ZKDatabase zkDb) {
    +            ZooKeeperServer zooKeeperServer = super.getZooKeeperServer(txnLog, config, zkDb);
    +            serverStats = zooKeeperServer.serverStats();
    +            return zooKeeperServer;
    +        }
    +
    +        @Override
             public void shutdown() {
                 super.shutdown();
             }
         }
     
    +    // Test for ZOOKEEPER-2770 ZooKeeper slow operation log
    +    @Test
    +    public void testRequestWarningThreshold() throws IOException, KeeperException, InterruptedException {
    +        ClientBase.setupTestEnv();
    +
    +        final int CLIENT_PORT = PortAssignment.unique();
    +
    +        MainThread main = new MainThread(CLIENT_PORT, true, null, 0);
    +        main.start();
    +
    +        Assert.assertTrue("waiting for server being up",
    +                ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT,
    +                        CONNECTION_TIMEOUT));
    +        // Get the stats object from the ZooKeeperServer to keep track of high latency requests.
    +        ServerStats stats = main.main.serverStats;
    +
    +        ZooKeeper zk = new ZooKeeper("127.0.0.1:" + CLIENT_PORT,
    +                ClientBase.CONNECTION_TIMEOUT, this);
    +
    +        zk.create("/foo1", "foobar".getBytes(), Ids.OPEN_ACL_UNSAFE,
    +                CreateMode.PERSISTENT);
    +
    +        Assert.assertEquals(new String(zk.getData("/foo1", null, null)), "foobar");
    +        // It takes a while for the counter to get updated sometimes, this is added to reduce flakyness
    +        Thread.sleep(1000);
    --- End diff --
    
    I didn't see it consistently. I ran the same test about 60-70 times individually and couple of times as a suite with all other tests. It happened only once for me that the final assertion failed because the registered count was 2 instead of 3, which was when I was running it with other tests from the same class. 
    What do you suggest? Should I reduce the time-out or remove it all together?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---