You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@zookeeper.apache.org by GitBox <gi...@apache.org> on 2020/01/08 22:52:54 UTC

[GitHub] [zookeeper] jhuan31 opened a new pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…

jhuan31 opened a new pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…
URL: https://github.com/apache/zookeeper/pull/1211
 
 
   …gured threshold

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] hanm commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…

Posted by GitBox <gi...@apache.org>.
hanm commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…
URL: https://github.com/apache/zookeeper/pull/1211#discussion_r389965773
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/KeeperException.java
 ##########
 @@ -404,7 +406,9 @@ public void setCode(int code) {
         /** The session has been closed by server because server requires client to do SASL authentication,
          *  but client is not configured with SASL authentication or configuted with SASL but failed
          *  (i.e. wrong credential used.). */
-        SESSIONCLOSEDREQUIRESASLAUTH(-124);
+        SESSIONCLOSEDREQUIRESASLAUTH(-124),
+        /** Operation was throttled and not executed at all. please, retry! */
 
 Review comment:
   It might be better to add more document here on how to deal with this new error code. `please, retry!` does not sound a good answer, as our intention, according to the design is to not to let clients retry the requests immediately. Something like - "this error code indicates that zookeeper server is under heavy load and can't process incoming requests at full speed; please retry with back off" sounds more useful to user.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] lvfangmin commented on issue #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…

Posted by GitBox <gi...@apache.org>.
lvfangmin commented on issue #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…
URL: https://github.com/apache/zookeeper/pull/1211#issuecomment-596630615
 
 
   This feature is not fully enabled on our prod environment, I think it's fine to go for 3.7 given we already have a lot of reliability and cool features added in 3.6.
   
   Any other feedback of this diff, or we can go ahead and merge it?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] ztzg commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…

Posted by GitBox <gi...@apache.org>.
ztzg commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…
URL: https://github.com/apache/zookeeper/pull/1211#discussion_r364612931
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/CommitProcessor.java
 ##########
 @@ -306,6 +309,11 @@ public void run() {
                         // Process committed head
                         request = committedRequests.peek();
 
+                        if (request.isThrottled()) {
+                            LOG.error("Throttled request in committed pool: " + request + ". Exiting.");
+                            System.exit(1);
 
 Review comment:
   See `ServiceUtils.requestSystemExit()` and [ZOOKEEPER-3620](https://issues.apache.org/jira/browse/ZOOKEEPER-3620).  (There are multiple instances of this.)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] anmolnar commented on issue #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…

Posted by GitBox <gi...@apache.org>.
anmolnar commented on issue #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…
URL: https://github.com/apache/zookeeper/pull/1211#issuecomment-586319580
 
 
   @eolivelli I understand just saying we had some committments and I feel we started releasing 3.6 a little bit too early. Though it's also my fault: I haven't shout against cutting 3.6

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] jhuan31 commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…

Posted by GitBox <gi...@apache.org>.
jhuan31 commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…
URL: https://github.com/apache/zookeeper/pull/1211#discussion_r364884064
 
 

 ##########
 File path: zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md
 ##########
 @@ -1133,6 +1133,15 @@ property, when available, is noted below.
     effect due to TLS handshake timeout when there are too many in-flight TLS 
     handshakes. Set it to something like 250 is good enough to avoid herd effect.
 
+* *throttledOpWaitTime*
+    (Jave system property: **zookeeper.throttled_op_wait_time**)
+    The time in the RequestThrottler queue longer than which a request will be marked as throttled.
 
 Review comment:
   fixed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] eolivelli commented on issue #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…

Posted by GitBox <gi...@apache.org>.
eolivelli commented on issue #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…
URL: https://github.com/apache/zookeeper/pull/1211#issuecomment-586257189
 
 
   branch-3.6 is almost stable and it should be put into only bugfix-and-minor-improvements-mode as soon as 3.6.0 is out.
   
   This patch is very big and It is also adding new RCs (ThrottledOpException)
   
   I would go for 3.7.
   
   We already got into troubles due to the multi address features committed right before the release :-) 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] hanm commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…

