Details

    • Type: Improvement
    • Status: Patch Available
    • Priority: Major
    • Resolution: Unresolved
    • Affects Version/s: None
    • Fix Version/s: None
    • Component/s: None
    • Labels:
      None

      Description

      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.

      Log the client and request details if the server discovers, when finally processing the request, that the current time minus arrival time of the request is beyond a configured threshold.

      Look at the HBase responseTooSlow feature for inspiration.

      1. ZOOKEEPER-2770.003.patch
        6 kB
        Karan Mehta
      2. ZOOKEEPER-2770.002.patch
        6 kB
        Karan Mehta
      3. ZOOKEEPER-2770.001.patch
        6 kB
        Karan Mehta

        Issue Links

          Activity

          Hide
          karanmehta93 Karan Mehta added a comment -

          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.

          Show
          karanmehta93 Karan Mehta added a comment - 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.
          Hide
          karanmehta93 Karan Mehta added a comment -

          Ping Ted Dunning Patrick Hunt Mahadev konar
          If you can review or suggest somebody who can review the patch. Thanks!

          Show
          karanmehta93 Karan Mehta added a comment - Ping Ted Dunning Patrick Hunt Mahadev konar If you can review or suggest somebody who can review the patch. Thanks!
          Hide
          tdunning Ted Dunning added a comment -

          I don't see any discussion of this on the mailing list. Also, the patch was posted to this bug 2 hours after it was filed.

          Is the problem you are trying to solve being discussed somewhere off list?

          Show
          tdunning Ted Dunning added a comment - I don't see any discussion of this on the mailing list. Also, the patch was posted to this bug 2 hours after it was filed. Is the problem you are trying to solve being discussed somewhere off list?
          Hide
          tdunning Ted Dunning added a comment -

          Would it be a good idea to extend the idea of this patch to include more generalized latency monitoring?

          Show
          tdunning Ted Dunning added a comment - Would it be a good idea to extend the idea of this patch to include more generalized latency monitoring?
          Hide
          karanmehta93 Karan Mehta added a comment -

          Ted Dunning
          We felt the need of adding this monitoring based on issues seen in production environments. It was discussed offline.
          I am ready to extend the patch, let me know what other places are a good fit for such monitoring.

          Show
          karanmehta93 Karan Mehta added a comment - Ted Dunning We felt the need of adding this monitoring based on issues seen in production environments. It was discussed offline. I am ready to extend the patch, let me know what other places are a good fit for such monitoring.
          Hide
          tdunning Ted Dunning added a comment -

          Who are we?

          This kind of feature needs to be discussed on the dev@zookeeper mailing list. I hate to be a prig about this, but one of the truisms at Apache is that if it didn't happen on the list, it didn't happen.

          Show
          tdunning Ted Dunning added a comment - Who are we? This kind of feature needs to be discussed on the dev@zookeeper mailing list. I hate to be a prig about this, but one of the truisms at Apache is that if it didn't happen on the list, it didn't happen.
          Hide
          apurtell Andrew Purtell added a comment -

          Ted Dunning It's pretty common in other projects (e.g. HBase, Hadoop, that constellation of projects...) to use JIRA to propose a change. When we (we = Trend Micro) worked on SASL authentication we went right to JIRA. I wouldn't have thought to propose the idea first on dev@. Now we (we = Salesforce, FWIW, Karan is a newcomer and a colleague) have this minor change we'd like to submit for your consideration. It's a small change, so small the code speaks for itself IMHO. If you think we must make a proposal on dev@ for this change to be considered, I'd suggest this is not the norm in my experience at Apache, but we'd be happy to do so.

          Show
          apurtell Andrew Purtell added a comment - Ted Dunning It's pretty common in other projects (e.g. HBase, Hadoop, that constellation of projects...) to use JIRA to propose a change. When we (we = Trend Micro) worked on SASL authentication we went right to JIRA. I wouldn't have thought to propose the idea first on dev@. Now we (we = Salesforce, FWIW, Karan is a newcomer and a colleague) have this minor change we'd like to submit for your consideration. It's a small change, so small the code speaks for itself IMHO. If you think we must make a proposal on dev@ for this change to be considered, I'd suggest this is not the norm in my experience at Apache, but we'd be happy to do so.
          Hide
          hanm Michael Han added a comment -

          Please creating a pull request (see https://cwiki.apache.org/confluence/display/ZOOKEEPER/HowToContribute) instead of attaching a patch.

          Show
          hanm Michael Han added a comment - Please creating a pull request (see https://cwiki.apache.org/confluence/display/ZOOKEEPER/HowToContribute ) instead of attaching a patch.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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 <karanmehta93@gmail.com>
          Date: 2017-05-02T21:20:39Z

          ZOOKEEPER-2770 ZooKeeper slow operation log


          Show
          githubbot ASF GitHub Bot added a comment - 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 <karanmehta93@gmail.com> Date: 2017-05-02T21:20:39Z ZOOKEEPER-2770 ZooKeeper slow operation log
          Hide
          hadoopqa Hadoop QA added a comment -

          -1 overall. GitHub Pull Request Build

          +1 @author. The patch does not contain any @author tags.

          +0 tests included. The patch appears to be a documentation patch that doesn't require tests.

          +1 javadoc. The javadoc tool did not generate any warning messages.

          +1 javac. The applied patch does not increase the total number of javac compiler warnings.

          -1 findbugs. The patch appears to introduce 1 new Findbugs (version 3.0.1) warnings.

          +1 release audit. The applied patch does not increase the total number of release audit warnings.

          +1 core tests. The patch passed core unit tests.

          +1 contrib tests. The patch passed contrib unit tests.

          Test results: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/876//testReport/
          Findbugs warnings: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/876//artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html
          Console output: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/876//console

          This message is automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - -1 overall. GitHub Pull Request Build +1 @author. The patch does not contain any @author tags. +0 tests included. The patch appears to be a documentation patch that doesn't require tests. +1 javadoc. The javadoc tool did not generate any warning messages. +1 javac. The applied patch does not increase the total number of javac compiler warnings. -1 findbugs. The patch appears to introduce 1 new Findbugs (version 3.0.1) warnings. +1 release audit. The applied patch does not increase the total number of release audit warnings. +1 core tests. The patch passed core unit tests. +1 contrib tests. The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/876//testReport/ Findbugs warnings: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/876//artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html Console output: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/876//console This message is automatically generated.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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.

          Show
          githubbot ASF GitHub Bot added a comment - 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.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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?

          Show
          githubbot ASF GitHub Bot added a comment - 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?
          Hide
          githubbot ASF GitHub Bot added a comment -

          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.

          Show
          githubbot ASF GitHub Bot added a comment - 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.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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.

          Show
          githubbot ASF GitHub Bot added a comment - 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.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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

          Show
          githubbot ASF GitHub Bot added a comment - 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
          Hide
          githubbot ASF GitHub Bot added a comment -

          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

          Show
          githubbot ASF GitHub Bot added a comment - 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
          Hide
          githubbot ASF GitHub Bot added a comment -

          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.

          Show
          githubbot ASF GitHub Bot added a comment - 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.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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.

          Show
          githubbot ASF GitHub Bot added a comment - 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.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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.

          Show
          githubbot ASF GitHub Bot added a comment - 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.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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) {
          ```

          Show
          githubbot ASF GitHub Bot added a comment - 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) { ```
          Hide
          githubbot ASF GitHub Bot added a comment -

          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.

          Show
          githubbot ASF GitHub Bot added a comment - 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.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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?

          Show
          githubbot ASF GitHub Bot added a comment - 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?
          Hide
          githubbot ASF GitHub Bot added a comment -

          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.

          Show
          githubbot ASF GitHub Bot added a comment - 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.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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()`?

          Show
          githubbot ASF GitHub Bot added a comment - 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()`?
          Hide
          githubbot ASF GitHub Bot added a comment -

          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

          Show
          githubbot ASF GitHub Bot added a comment - 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
          Hide
          githubbot ASF GitHub Bot added a comment -

          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?

          Show
          githubbot ASF GitHub Bot added a comment - 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?
          Hide
          githubbot ASF GitHub Bot added a comment -

          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.

          Show
          githubbot ASF GitHub Bot added a comment - 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.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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).

          Show
          githubbot ASF GitHub Bot added a comment - 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).
          Hide
          githubbot ASF GitHub Bot added a comment -

          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.

          Show
          githubbot ASF GitHub Bot added a comment - 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.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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.

          Show
          githubbot ASF GitHub Bot added a comment - 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.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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.

          Show
          githubbot ASF GitHub Bot added a comment - 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.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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?

          Show
          githubbot ASF GitHub Bot added a comment - 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?
          Hide
          githubbot ASF GitHub Bot added a comment -

          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!

          Show
          githubbot ASF GitHub Bot added a comment - 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!
          Hide
          githubbot ASF GitHub Bot added a comment -

          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).

          Show
          githubbot ASF GitHub Bot added a comment - 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).
          Hide
          githubbot ASF GitHub Bot added a comment -

          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

          Show
          githubbot ASF GitHub Bot added a comment - 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
          Hide
          githubbot ASF GitHub Bot added a comment -

          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.

          Show
          githubbot ASF GitHub Bot added a comment - 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.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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).

          Show
          githubbot ASF GitHub Bot added a comment - 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).
          Hide
          githubbot ASF GitHub Bot added a comment -

          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
            Unknown macro: { + + 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.

          Show
          githubbot ASF GitHub Bot added a comment - 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 Unknown macro: { + + 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.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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
            Unknown macro: { + + 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);`?

          Show
          githubbot ASF GitHub Bot added a comment - 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 Unknown macro: { + + 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);`?
          Hide
          githubbot ASF GitHub Bot added a comment -

          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
            Unknown macro: { + + 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?

          Show
          githubbot ASF GitHub Bot added a comment - 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 Unknown macro: { + + 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?
          Hide
          githubbot ASF GitHub Bot added a comment -

          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 `(`

          Show
          githubbot ASF GitHub Bot added a comment - 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 `(`
          Hide
          githubbot ASF GitHub Bot added a comment -

          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

          Show
          githubbot ASF GitHub Bot added a comment - 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
          Hide
          githubbot ASF GitHub Bot added a comment -

          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
            Unknown macro: { + + 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?

          Show
          githubbot ASF GitHub Bot added a comment - 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 Unknown macro: { + + 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?
          Hide
          githubbot ASF GitHub Bot added a comment -

          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
            Unknown macro: { + + 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.

          Show
          githubbot ASF GitHub Bot added a comment - 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 Unknown macro: { + + 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.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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
            Unknown macro: { + + 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.

          Show
          githubbot ASF GitHub Bot added a comment - 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 Unknown macro: { + + 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.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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
            Unknown macro: { + + 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?

          Show
          githubbot ASF GitHub Bot added a comment - 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 Unknown macro: { + + 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?
          Hide
          githubbot ASF GitHub Bot added a comment -

          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?

          Show
          githubbot ASF GitHub Bot added a comment - 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?
          Hide
          githubbot ASF GitHub Bot added a comment -

          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
            Unknown macro: { + + 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.

          Show
          githubbot ASF GitHub Bot added a comment - 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 Unknown macro: { + + 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.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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
            Unknown macro: { + + 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.

          Show
          githubbot ASF GitHub Bot added a comment - 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 Unknown macro: { + + 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.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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
            Unknown macro: { + + 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.

          Show
          githubbot ASF GitHub Bot added a comment - 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 Unknown macro: { + + 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.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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
            Unknown macro: { + + 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.

          Show
          githubbot ASF GitHub Bot added a comment - 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 Unknown macro: { + + 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.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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
            Unknown macro: { + + 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.

          Show
          githubbot ASF GitHub Bot added a comment - 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 Unknown macro: { + + 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.
          Hide
          hadoopqa Hadoop QA added a comment -

          -1 overall. GitHub Pull Request Build

          +1 @author. The patch does not contain any @author tags.

          +1 tests included. The patch appears to include 3 new or modified tests.

          +1 javadoc. The javadoc tool did not generate any warning messages.

          +1 javac. The applied patch does not increase the total number of javac compiler warnings.

          -1 findbugs. The patch appears to introduce 1 new Findbugs (version 3.0.1) warnings.

          +1 release audit. The applied patch does not increase the total number of release audit warnings.

          -1 core tests. The patch failed core unit tests.

          +1 contrib tests. The patch passed contrib unit tests.

          Test results: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/888//testReport/
          Findbugs warnings: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/888//artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html
          Console output: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/888//console

          This message is automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - -1 overall. GitHub Pull Request Build +1 @author. The patch does not contain any @author tags. +1 tests included. The patch appears to include 3 new or modified tests. +1 javadoc. The javadoc tool did not generate any warning messages. +1 javac. The applied patch does not increase the total number of javac compiler warnings. -1 findbugs. The patch appears to introduce 1 new Findbugs (version 3.0.1) warnings. +1 release audit. The applied patch does not increase the total number of release audit warnings. -1 core tests. The patch failed core unit tests. +1 contrib tests. The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/888//testReport/ Findbugs warnings: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/888//artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html Console output: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/888//console This message is automatically generated.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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).

          Show
          githubbot ASF GitHub Bot added a comment - 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).
          Hide
          githubbot ASF GitHub Bot added a comment -

          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.

          Show
          githubbot ASF GitHub Bot added a comment - 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.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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
            Unknown macro: { + + 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.

          Show
          githubbot ASF GitHub Bot added a comment - 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 Unknown macro: { + + 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.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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
            Unknown macro: { + + 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.

          Show
          githubbot ASF GitHub Bot added a comment - 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 Unknown macro: { + + 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.
          Hide
          hadoopqa Hadoop QA added a comment -

          -1 overall. GitHub Pull Request Build

          +1 @author. The patch does not contain any @author tags.

          +1 tests included. The patch appears to include 3 new or modified tests.

          +1 javadoc. The javadoc tool did not generate any warning messages.

          +1 javac. The applied patch does not increase the total number of javac compiler warnings.

          -1 findbugs. The patch appears to introduce 1 new Findbugs (version 3.0.1) warnings.

          +1 release audit. The applied patch does not increase the total number of release audit warnings.

          -1 core tests. The patch failed core unit tests.

          +1 contrib tests. The patch passed contrib unit tests.

          Test results: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/889//testReport/
          Findbugs warnings: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/889//artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html
          Console output: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/889//console

          This message is automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - -1 overall. GitHub Pull Request Build +1 @author. The patch does not contain any @author tags. +1 tests included. The patch appears to include 3 new or modified tests. +1 javadoc. The javadoc tool did not generate any warning messages. +1 javac. The applied patch does not increase the total number of javac compiler warnings. -1 findbugs. The patch appears to introduce 1 new Findbugs (version 3.0.1) warnings. +1 release audit. The applied patch does not increase the total number of release audit warnings. -1 core tests. The patch failed core unit tests. +1 contrib tests. The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/889//testReport/ Findbugs warnings: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/889//artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html Console output: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/889//console This message is automatically generated.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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.

          Show
          githubbot ASF GitHub Bot added a comment - 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.
          Hide
          hadoopqa Hadoop QA added a comment -

          -1 overall. GitHub Pull Request Build

          +1 @author. The patch does not contain any @author tags.

          +1 tests included. The patch appears to include 3 new or modified tests.

          +1 javadoc. The javadoc tool did not generate any warning messages.

          +1 javac. The applied patch does not increase the total number of javac compiler warnings.

          -1 findbugs. The patch appears to introduce 1 new Findbugs (version 3.0.1) warnings.

          +1 release audit. The applied patch does not increase the total number of release audit warnings.

          -1 core tests. The patch failed core unit tests.

          +1 contrib tests. The patch passed contrib unit tests.

          Test results: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/890//testReport/
          Findbugs warnings: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/890//artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html
          Console output: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/890//console

          This message is automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - -1 overall. GitHub Pull Request Build +1 @author. The patch does not contain any @author tags. +1 tests included. The patch appears to include 3 new or modified tests. +1 javadoc. The javadoc tool did not generate any warning messages. +1 javac. The applied patch does not increase the total number of javac compiler warnings. -1 findbugs. The patch appears to introduce 1 new Findbugs (version 3.0.1) warnings. +1 release audit. The applied patch does not increase the total number of release audit warnings. -1 core tests. The patch failed core unit tests. +1 contrib tests. The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/890//testReport/ Findbugs warnings: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/890//artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html Console output: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/890//console This message is automatically generated.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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.

          Show
          githubbot ASF GitHub Bot added a comment - 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.
          Hide
          hadoopqa Hadoop QA added a comment -

          -1 overall. GitHub Pull Request Build

          +1 @author. The patch does not contain any @author tags.

          +1 tests included. The patch appears to include 3 new or modified tests.

          +1 javadoc. The javadoc tool did not generate any warning messages.

          +1 javac. The applied patch does not increase the total number of javac compiler warnings.

          -1 findbugs. The patch appears to introduce 1 new Findbugs (version 3.0.1) warnings.

          +1 release audit. The applied patch does not increase the total number of release audit warnings.

          -1 core tests. The patch failed core unit tests.

          +1 contrib tests. The patch passed contrib unit tests.

          Test results: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/900//testReport/
          Findbugs warnings: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/900//artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html
          Console output: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/900//console

          This message is automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - -1 overall. GitHub Pull Request Build +1 @author. The patch does not contain any @author tags. +1 tests included. The patch appears to include 3 new or modified tests. +1 javadoc. The javadoc tool did not generate any warning messages. +1 javac. The applied patch does not increase the total number of javac compiler warnings. -1 findbugs. The patch appears to introduce 1 new Findbugs (version 3.0.1) warnings. +1 release audit. The applied patch does not increase the total number of release audit warnings. -1 core tests. The patch failed core unit tests. +1 contrib tests. The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/900//testReport/ Findbugs warnings: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/900//artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html Console output: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/900//console This message is automatically generated.
          Hide
          fournc Camille Fournier added a comment -

          I completely agree with Ted Dunning I don't understand the motivation for this. Are we just timing the internal processing time for the request? ZK is not the same type of system as HBase so I'm not sure we are comparing apples to oranges trying to cross-implement this feature.

          Show
          fournc Camille Fournier added a comment - I completely agree with Ted Dunning I don't understand the motivation for this. Are we just timing the internal processing time for the request? ZK is not the same type of system as HBase so I'm not sure we are comparing apples to oranges trying to cross-implement this feature.
          Hide
          tdunning Ted Dunning added a comment -

          Camille Fournier,

          I am not so sure that I agree with me at this point.

          It is fair to say that on occasion there are slow operations in ZK and it would be good to know about them.

          This kind of problem is almost always due, in my own vicarious experience, to bad configuration. Often the bad configuration is simply collocation with a noisy neighbor on a deficient storage layer. There might be situations where an operation is slow due to the content of the query itself, but I cannot imagine what those situations might be. Writing a large value (but that is strictly limited in size), or even doing a huge multi-op (which has the same limited size in aggregate) should never take very long.

          As such, I would expect that the highest diagnostic value would not be something that dumped the contents of slow queries, but rather a capability that characterizes the entire distribution of query times. The frequency of slow queries is a diagnostic of sorts, but is one that could be inferred from the time-varying distributional information I was suggesting.

          That said, I don't think that a slow query log is a BAD thing (except a bit bad in terms of security if it logs the actual query). And I wouldn't want the BEST thing (a distribution log) to stop somebody contributing something.

          Show
          tdunning Ted Dunning added a comment - Camille Fournier , I am not so sure that I agree with me at this point. It is fair to say that on occasion there are slow operations in ZK and it would be good to know about them. This kind of problem is almost always due, in my own vicarious experience, to bad configuration. Often the bad configuration is simply collocation with a noisy neighbor on a deficient storage layer. There might be situations where an operation is slow due to the content of the query itself, but I cannot imagine what those situations might be. Writing a large value (but that is strictly limited in size), or even doing a huge multi-op (which has the same limited size in aggregate) should never take very long. As such, I would expect that the highest diagnostic value would not be something that dumped the contents of slow queries, but rather a capability that characterizes the entire distribution of query times. The frequency of slow queries is a diagnostic of sorts, but is one that could be inferred from the time-varying distributional information I was suggesting. That said, I don't think that a slow query log is a BAD thing (except a bit bad in terms of security if it logs the actual query). And I wouldn't want the BEST thing (a distribution log) to stop somebody contributing something.
          Hide
          fournc Camille Fournier added a comment -

          Are there really 10s long slow requests? It's defaults like this that make me skeptical about the usefulness of this particular implementation. If we have a request through ZK that takes 10s to process your whole system is completely effed.

          I don't think we should add complexity to the code base without suitable justification for the value of the new feature. With that in mind, I'd like to understand what, specifically, the circumstances we're trying to measure are. It looks like processing time for a request through the ZK quorum alone, correct? The only network time that might be captured would be, in the case of a write, the quorum voting time.

          I'm all for making ZK more operable and exposing metrics but I don't think exposing low-value metrics is worth the additional code complexity without justification.

          Show
          fournc Camille Fournier added a comment - Are there really 10s long slow requests? It's defaults like this that make me skeptical about the usefulness of this particular implementation. If we have a request through ZK that takes 10s to process your whole system is completely effed. I don't think we should add complexity to the code base without suitable justification for the value of the new feature. With that in mind, I'd like to understand what, specifically, the circumstances we're trying to measure are. It looks like processing time for a request through the ZK quorum alone, correct? The only network time that might be captured would be, in the case of a write, the quorum voting time. I'm all for making ZK more operable and exposing metrics but I don't think exposing low-value metrics is worth the additional code complexity without justification.
          Hide
          apurtell Andrew Purtell added a comment - - edited

          The originally proposed change is hardly complex. I don't understand that aspect of this discussion. Whether or not the metric is useful, on the other hand... ok. That is a matter of opinion. I think we'd like to know if any ZK op takes longer than a second to complete, and how often that might happen, and on what host(s)/quorum it is happening. We have fleet of thousands of servers. We have tens of ZooKeeper installations, each on five servers. Hardware does funny things from time to time. We'd like to be proactive.

          Edit: More like 160 quorums, I think.

          Show
          apurtell Andrew Purtell added a comment - - edited The originally proposed change is hardly complex. I don't understand that aspect of this discussion. Whether or not the metric is useful, on the other hand... ok. That is a matter of opinion. I think we'd like to know if any ZK op takes longer than a second to complete, and how often that might happen, and on what host(s)/quorum it is happening. We have fleet of thousands of servers. We have tens of ZooKeeper installations, each on five servers. Hardware does funny things from time to time. We'd like to be proactive. Edit: More like 160 quorums, I think.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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?

          Show
          githubbot ASF GitHub Bot added a comment - 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?
          Hide
          githubbot ASF GitHub Bot added a comment -

          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.

          Show
          githubbot ASF GitHub Bot added a comment - 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.
          Hide
          hanm Michael Han added a comment -

          A hardcoded default value in code is unlikely to work for everyone and it is possible to have false negatives if the value is too small. I am leaning towards have this feature as an opt-in feature with the value has its default -1 only and for those who want use it they can tune the parameter for their deployment but it has to be enabled explicitly.

          Show
          hanm Michael Han added a comment - A hardcoded default value in code is unlikely to work for everyone and it is possible to have false negatives if the value is too small. I am leaning towards have this feature as an opt-in feature with the value has its default -1 only and for those who want use it they can tune the parameter for their deployment but it has to be enabled explicitly.
          Hide
          apurtell Andrew Purtell added a comment -

          From the original patch the warning threshold has been configurable. Calling it 'hardcoded' isn't correct. Maybe you meant a simple threshold only? That's true. It's better than nothing. FWIW I also like Ted's suggestion as a followup, and in fact would like to carry that over to HBase if it works out well here.

          Show
          apurtell Andrew Purtell added a comment - From the original patch the warning threshold has been configurable. Calling it 'hardcoded' isn't correct. Maybe you meant a simple threshold only? That's true. It's better than nothing. FWIW I also like Ted's suggestion as a followup, and in fact would like to carry that over to HBase if it works out well here.
          Hide
          hanm Michael Han added a comment -

          For hardcode I meant the default value of "requestWarnThresholdMs" baked in code.

          Show
          hanm Michael Han added a comment - For hardcode I meant the default value of "requestWarnThresholdMs" baked in code.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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.

          Show
          githubbot ASF GitHub Bot added a comment - 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.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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.

          Show
          githubbot ASF GitHub Bot added a comment - 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.
          Hide
          tdunning Ted Dunning added a comment -

          To put some color on Camille's surprise, I would consider any operation over a second to be indicative of gross failure in the quorum. Operations over 100ms should be vanishingly rare, but I wouldn't leap up to find out what is happening. I would be fairly unhappy, though, and would start checking.

          Show
          tdunning Ted Dunning added a comment - To put some color on Camille's surprise, I would consider any operation over a second to be indicative of gross failure in the quorum. Operations over 100ms should be vanishingly rare, but I wouldn't leap up to find out what is happening. I would be fairly unhappy, though, and would start checking.
          Hide
          tdunning Ted Dunning added a comment -

          On second thought, I could imagine that startup transients could cause a long operation. Once you have your quorum in a groove, however, >1 second is very bad, especially if you don't have something like a quorum leader change happening.

          Show
          tdunning Ted Dunning added a comment - On second thought, I could imagine that startup transients could cause a long operation. Once you have your quorum in a groove, however, >1 second is very bad, especially if you don't have something like a quorum leader change happening.
          Hide
          karanmehta93 Karan Mehta added a comment -

          Operations over 100ms should be vanishingly rare, but I wouldn't leap up to find out what is happening. I would be fairly unhappy, though, and would start checking.

          Let's take this as a motivation.
          With that said, is 300 ms a good value or even less is better?

          Show
          karanmehta93 Karan Mehta added a comment - Operations over 100ms should be vanishingly rare, but I wouldn't leap up to find out what is happening. I would be fairly unhappy, though, and would start checking. Let's take this as a motivation. With that said, is 300 ms a good value or even less is better?
          Hide
          tdunning Ted Dunning added a comment -

          With that said, is 300 ms a good value or even less is better?

          I would suggest that getting a real time varying histogram is the right answer. I suggested that early on for just this kind of reason.

          Show
          tdunning Ted Dunning added a comment - With that said, is 300 ms a good value or even less is better? I would suggest that getting a real time varying histogram is the right answer. I suggested that early on for just this kind of reason.
          Hide
          tdunning Ted Dunning added a comment -

          Btw.... I note that there is no metering on this logging.

          That raise an obligatory question. Is there a plausible circumstance where thousands of nearly identical messages might be logged?

          Show
          tdunning Ted Dunning added a comment - Btw.... I note that there is no metering on this logging. That raise an obligatory question. Is there a plausible circumstance where thousands of nearly identical messages might be logged?
          Hide
          karanmehta93 Karan Mehta added a comment -

          Ted Dunning
          There is no rate limiting on logging. If the threshold is too low, it might result in huge amount of messages getting printed. At this point, I suggest the following things. Please suggest your opinion.
          1. Turn off this feature by default, so that we don't end up on an arbitrary value.
          I personally not want this, since I believe that no matter what your requirements are or your hardware, it can be possible to put some upper bound on this value. Experienced people can comment on this more than I can.
          2. Add rate limiter based on some logic
          2.1 Time based logic (Limit messages printed in a given amount of time)
          2.2 Random sampling based on some probability percentage.

          I am not aware of how typically these things are implemented. It would be good if you can suggest some part of code which does similar stuff. Thanks!

          Show
          karanmehta93 Karan Mehta added a comment - Ted Dunning There is no rate limiting on logging. If the threshold is too low, it might result in huge amount of messages getting printed. At this point, I suggest the following things. Please suggest your opinion. 1. Turn off this feature by default, so that we don't end up on an arbitrary value. I personally not want this, since I believe that no matter what your requirements are or your hardware, it can be possible to put some upper bound on this value. Experienced people can comment on this more than I can. 2. Add rate limiter based on some logic 2.1 Time based logic (Limit messages printed in a given amount of time) 2.2 Random sampling based on some probability percentage. I am not aware of how typically these things are implemented. It would be good if you can suggest some part of code which does similar stuff. Thanks!
          Hide
          tdunning Ted Dunning added a comment -

          The typical approach is to set a limit on number of messages per unit time
          (say one every 10 minutes). Each message that is printed sets a coalescence
          time during which no further messages are printed, but a counter is
          updated. At the end of the coalescence time a modified message which
          mentions that n additional events were detected and the coalescence time is
          disabled.

          This way if the warnings are rare, you get normal behavior. If the warnings
          are frequent, you get at most one message per 10 minutes (or whatever
          coalescence period you choose). You get instant notification of a problem
          and limited log output.

          On Wed, Jul 26, 2017 at 10:05 PM, Karan Mehta (JIRA) <jira@apache.org>

          Show
          tdunning Ted Dunning added a comment - The typical approach is to set a limit on number of messages per unit time (say one every 10 minutes). Each message that is printed sets a coalescence time during which no further messages are printed, but a counter is updated. At the end of the coalescence time a modified message which mentions that n additional events were detected and the coalescence time is disabled. This way if the warnings are rare, you get normal behavior. If the warnings are frequent, you get at most one message per 10 minutes (or whatever coalescence period you choose). You get instant notification of a problem and limited log output. On Wed, Jul 26, 2017 at 10:05 PM, Karan Mehta (JIRA) <jira@apache.org>
          Hide
          githubbot ASF GitHub Bot added a comment -

          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.

          Show
          githubbot ASF GitHub Bot added a comment - 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.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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!

          Show
          githubbot ASF GitHub Bot added a comment - 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!
          Hide
          githubbot ASF GitHub Bot added a comment -

          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?

          Show
          githubbot ASF GitHub Bot added a comment - 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?
          Hide
          hadoopqa Hadoop QA added a comment -

          -1 overall. GitHub Pull Request Build

          +1 @author. The patch does not contain any @author tags.

          +1 tests included. The patch appears to include 5 new or modified tests.

          +1 javadoc. The javadoc tool did not generate any warning messages.

          +1 javac. The applied patch does not increase the total number of javac compiler warnings.

          -1 findbugs. The patch appears to introduce 3 new Findbugs (version 3.0.1) warnings.

          -1 release audit. The applied patch generated 1 release audit warnings (more than the trunk's current 0 warnings).

          -1 core tests. The patch failed core unit tests.

          +1 contrib tests. The patch passed contrib unit tests.

          Test results: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/944//testReport/
          Release audit warnings: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/944//artifact/trunk/patchprocess/patchReleaseAuditProblems.txt
          Findbugs warnings: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/944//artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html
          Console output: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/944//console

          This message is automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - -1 overall. GitHub Pull Request Build +1 @author. The patch does not contain any @author tags. +1 tests included. The patch appears to include 5 new or modified tests. +1 javadoc. The javadoc tool did not generate any warning messages. +1 javac. The applied patch does not increase the total number of javac compiler warnings. -1 findbugs. The patch appears to introduce 3 new Findbugs (version 3.0.1) warnings. -1 release audit. The applied patch generated 1 release audit warnings (more than the trunk's current 0 warnings). -1 core tests. The patch failed core unit tests. +1 contrib tests. The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/944//testReport/ Release audit warnings: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/944//artifact/trunk/patchprocess/patchReleaseAuditProblems.txt Findbugs warnings: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/944//artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html Console output: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/944//console This message is automatically generated.
          Hide
          hadoopqa Hadoop QA added a comment -

          -1 overall. GitHub Pull Request Build

          +1 @author. The patch does not contain any @author tags.

          +1 tests included. The patch appears to include 8 new or modified tests.

          +1 javadoc. The javadoc tool did not generate any warning messages.

          +1 javac. The applied patch does not increase the total number of javac compiler warnings.

          -1 findbugs. The patch appears to introduce 3 new Findbugs (version 3.0.1) warnings.

          -1 release audit. The applied patch generated 1 release audit warnings (more than the trunk's current 0 warnings).

          +1 core tests. The patch passed core unit tests.

          +1 contrib tests. The patch passed contrib unit tests.

          Test results: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/945//testReport/
          Release audit warnings: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/945//artifact/trunk/patchprocess/patchReleaseAuditProblems.txt
          Findbugs warnings: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/945//artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html
          Console output: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/945//console

          This message is automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - -1 overall. GitHub Pull Request Build +1 @author. The patch does not contain any @author tags. +1 tests included. The patch appears to include 8 new or modified tests. +1 javadoc. The javadoc tool did not generate any warning messages. +1 javac. The applied patch does not increase the total number of javac compiler warnings. -1 findbugs. The patch appears to introduce 3 new Findbugs (version 3.0.1) warnings. -1 release audit. The applied patch generated 1 release audit warnings (more than the trunk's current 0 warnings). +1 core tests. The patch passed core unit tests. +1 contrib tests. The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/945//testReport/ Release audit warnings: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/945//artifact/trunk/patchprocess/patchReleaseAuditProblems.txt Findbugs warnings: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/945//artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html Console output: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/945//console This message is automatically generated.
          Hide
          hadoopqa Hadoop QA added a comment -

          -1 overall. GitHub Pull Request Build

          +1 @author. The patch does not contain any @author tags.

          +1 tests included. The patch appears to include 8 new or modified tests.

          +1 javadoc. The javadoc tool did not generate any warning messages.

          +1 javac. The applied patch does not increase the total number of javac compiler warnings.

          -1 findbugs. The patch appears to introduce 3 new Findbugs (version 3.0.1) warnings.

          -1 release audit. The applied patch generated 1 release audit warnings (more than the trunk's current 0 warnings).

          -1 core tests. The patch failed core unit tests.

          +1 contrib tests. The patch passed contrib unit tests.

          Test results: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/946//testReport/
          Release audit warnings: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/946//artifact/trunk/patchprocess/patchReleaseAuditProblems.txt
          Findbugs warnings: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/946//artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html
          Console output: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/946//console

          This message is automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - -1 overall. GitHub Pull Request Build +1 @author. The patch does not contain any @author tags. +1 tests included. The patch appears to include 8 new or modified tests. +1 javadoc. The javadoc tool did not generate any warning messages. +1 javac. The applied patch does not increase the total number of javac compiler warnings. -1 findbugs. The patch appears to introduce 3 new Findbugs (version 3.0.1) warnings. -1 release audit. The applied patch generated 1 release audit warnings (more than the trunk's current 0 warnings). -1 core tests. The patch failed core unit tests. +1 contrib tests. The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/946//testReport/ Release audit warnings: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/946//artifact/trunk/patchprocess/patchReleaseAuditProblems.txt Findbugs warnings: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/946//artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html Console output: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/946//console This message is automatically generated.
          Hide
          hadoopqa Hadoop QA added a comment -

          -1 overall. GitHub Pull Request Build

          +1 @author. The patch does not contain any @author tags.

          +1 tests included. The patch appears to include 8 new or modified tests.

          +1 javadoc. The javadoc tool did not generate any warning messages.

          +1 javac. The applied patch does not increase the total number of javac compiler warnings.

          -1 findbugs. The patch appears to introduce 3 new Findbugs (version 3.0.1) warnings.

          -1 release audit. The applied patch generated 1 release audit warnings (more than the trunk's current 0 warnings).

          -1 core tests. The patch failed core unit tests.

          +1 contrib tests. The patch passed contrib unit tests.

          Test results: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/947//testReport/
          Release audit warnings: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/947//artifact/trunk/patchprocess/patchReleaseAuditProblems.txt
          Findbugs warnings: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/947//artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html
          Console output: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/947//console

          This message is automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - -1 overall. GitHub Pull Request Build +1 @author. The patch does not contain any @author tags. +1 tests included. The patch appears to include 8 new or modified tests. +1 javadoc. The javadoc tool did not generate any warning messages. +1 javac. The applied patch does not increase the total number of javac compiler warnings. -1 findbugs. The patch appears to introduce 3 new Findbugs (version 3.0.1) warnings. -1 release audit. The applied patch generated 1 release audit warnings (more than the trunk's current 0 warnings). -1 core tests. The patch failed core unit tests. +1 contrib tests. The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/947//testReport/ Release audit warnings: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/947//artifact/trunk/patchprocess/patchReleaseAuditProblems.txt Findbugs warnings: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/947//artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html Console output: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/947//console This message is automatically generated.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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.

          Show
          githubbot ASF GitHub Bot added a comment - 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.
          Hide
          hadoopqa Hadoop QA added a comment -

          -1 overall. GitHub Pull Request Build

          +1 @author. The patch does not contain any @author tags.

          +1 tests included. The patch appears to include 8 new or modified tests.

          +1 javadoc. The javadoc tool did not generate any warning messages.

          +1 javac. The applied patch does not increase the total number of javac compiler warnings.

          -1 findbugs. The patch appears to introduce 3 new Findbugs (version 3.0.1) warnings.

          -1 release audit. The applied patch generated 1 release audit warnings (more than the trunk's current 0 warnings).

          -1 core tests. The patch failed core unit tests.

          +1 contrib tests. The patch passed contrib unit tests.

          Test results: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/1032//testReport/
          Release audit warnings: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/1032//artifact/trunk/patchprocess/patchReleaseAuditProblems.txt
          Findbugs warnings: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/1032//artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html
          Console output: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/1032//console

          This message is automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - -1 overall. GitHub Pull Request Build +1 @author. The patch does not contain any @author tags. +1 tests included. The patch appears to include 8 new or modified tests. +1 javadoc. The javadoc tool did not generate any warning messages. +1 javac. The applied patch does not increase the total number of javac compiler warnings. -1 findbugs. The patch appears to introduce 3 new Findbugs (version 3.0.1) warnings. -1 release audit. The applied patch generated 1 release audit warnings (more than the trunk's current 0 warnings). -1 core tests. The patch failed core unit tests. +1 contrib tests. The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/1032//testReport/ Release audit warnings: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/1032//artifact/trunk/patchprocess/patchReleaseAuditProblems.txt Findbugs warnings: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/1032//artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html Console output: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/1032//console This message is automatically generated.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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`?

          Show
          githubbot ASF GitHub Bot added a comment - 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`?
          Hide
          githubbot ASF GitHub Bot added a comment -

          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?

          Show
          githubbot ASF GitHub Bot added a comment - 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?
          Hide
          githubbot ASF GitHub Bot added a comment -

          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 `(`

          Show
          githubbot ASF GitHub Bot added a comment - 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 `(`

            People

            • Assignee:
              karanmehta93 Karan Mehta
              Reporter:
              karanmehta93 Karan Mehta
            • Votes:
              0 Vote for this issue
              Watchers:
              8 Start watching this issue

              Dates

              • Created:
                Updated:

                Development