Posted by GitBox <gi...@apache.org>.
hanm commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…
URL: https://github.com/apache/zookeeper/pull/1211#discussion_r389980200
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/Request.java
 ##########
 @@ -100,6 +100,8 @@ public Request(long sessionId, int xid, int type, TxnHeader hdr, Record txn, lon
 
     public long syncQueueStartTime;
 
+    public long requestThrottleQueueTime;
 
 Review comment:
   I might miss something here, but I don't see where this value is actually get set anywhere in this patch. Without this value being set, I don't think we'll know when a request will be throttled or not ...

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] hanm commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…

Posted by GitBox <gi...@apache.org>.
hanm commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…
URL: https://github.com/apache/zookeeper/pull/1211#discussion_r389963681
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/FinalRequestProcessor.java
 ##########
 @@ -150,6 +139,24 @@ public void processRequest(Request request) {
             }
         }
 
+        return rc;
+    }
+
+    public void processRequest(Request request) {
+        LOG.debug("Processing request:: " + request);
 
 Review comment:
   Please use parameterized logging format here - I believe that in its current form the string concatenation will happen as the parameter has to be evaluated first, even when debug level logging is disabled. Since this is on critical path of request processing, the performance will take a hit.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] ztzg commented on issue #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…

Posted by GitBox <gi...@apache.org>.
ztzg commented on issue #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…
URL: https://github.com/apache/zookeeper/pull/1211#issuecomment-604577534
 
 
   @hanm, re `requestThrottleQueueTime`: Nice catch!

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] hanm commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…

Posted by GitBox <gi...@apache.org>.
hanm commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…
URL: https://github.com/apache/zookeeper/pull/1211#discussion_r389962575
 
 

 ##########
 File path: zookeeper-client/zookeeper-client-c/include/zookeeper.h
 ##########
 @@ -136,7 +136,8 @@ enum ZOO_ERRORS {
   ZEPHEMERALONLOCALSESSION = -120, /*!< Attempt to create ephemeral node on a local session */
   ZNOWATCHER = -121, /*!< The watcher couldn't be found */
   ZRECONFIGDISABLED = -123, /*!< Attempts to perform a reconfiguration operation when reconfiguration feature is disabled */
-  ZSESSIONCLOSEDREQUIRESASLAUTH = -124 /*!< The session has been closed by server because server requires client to do SASL authentication, but client is not configured with SASL authentication or configuted with SASL but failed (i.e. wrong credential used.). */
+  ZSESSIONCLOSEDREQUIRESASLAUTH = -124, /*!< The session has been closed by server because server requires client to do SASL authentication, but client is not configured with SASL authentication or configuted with SASL but failed (i.e. wrong credential used.). */
+  ZTHROTTLEDOP = -127 /*!< Operation was throttled and not executed at all. please, retry! */
 
 Review comment:
   Any reason not to use -125 as the value of `ZTHROTTLEDOP` here?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] jhuan31 commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…

Posted by GitBox <gi...@apache.org>.
jhuan31 commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…
URL: https://github.com/apache/zookeeper/pull/1211#discussion_r397387285
 
 

 ##########
 File path: zookeeper-client/zookeeper-client-c/include/zookeeper.h
 ##########
 @@ -136,7 +136,8 @@ enum ZOO_ERRORS {
   ZEPHEMERALONLOCALSESSION = -120, /*!< Attempt to create ephemeral node on a local session */
   ZNOWATCHER = -121, /*!< The watcher couldn't be found */
   ZRECONFIGDISABLED = -123, /*!< Attempts to perform a reconfiguration operation when reconfiguration feature is disabled */
-  ZSESSIONCLOSEDREQUIRESASLAUTH = -124 /*!< The session has been closed by server because server requires client to do SASL authentication, but client is not configured with SASL authentication or configuted with SASL but failed (i.e. wrong credential used.). */
+  ZSESSIONCLOSEDREQUIRESASLAUTH = -124, /*!< The session has been closed by server because server requires client to do SASL authentication, but client is not configured with SASL authentication or configuted with SASL but failed (i.e. wrong credential used.). */
+  ZTHROTTLEDOP = -127 /*!< Operation was throttled and not executed at all. please, retry! */
 
 Review comment:
   We usually upstream a feature after it is deployed and if the number is taken when we upstream it, it's quite some trouble especially the number is used on the client side too. So we usually leave some holes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] ztzg commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…

Posted by GitBox <gi...@apache.org>.
ztzg commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…
URL: https://github.com/apache/zookeeper/pull/1211#discussion_r364615039
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java
 ##########
 @@ -65,7 +65,6 @@
  * message (including heartbeats) from the leader.</li>
  * <li>server.<i>id</i> - This is the host:port[:port] that the server with the
  * given id will use for the quorum protocol.</li>
- * </ol>
  * In addition to the config file. There is a file in the data directory called
 
 Review comment:
   This looks like an accidental deletion, and breaks the `<ol>` list.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] ztzg commented on issue #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…

Posted by GitBox <gi...@apache.org>.
ztzg commented on issue #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…
URL: https://github.com/apache/zookeeper/pull/1211#issuecomment-573604866
 
 
   Hi @jhuan31,
   
   I am very interested in "multitenancy" topics such as client identification, throttling and quotas, and notably in the components of this ticket submitted by @gmcatsf:
   
   [ZOOKEEPER-3467, Complete quota system for zookeeper server](https://issues.apache.org/jira/browse/ZOOKEEPER-3467)
   
   It seems there is a constant stream of patches coming from Facebook which progressively add missing bits and pieces. This is great, but it also means that any new work in that area is likely to duplicate something you already have working internally.
   
   So I am wondering: would you, per chance, have a rough list of "features" you are planning to upstream, so that we can coordinate and minimize conflicts/duplication?
   
   Thanks, -D

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] jhuan31 commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…

Posted by GitBox <gi...@apache.org>.
jhuan31 commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…
URL: https://github.com/apache/zookeeper/pull/1211#discussion_r364912973
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/SyncRequestProcessor.java
 ##########
 @@ -178,7 +178,7 @@ public void run() {
                 ServerMetrics.getMetrics().SYNC_PROCESSOR_QUEUE_TIME.add(startProcessTime - si.syncQueueStartTime);
 
                 // track the number of records written to the log
-                if (zks.getZKDatabase().append(si)) {
+                if (!si.isThrottled() && zks.getZKDatabase().append(si)) {
 
 Review comment:
   updated the comment

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] hanm commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…

Posted by GitBox <gi...@apache.org>.
hanm commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…
URL: https://github.com/apache/zookeeper/pull/1211#discussion_r397576620
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/Request.java
 ##########
 @@ -100,6 +100,8 @@ public Request(long sessionId, int xid, int type, TxnHeader hdr, Record txn, lon
 
     public long syncQueueStartTime;
 
+    public long requestThrottleQueueTime;
 
 Review comment:
   :) 
   
   Would be good to add a test case that cover the real logic of `shouldThrottleOp` instead of using a mock; if we have such test it will catch issue like this but this is non blocking and latest change lgtm.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] jhuan31 commented on issue #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…

Posted by GitBox <gi...@apache.org>.
jhuan31 commented on issue #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…
URL: https://github.com/apache/zookeeper/pull/1211#issuecomment-603527423
 
 
   @hanm Thank you very much for your comments. I've addressed them. Sorry about the delay.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] jhuan31 commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…

Posted by GitBox <gi...@apache.org>.
jhuan31 commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…
URL: https://github.com/apache/zookeeper/pull/1211#discussion_r364923150
 
 

 ##########
 File path: zookeeper-server/src/test/java/org/apache/zookeeper/test/ThrottledOpHelper.java
 ##########
 @@ -0,0 +1,224 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import mockit.Mock;
+import mockit.MockUp;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.server.Request;
+import org.apache.zookeeper.server.RequestThrottler;
+import org.apache.zookeeper.server.ZooKeeperServer;
+import org.junit.Assert;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ThrottledOpHelper {
+    protected static final Logger LOG = LoggerFactory.getLogger(ThrottledOpHelper.class);
+
+    public static final class RequestThrottleMock extends MockUp<RequestThrottler> {
+        public static void throttleEveryNthOp(int n) {
+            everyNthOp = n;
+            opCounter = 0;
+        }
+        private static int everyNthOp = 0;
+        private static int opCounter = 0;
+
+        @Mock
+        private boolean shouldThrottleOp(Request request, long elapsedTime) {
+            if (everyNthOp > 0 && request.isThrottlable() && (++opCounter % everyNthOp == 0)) {
+                opCounter %= everyNthOp;
+                return true;
+            }
+            return false;
+        }
+    }
+
+    public static void applyMockUps() {
+        new RequestThrottleMock();
+    }
+
+    public void testThrottledOp(ZooKeeper zk, ZooKeeperServer zs) throws IOException, InterruptedException, KeeperException {
+        final int N = 5; // must be greater than 3
+        final int COUNT = 100;
+        RequestThrottleMock.throttleEveryNthOp(N);
+        LOG.info("Before create /ivailo nodes");
+        int opCount = 0;
+        for (int i = 0; i < COUNT; i++) {
+            String nodeName = "/ivailo" + i;
+            if (opCount % N == N - 1) {
+                try {
+                    zk.create(nodeName, "".getBytes(), Ids.OPEN_ACL_UNSAFE,
+                        (i % 2 == 0) ? CreateMode.PERSISTENT : CreateMode.EPHEMERAL);
+                    Assert.fail("Should have gotten ThrottledOp exception");
+                } catch (KeeperException.ThrottledOpException e) {
+                    // anticipated outcome
+                    Stat stat = zk.exists(nodeName, null);
+                    Assert.assertNull(stat);
+                    zk.create(nodeName, "".getBytes(), Ids.OPEN_ACL_UNSAFE,
+                        (i % 2 == 0) ? CreateMode.PERSISTENT : CreateMode.EPHEMERAL);
+                } catch (KeeperException e) {
+                    Assert.fail("Should have gotten ThrottledOp exception");
+                }
+                opCount += 3; // three ops issues
+            } else {
+                zk.create(nodeName, "".getBytes(), Ids.OPEN_ACL_UNSAFE,
+                    (i % 2 == 0) ? CreateMode.PERSISTENT : CreateMode.EPHEMERAL);
+                opCount++; // one op only issued
+            }
+            if (opCount % N == N - 1) {
+                try {
+                    zk.setData(nodeName, nodeName.getBytes(), -1);
+                    Assert.fail("Should have gotten ThrottledOp exception");
+                } catch (KeeperException.ThrottledOpException e) {
+                    // anticipated outcome & retry
+                    zk.setData(nodeName, nodeName.getBytes(), -1);
+                } catch (KeeperException e) {
+                    Assert.fail("Should have gotten ThrottledOp exception");
+                }
+                opCount += 2; // two ops issues, one for retry
+            } else {
+                zk.setData(nodeName, nodeName.getBytes(), -1);
+                opCount++; // one op only issued
+            }
+        }
+        LOG.info("Before delete /ivailo nodes");
+        for (int i = 0; i < COUNT; i++) {
+            String nodeName = "/ivailo" + i;
+            if (opCount % N == N - 1) {
+                try {
+                    zk.exists(nodeName, null);
+                    Assert.fail("Should have gotten ThrottledOp exception");
+                } catch (KeeperException.ThrottledOpException e) {
+                    // anticipated outcome & retry
+                    Stat stat = zk.exists(nodeName, null);
+                    Assert.assertNotNull(stat);
+                    opCount += 2; // two ops issued, one is retry
+                } catch (KeeperException e) {
+                    Assert.fail("Should have gotten ThrottledOp exception");
+                }
+            } else {
+                Stat stat = zk.exists(nodeName, null);
+                Assert.assertNotNull(stat);
+                opCount++;
+            }
+            if (opCount % N == N - 1) {
+                try {
+                    zk.getData(nodeName, null, null);
+                    Assert.fail("Should have gotten ThrottledOp exception");
+                } catch (KeeperException.ThrottledOpException e) {
+                    // anticipated outcome & retry
+                    byte[] data = zk.getData(nodeName, null, null);
+                    Assert.assertEquals(nodeName, new String(data));
+                    opCount += 2; // two ops issued, one is retry
+                } catch (KeeperException e) {
+                    Assert.fail("Should have gotten ThrottledOp exception");
+                }
+            } else {
+                byte[] data = zk.getData(nodeName, null, null);
+                Assert.assertEquals(nodeName, new String(data));
+                opCount++;
+            }
+            if (opCount % N == N - 1) {
+                try {
+                    // version 0 should not trigger BadVersion exception
+                    zk.delete(nodeName, 0);
 
 Review comment:
   yes. we just want to make sure that no BadVersion exception is thrown.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] jhuan31 commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…

Posted by GitBox <gi...@apache.org>.
jhuan31 commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…
URL: https://github.com/apache/zookeeper/pull/1211#discussion_r397389302
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/FinalRequestProcessor.java
 ##########
 @@ -150,6 +139,24 @@ public void processRequest(Request request) {
             }
         }
 
+        return rc;
+    }
+
+    public void processRequest(Request request) {
+        LOG.debug("Processing request:: " + request);
 
 Review comment:
   done.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] jhuan31 commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…

Posted by GitBox <gi...@apache.org>.
jhuan31 commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…
URL: https://github.com/apache/zookeeper/pull/1211#discussion_r397390969
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/KeeperException.java
 ##########
 @@ -404,7 +406,9 @@ public void setCode(int code) {
         /** The session has been closed by server because server requires client to do SASL authentication,
          *  but client is not configured with SASL authentication or configuted with SASL but failed
          *  (i.e. wrong credential used.). */
-        SESSIONCLOSEDREQUIRESASLAUTH(-124);
+        SESSIONCLOSEDREQUIRESASLAUTH(-124),
+        /** Operation was throttled and not executed at all. please, retry! */
 
 Review comment:
   Thanks!

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] eolivelli commented on issue #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…

Posted by GitBox <gi...@apache.org>.
eolivelli commented on issue #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…
URL: https://github.com/apache/zookeeper/pull/1211#issuecomment-573606271
 
 
   @ztzg yes, ZookKeeper Friends @ Facebook are porting most of their internal changes to Apache ZooKeeper.
   They are creating JIRAs.
   
   If you check on the ML they are hosting a meetup this month in order to present their work. They did the same some month ago.
   
   It is a great chance for the community to benefit from their experience as they are running ZooKeeper at scale.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] jhuan31 commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…

Posted by GitBox <gi...@apache.org>.
jhuan31 commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…
URL: https://github.com/apache/zookeeper/pull/1211#discussion_r364885314
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java
 ##########
 @@ -65,7 +65,6 @@
  * message (including heartbeats) from the leader.</li>
  * <li>server.<i>id</i> - This is the host:port[:port] that the server with the
  * given id will use for the quorum protocol.</li>
- * </ol>
  * In addition to the config file. There is a file in the data directory called
 
 Review comment:
   oh yea. fixed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] jhuan31 commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…

Posted by GitBox <gi...@apache.org>.
jhuan31 commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…
URL: https://github.com/apache/zookeeper/pull/1211#discussion_r364896464
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/SyncRequestProcessor.java
 ##########
 @@ -178,7 +178,7 @@ public void run() {
                 ServerMetrics.getMetrics().SYNC_PROCESSOR_QUEUE_TIME.add(startProcessTime - si.syncQueueStartTime);
 
                 // track the number of records written to the log
-                if (zks.getZKDatabase().append(si)) {
+                if (!si.isThrottled() && zks.getZKDatabase().append(si)) {
 
 Review comment:
   Thanks for raising the red flag. Now I see it too!

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] jhuan31 commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…

Posted by GitBox <gi...@apache.org>.
jhuan31 commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…
URL: https://github.com/apache/zookeeper/pull/1211#discussion_r397472588
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java
 ##########
 @@ -1237,6 +1240,15 @@ public void setTickTime(int tickTime) {
         this.tickTime = tickTime;
     }
 
+    public static int getThrottledOpWaitTime() {
+        return throttledOpWaitTime;
+    }
+
+    public static void setThrottledOpWaitTime(int time) {
+        LOG.info("throttledOpWaitTime set to " + time);
 
 Review comment:
   changed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] jhuan31 commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…

Posted by GitBox <gi...@apache.org>.
jhuan31 commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…
URL: https://github.com/apache/zookeeper/pull/1211#discussion_r364884269
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/CommitProcessor.java
 ##########
 @@ -306,6 +309,11 @@ public void run() {
                         // Process committed head
                         request = committedRequests.peek();
 
+                        if (request.isThrottled()) {
+                            LOG.error("Throttled request in committed pool: " + request + ". Exiting.");
+                            System.exit(1);
 
 Review comment:
   fixed. Thanks!

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] anmolnar commented on issue #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…

Posted by GitBox <gi...@apache.org>.
anmolnar commented on issue #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…
URL: https://github.com/apache/zookeeper/pull/1211#issuecomment-586198980
 
 
   @eolivelli @jhuan31 
   
   Theoretic question: this is a very useful and huge patch from FB. We discussed that 3.6.0 will be the "Facebook" release which means contains all patches which FB folks wanted to upstream. So, question for committers: can we submit this to `branch-3.6` once accepted?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] ztzg commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…

Posted by GitBox <gi...@apache.org>.
ztzg commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…
URL: https://github.com/apache/zookeeper/pull/1211#discussion_r364609931
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/SyncRequestProcessor.java
 ##########
 @@ -178,7 +178,7 @@ public void run() {
                 ServerMetrics.getMetrics().SYNC_PROCESSOR_QUEUE_TIME.add(startProcessTime - si.syncQueueStartTime);
 
                 // track the number of records written to the log
-                if (zks.getZKDatabase().append(si)) {
+                if (!si.isThrottled() && zks.getZKDatabase().append(si)) {
 
 Review comment:
   I believe this new code to be correct, but its "shape" raised a red flag for me as 1/ there is an `else` branch and 2/ it sports an "outdated" comment (the "iff read," in particular, is confusing):
   
   ```java
   } else if (toFlush.isEmpty()) {
       // optimization for read heavy workloads
       // iff this is a read, and there are no pending
       // flushes (writes), then just pass this to the next
       // processor
   ```
   
   (Feel free to ignore this, but I thought I'd mention it.)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] ztzg commented on issue #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…

Posted by GitBox <gi...@apache.org>.
ztzg commented on issue #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…
URL: https://github.com/apache/zookeeper/pull/1211#issuecomment-573617902
 
 
   > @ztzg yes, ZookKeeper Friends @ Facebook are porting most of their internal changes to Apache ZooKeeper.  They are creating JIRAs.
   
   Great :)
   
   > If you check on the ML they are hosting a meetup this month in order to present their work. They did the same some month ago.
   
   Right; I should have mentioned that.  I won't be able to attend (I'm in Germany), at least this time, but am planning to try and follow the live stream.  But I now see that there are recordings of the previous sessions; I will check those!
   
   > It is a great chance for the community to benefit from their experience as they are running ZooKeeper at scale.
   
   Indeed!
   
   Cheers, -D

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] jhuan31 commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…

Posted by GitBox <gi...@apache.org>.
jhuan31 commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…
URL: https://github.com/apache/zookeeper/pull/1211#discussion_r397472860
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/Request.java
 ##########
 @@ -100,6 +100,8 @@ public Request(long sessionId, int xid, int type, TxnHeader hdr, Record txn, lon
 
     public long syncQueueStartTime;
 
+    public long requestThrottleQueueTime;
 
 Review comment:
   You've saved my PR again :) Thanks!

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] hanm commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…

Posted by GitBox <gi...@apache.org>.
hanm commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…
URL: https://github.com/apache/zookeeper/pull/1211#discussion_r389966549
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/FinalRequestProcessor.java
 ##########
 @@ -150,6 +139,24 @@ public void processRequest(Request request) {
             }
         }
 
+        return rc;
+    }
+
+    public void processRequest(Request request) {
+        LOG.debug("Processing request:: " + request);
 
 Review comment:
   Update -  I realized that this was not new code, so my previous comment is non blocking for landing the PR. But it's still good to update the log format since we are here...

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] ztzg commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…

Posted by GitBox <gi...@apache.org>.
ztzg commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…
URL: https://github.com/apache/zookeeper/pull/1211#discussion_r364643293
 
 

 ##########
 File path: zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md
 ##########
 @@ -1133,6 +1133,15 @@ property, when available, is noted below.
     effect due to TLS handshake timeout when there are too many in-flight TLS 
     handshakes. Set it to something like 250 is good enough to avoid herd effect.
 
+* *throttledOpWaitTime*
+    (Jave system property: **zookeeper.throttled_op_wait_time**)
+    The time in the RequestThrottler queue longer than which a request will be marked as throttled.
 
 Review comment:
   Nit: `Jave` is a typo for `Java`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] jhuan31 commented on issue #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…

Posted by GitBox <gi...@apache.org>.
jhuan31 commented on issue #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…
URL: https://github.com/apache/zookeeper/pull/1211#issuecomment-603528767
 
 
   retest maven build

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] asfgit closed pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…
URL: https://github.com/apache/zookeeper/pull/1211
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] hanm commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…

Posted by GitBox <gi...@apache.org>.
hanm commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…
URL: https://github.com/apache/zookeeper/pull/1211#discussion_r389972299
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java
 ##########
 @@ -1237,6 +1240,15 @@ public void setTickTime(int tickTime) {
         this.tickTime = tickTime;
     }
 
+    public static int getThrottledOpWaitTime() {
+        return throttledOpWaitTime;
+    }
+
+    public static void setThrottledOpWaitTime(int time) {
+        LOG.info("throttledOpWaitTime set to " + time);
 
 Review comment:
   please use parameterized logging here :)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [zookeeper] ztzg commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…

Posted by GitBox <gi...@apache.org>.
ztzg commented on a change in pull request #1211: ZOOKEEPER-3683: Discard requests that are delayed longer than a confi…
URL: https://github.com/apache/zookeeper/pull/1211#discussion_r364633021
 
 

 ##########
 File path: zookeeper-server/src/test/java/org/apache/zookeeper/test/ThrottledOpHelper.java
 ##########
 @@ -0,0 +1,224 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import mockit.Mock;
+import mockit.MockUp;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.server.Request;
+import org.apache.zookeeper.server.RequestThrottler;
+import org.apache.zookeeper.server.ZooKeeperServer;
+import org.junit.Assert;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ThrottledOpHelper {
+    protected static final Logger LOG = LoggerFactory.getLogger(ThrottledOpHelper.class);
+
+    public static final class RequestThrottleMock extends MockUp<RequestThrottler> {
+        public static void throttleEveryNthOp(int n) {
+            everyNthOp = n;
+            opCounter = 0;
+        }
+        private static int everyNthOp = 0;
+        private static int opCounter = 0;
+
+        @Mock
+        private boolean shouldThrottleOp(Request request, long elapsedTime) {
+            if (everyNthOp > 0 && request.isThrottlable() && (++opCounter % everyNthOp == 0)) {
+                opCounter %= everyNthOp;
+                return true;
+            }
+            return false;
+        }
+    }
+
+    public static void applyMockUps() {
+        new RequestThrottleMock();
+    }
+
+    public void testThrottledOp(ZooKeeper zk, ZooKeeperServer zs) throws IOException, InterruptedException, KeeperException {
+        final int N = 5; // must be greater than 3
+        final int COUNT = 100;
+        RequestThrottleMock.throttleEveryNthOp(N);
+        LOG.info("Before create /ivailo nodes");
+        int opCount = 0;
+        for (int i = 0; i < COUNT; i++) {
+            String nodeName = "/ivailo" + i;
+            if (opCount % N == N - 1) {
+                try {
+                    zk.create(nodeName, "".getBytes(), Ids.OPEN_ACL_UNSAFE,
+                        (i % 2 == 0) ? CreateMode.PERSISTENT : CreateMode.EPHEMERAL);
+                    Assert.fail("Should have gotten ThrottledOp exception");
+                } catch (KeeperException.ThrottledOpException e) {
+                    // anticipated outcome
+                    Stat stat = zk.exists(nodeName, null);
+                    Assert.assertNull(stat);
+                    zk.create(nodeName, "".getBytes(), Ids.OPEN_ACL_UNSAFE,
+                        (i % 2 == 0) ? CreateMode.PERSISTENT : CreateMode.EPHEMERAL);
+                } catch (KeeperException e) {
+                    Assert.fail("Should have gotten ThrottledOp exception");
+                }
+                opCount += 3; // three ops issues
+            } else {
+                zk.create(nodeName, "".getBytes(), Ids.OPEN_ACL_UNSAFE,
+                    (i % 2 == 0) ? CreateMode.PERSISTENT : CreateMode.EPHEMERAL);
+                opCount++; // one op only issued
+            }
+            if (opCount % N == N - 1) {
+                try {
+                    zk.setData(nodeName, nodeName.getBytes(), -1);
+                    Assert.fail("Should have gotten ThrottledOp exception");
+                } catch (KeeperException.ThrottledOpException e) {
+                    // anticipated outcome & retry
+                    zk.setData(nodeName, nodeName.getBytes(), -1);
+                } catch (KeeperException e) {
+                    Assert.fail("Should have gotten ThrottledOp exception");
+                }
+                opCount += 2; // two ops issues, one for retry
+            } else {
+                zk.setData(nodeName, nodeName.getBytes(), -1);
+                opCount++; // one op only issued
+            }
+        }
+        LOG.info("Before delete /ivailo nodes");
+        for (int i = 0; i < COUNT; i++) {
+            String nodeName = "/ivailo" + i;
+            if (opCount % N == N - 1) {
+                try {
+                    zk.exists(nodeName, null);
+                    Assert.fail("Should have gotten ThrottledOp exception");
+                } catch (KeeperException.ThrottledOpException e) {
+                    // anticipated outcome & retry
+                    Stat stat = zk.exists(nodeName, null);
+                    Assert.assertNotNull(stat);
+                    opCount += 2; // two ops issued, one is retry
+                } catch (KeeperException e) {
+                    Assert.fail("Should have gotten ThrottledOp exception");
+                }
+            } else {
+                Stat stat = zk.exists(nodeName, null);
+                Assert.assertNotNull(stat);
+                opCount++;
+            }
+            if (opCount % N == N - 1) {
+                try {
+                    zk.getData(nodeName, null, null);
+                    Assert.fail("Should have gotten ThrottledOp exception");
+                } catch (KeeperException.ThrottledOpException e) {
+                    // anticipated outcome & retry
+                    byte[] data = zk.getData(nodeName, null, null);
+                    Assert.assertEquals(nodeName, new String(data));
+                    opCount += 2; // two ops issued, one is retry
+                } catch (KeeperException e) {
+                    Assert.fail("Should have gotten ThrottledOp exception");
+                }
+            } else {
+                byte[] data = zk.getData(nodeName, null, null);
+                Assert.assertEquals(nodeName, new String(data));
+                opCount++;
+            }
+            if (opCount % N == N - 1) {
+                try {
+                    // version 0 should not trigger BadVersion exception
+                    zk.delete(nodeName, 0);
 
 Review comment:
   Is this correct/intentional?  `setData` is called once on each node, so I would expect the actual version to be `1`.  Of course, the check won't trigger, as the request is throttled—I suppose that's what that comment is trying to draw attention to?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services