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 2021/01/09 06:40:45 UTC

[GitHub] [zookeeper] maoling opened a new pull request #934: ZOOKEEPER-3301:Enforce the quota limit

maoling opened a new pull request #934:
URL: https://github.com/apache/zookeeper/pull/934


   - Thanks for the original work from ZOOKEEPER-1383, ZOOKEEPER-2593, ZOOKEEPER-451, especially the work from ZOOKEEPER-1383 contributed by [Thawan Kooburat](https://issues.apache.org/jira/secure/ViewProfile.jspa?name=thawan)(I also sign off his name in the commit message) which also implemented the very valuable throughput quota.In the further, we will also do this.
   - `zookeeper.enforeQuota`. When enabled and the client exceeds the total bytes or children count hard quota under a znode, the server will reject the request and reply the client a `QuotaExceededException` by force. The default value is: false.
   - the checkQuota involves the `create()` and `setData()` api, not including the `delete()`.
   - When users set the quota which's less than the existing stats, we give a thoughtful warning info.
   - the following code in the StatsTrack has a bad augmentability:       
   
     >             if (split.length != 2) {
     >                 throw new IllegalArgumentException("invalid string " + stats);
     >             }
   
      we do a trick to solve it for the expansibility, but we will get a little strange quota info(`Output quota for /c2 count=-1,bytes=-1=;byteHardLimit=-1;countHardLimit=5`) when using `listquota`. some UTs has covered it.
   - the logic about `checkQuota` should be put in the `PrepRequestProcessor`, other than `DataTree`.
     we will get the following two negative effects if putting `checkQuota` in the `DataTree`:
     - 1. When the write request has exceeded the quota, the corresponding transaction log will load into disk successfully.It's not good, although it has any data inconsistency issue, because when the server restart, so long as the transaction logs are applied in the same order, the exceeded nodes will not be applied into the state machine.
     - 2. the client will be blocking and waiting for the response, because when throwing `QuotaExceededException` in the the `DataTree`, the` rc.stat` will be `null` and `BinaryOutputArchive#writeRecord` will throw `NPE`.
     - 3. Overall, the pre-check about the write request should be done in the `PrepRequestProcessor`(at least before `SyncRequestProcessor`)(Look at an example from `checkACL()`)
   - more detail in the [ZOOKEEPER-3301](https://issues.apache.org/jira/browse/ZOOKEEPER-3301).
   - [Added in 2020-02-25] use `RateLogger` to replace `LOG` to avoid quota exceed logs flooding the disk
   - A `TODO` improvement is: only users have admin permission can write to `/zookeeper/quota`(just like `/zookeeper/config`) to avoid some users' misoperation


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



[GitHub] [zookeeper] muse-dev[bot] commented on a change in pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
muse-dev[bot] commented on a change in pull request #934:
URL: https://github.com/apache/zookeeper/pull/934#discussion_r511423797



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/PrepRequestProcessor.java
##########
@@ -393,6 +393,7 @@ protected void pRequest2Txn(int type, long zxid, Request request, Record record,
             validatePath(path, request.sessionId);
             nodeRecord = getRecordForPath(path);
             zks.checkACL(request.cnxn, nodeRecord.acl, ZooDefs.Perms.WRITE, request.authInfo, path, null);
+            zks.checkQuota(path, setDataRequest.getData(), OpCode.setData);

Review comment:
       *THREAD_SAFETY_VIOLATION:*  Unprotected write. Non-private method `PrepRequestProcessor.pRequest2Txn(...)` indirectly writes to field `server.ZooKeeperServer.RATE_LOGGER.count` outside of synchronization.
    Reporting because this access may occur on a background thread.

##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java
##########
@@ -1946,6 +1960,123 @@ public void checkACL(ServerCnxn cnxn, List<ACL> acl, int perm, List<Id> ids, Str
         throw new KeeperException.NoAuthException();
     }
 
+    /**
+     * check a path whether exceeded the quota.
+     *
+     * @param path
+     *            the path of the node
+     * @param data
+     *            the data of the path
+     * @param type
+     *            currently, create and setData need to check quota
+     */
+
+    public void checkQuota(String path, byte[] data, int type) throws KeeperException.QuotaExceededException {
+        if (!enforceQuota) {
+            return;
+        }
+        long dataBytes = (data == null) ? 0 : data.length;
+        ZKDatabase zkDatabase = getZKDatabase();

Review comment:
       *THREAD_SAFETY_VIOLATION:*  Read/Write race. Non-private method `ZooKeeperServer.checkQuota(...)` indirectly reads without synchronization from `this.zkDb`. Potentially races with write in method `ZooKeeperServer.startdata()`.
    Reporting because this access may occur on a background thread.

##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java
##########
@@ -1946,6 +1960,123 @@ public void checkACL(ServerCnxn cnxn, List<ACL> acl, int perm, List<Id> ids, Str
         throw new KeeperException.NoAuthException();
     }
 
+    /**
+     * check a path whether exceeded the quota.
+     *
+     * @param path
+     *            the path of the node
+     * @param data
+     *            the data of the path
+     * @param type
+     *            currently, create and setData need to check quota
+     */
+
+    public void checkQuota(String path, byte[] data, int type) throws KeeperException.QuotaExceededException {
+        if (!enforceQuota) {
+            return;
+        }
+        long dataBytes = (data == null) ? 0 : data.length;
+        ZKDatabase zkDatabase = getZKDatabase();
+        String lastPrefix = zkDatabase.getDataTree().getMaxPrefixWithQuota(path);
+        if (StringUtils.isEmpty(lastPrefix)) {
+            return;
+        }
+
+        switch (type) {
+            case OpCode.create:
+                checkQuota(lastPrefix, dataBytes, 1);

Review comment:
       *THREAD_SAFETY_VIOLATION:*  Unprotected write. Non-private method `ZooKeeperServer.checkQuota(...)` indirectly writes to field `server.ZooKeeperServer.RATE_LOGGER.count` outside of synchronization.
    Reporting because this access may occur on a background thread.




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



[GitHub] [zookeeper] hanm commented on pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
hanm commented on pull request #934:
URL: https://github.com/apache/zookeeper/pull/934#issuecomment-716971854


   > Some thoughts about Quotas:
   > 
   > ### 0. Why is quota necessary?
   > It is possible for clients to send/fetch very high volumes of data and thus monopolize server resources, cause network saturation and even DOS attack to server. Having quotas protects against these issues and is all the more important in large multi-tenant clusters where a small set of badly behaved clients can degrade user experience for the well behaved ones. Quota provides an isolation effect on users applications, eliminate the noisy of neighbor, avoid users to deploy multiple sets of zk cluster for every application in most general scenarios. Of course, because in the same JVM instance, this isolation provided is not as hard as the hardware/physics isolation.
   > 
   > ### 1. Space Quota
   > We now can use quota feature to limit the the total children count or bytes under a znode. we can called this Quota type: `Space Quota`. (Not repeat it again here).
   > 
   > ### 2. Throttle Quota
   > #### 2.0 brief introduction
   > We also require another Quota type: `Throttle Quota`(or throughput Quota) which is able to limit the QPS of requests, throttle request peak/pressure and smooth request traffic. Throttle Quota can also have protective effects on the server as `RequestThrottler` did, and more flexible as it can be tuning during the runtime
   > 
   > #### 2.1 manual
   > * Throttle type: it can be expressed as READ, WRITE, or the default type(read + write).
   > * Timeframes: it can be expressed in the following units: sec, min, hour, day. Actually, for simplification, it's enough to only support sec for most user cases.
   > * Request sizes: it can be expressed in the following units: B (bytes), K (kilobytes), M (megabytes), G (gigabytes), T(
   >   terabytes), P (petabytes). For example: 1048576 bytes/sec means: allow to request 1MB data every second.
   > * Request number: it is a number. For example: 1000 reqs/sec represents: allow one thousand requests every second.
   > * How to measure the request rate?
   >   
   >   * Most systems take a `Sliding Window algorithm` approach. Its base idea is: the sliding window is that instead of making discrete jumps as the fixed window does, it smoothly slides with passing time. For example, split the one second into ten small time window(every time window is 100ms, we call it a slot or block), then judge which slot the current request located and calculate current QPS. more detail in [6]
   >   
   >   ```
   >       AtomicLong[] countPerSlot = new AtomicLong();
   >       countPerSlot[index].incrementAndGet();
   >       count.incrementAndGet();
   >       index = (index + 1) % slot;
   >       scheduledExecutorService.scheduleAtFixedRate(slidingWindowRateLimiter, 100, 100, 
   >       TimeUnit.MILLISECONDS);// slot = 10;
   >   ```
   > 
   > #### 2.2 Scope
   > Background: Suppose that we have 26 downstream businesses(from business-A to business-Z). If latency spike happens and traffic surges, how can we detect which business is responsible for this and control this emergency situation rapidly? To meet this demand, we need to do some works to identity the client. The scope of Throttle Quota can be implemented from these two alternative dimensions: `User` or `Client.id`
   > 
   > ##### 2.2.1 User:
   > ZooKeeper doesn't have a thorough user management component. For simplification, we can take advantage of the `username:password` of ACL digest auth.
   > 
   > ```
   > String auth = userName + ":" +password;
   > zk.addAuthInfo("digest", auth.getBytes());
   > ```
   > 
   > ##### 2.2.2 Client.id:
   > `client.id` is a logical grouping of clients with a meaningful name chosen by the client application.
   > `client.id` are defined in the application using the `client.id` property. A client ID identifies an application making a request.
   > 
   > ##### 2.2.2.1 How to generate the client.id?
   > * If using a unauthorized client, the client id is hard-coded by a "client.id" property in the `ZKClientConfig`.
   > * If using an authorized client, we can obtain the secured identity from client certificate as the client.id.
   >   For a Kerberos' example:
   >   ```
   >   Client {
   >     xxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxx
   >     principal="businessA/cluster2-host1@HADOOP.COM";
   >   };
   >   ```
   >   
   >   
   >   Parse the keyword: "businessA" as the client.id. Of course, an authorized client can also use the hard-coded way as the unauthorized client did. Some discussions in [5].
   > 
   > ##### 2.2.3 Hard/Soft Throttle Quota
   > * For the hard Throttle Quota, the server will reject requests and reply the client a `QuotaExceededException` by force.
   > * However, for the soft Throttle Quota, we should not do nothing, just log the warnings as count or bytes space quota did. At this case, when server detects this quota violation, the server slows down a client exceeding its quota. It computes the amount of delay needed to bring a guilty client under it's quota and delays the response for that time. This approach keeps the quota violation transparent to clients. This also keeps them from having to implement any special backoff and retry behavior which can get tricky. In fact, bad client behavior (retry without backoff) can exacerbate the very problem quotas are trying to solve. Of course at this case, users need to increase the value of `zookeeper.request.timeout` to avoid too much client's timeout
   > 
   > ### 3 Some existing designs
   > #### 3.1
   > In the original implementation of ZOOKEEPER-1383, it counted the throughput from `ServerCnxn#packetReceived` based on a simple time-window approach. However it did not distinguish between read and write requests, and it's not easy and appropriate to do it at that place: `ServerCnxn`. It used the `ServerCnxn#resetStats` to split/divide time windows, it's not good for controlling the traffic more precisely in a fine-grained manner.
   > 
   > ```
   > A Simplified request workflow:
   > request ---> ServerCnxn ---> RequestThrottler ---> PrepRequestProcessor --->......---> FinalRequestProcessor
   > ```
   > 
   > #### 3.2
   > In the FB's design[1], they combine with `RequestThrottler` based on the `Rolling-window rate throttling` approach. When request arrives at the `RequestThrottler`, asks for the `RequestOracle` to update/check Quotas and make a throttling decision, then pipeline requests in the RequestProcessors. more details in [1],[2]
   > 
   > #### Reference:
   > * [1] https://www.facebook.com/zkmeetup/videos/2984998221531756/
   > * [2] https://engineering.fb.com/open-source/zookeeper-meetup/
   > * [3] https://kafka.apache.org/documentation/#design_quotas
   > * [4] https://kafka.apache.org/documentation/#quotas
   > * [5] https://issues.apache.org/jira/browse/ZOOKEEPER-3467
   > * [6] https://tech.domain.com.au/2017/11/protect-your-api-resources-with-rate-limiting/
   
   This is an excellent write up and summarize of existing public JIRA tickets, thanks for spending time pull the information together. Can we move this into a google doc or in a specific JIRA ticket? 
   
   @arshadmohammad had an excellent point from security point of view which I agree - and I also think crafting admin only Quota APIs can be done separately because the quota enforce feature is disabled by default. 
   
   I'll give a detailed review of this later this week - please bear with my limited review bandwidth.


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



[GitHub] [zookeeper] maoling commented on pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
maoling commented on pull request #934:
URL: https://github.com/apache/zookeeper/pull/934#issuecomment-757106048


   A little busy recently, just basic


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



[GitHub] [zookeeper] maoling commented on pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
maoling commented on pull request #934:
URL: https://github.com/apache/zookeeper/pull/934#issuecomment-715908984


   @ztzg 
   - Haha, sorry for that English is not my mother tongue, so sometimes I cannot express myself clearly, I'm not feeling that you're pushing me not at all.
   
   - > You overrate me(feel an obligation to do it).
   
   The root cause is the differences between Chinese and Western cultures. Chinese advocates modesty and introspection.
   What I want to express is that my work is so trivial to undertake your praise, and it's my duty to do these things, instead of what you understand that you're putting pressure on me :)
   - I'll resolve the `musedev's` findings and show my draft about some thoughts about how the quota feature evolutes tomorrow


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



[GitHub] [zookeeper] maoling edited a comment on pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
maoling edited a comment on pull request #934:
URL: https://github.com/apache/zookeeper/pull/934#issuecomment-757106048


   A little busy recently, just basically solve the conflicts in this PR. I will have a booked time at next weekend(01-16 and 01-17) to recheck this patch and give it another push


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



[GitHub] [zookeeper] anmolnar commented on pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
anmolnar commented on pull request #934:
URL: https://github.com/apache/zookeeper/pull/934#issuecomment-708360736


   @maoling Great!


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



[GitHub] [zookeeper] maoling commented on pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
maoling commented on pull request #934:
URL: https://github.com/apache/zookeeper/pull/934#issuecomment-708105365


   @hanm @eolivelli  I will rebase this PR and address all your review comments at this weekend


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



[GitHub] [zookeeper] maoling commented on pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
maoling commented on pull request #934:
URL: https://github.com/apache/zookeeper/pull/934#issuecomment-713280035


   I have rebased this patch, but I think I need to spent some times to write a draft to think about how the quota feature evolutes in the future for discussion, I will push again at the this weekend(at latest, 10-25 )


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



[GitHub] [zookeeper] arshadmohammad commented on pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
arshadmohammad commented on pull request #934:
URL: https://github.com/apache/zookeeper/pull/934#issuecomment-715509092


   @maoling if you cover above points it will be a big code change. If you want I can submit a fresh patch for this feature.


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



[GitHub] [zookeeper] maoling commented on pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
maoling commented on pull request #934:
URL: https://github.com/apache/zookeeper/pull/934#issuecomment-716320687


   @hanm @eolivelli @anmolnar I have rebased this PR, update the PR description and address all your review comments. Please remind me if I forget something. New changes:
   
   ```
   1. Only one Exception: QuotaExceededException(delete ByteQuotaExceededException and CountQuotaExceededException)
   2. zookeeper.enforceQuota is false by default now
   3. extract a methond in the SetQuotaCommand#checkQuota to reduce the duplicated codes
   ```
   
   > Quota check sometimes is expensive so it might worth to add a new processor stage to optimize processing pipeline, instead of doing the check in pre processor.
   
   IMO, if the logics about the quota inflate in the future, we can use a new class: `QuotaManager` to decouple codes.
   
   > Quota can be applied to not just write operations, but also reads.
   
   Strongly Agree


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



[GitHub] [zookeeper] maoling commented on pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
maoling commented on pull request #934:
URL: https://github.com/apache/zookeeper/pull/934#issuecomment-716363922


   Some thoughts about Quotas:
   
   ### 0. Why is quota necessary?
   It is possible for clients to send/fetch very high volumes of data and thus monopolize server resources, cause network saturation and even DOS attack to server. Having quotas protects against these issues and is all the more important in large multi-tenant clusters where a small set of badly behaved clients can degrade user experience for the well behaved ones. Quota provides an isolation effect on users applications, eliminate the noisy of neighbor, avoid users to deploy multiple sets of zk cluster for every application in most general scenarios. Of course, because in the same JVM instance, this isolation provided is not as hard as the hardware/physics isolation.
   
   ### 1. Space Quota
   
   We now can use quota feature to limit the the total children count or bytes under a znode. we can called this Quota type: `Space Quota`. (Not repeat it again here).
   
   ### 2. Throttle Quota
   
   #### 2.0 brief introduction
   We also require another Quota type: `Throttle Quota`(or throughput Quota) which is able to limit the QPS of requests, throttle request peak/pressure and smooth request traffic. Throttle Quota can also have protective effects on the server as `RequestThrottler` did, and more flexible as it can be tuning during the runtime
   
   #### 2.1 manual
   
     - Throttle type: it can be expressed as READ, WRITE, or the default type(read + write).
     - Timeframes: it can be expressed in the following units: sec, min, hour, day. Actually, for simplification, it's enough to only support sec for most user cases. 
     - Request sizes: it can be expressed in the following units: B (bytes), K (kilobytes), M (megabytes), G (gigabytes), T(
   terabytes), P (petabytes). For example: 1048576 bytes/sec means: allow to request 1MB data every second.
     - Request number: it is a number. For example: 1000 reqs/sec represents: allow one thousand requests every second.
   
     - How to measure the request rate?
   	   - Most systems take a `Sliding Window algorithm` approach. Its base idea is: the sliding window is that instead of making discrete jumps as the fixed window does, it smoothly slides with passing time. For example, split the one second into ten small time window(every time window is 100ms, we call it a slot or block), then judge which slot the current request located and calculate current QPS. more detail in [6]
   	
         ```
             AtomicLong[] countPerSlot = new AtomicLong();
             countPerSlot[index].incrementAndGet();
             count.incrementAndGet();
             index = (index + 1) % slot;
             scheduledExecutorService.scheduleAtFixedRate(slidingWindowRateLimiter, 100, 100, 
             TimeUnit.MILLISECONDS);// slot = 10;
         ```
   
   
   #### 2.2 Scope
   Background: Suppose that we have 26 downstream businesses(from business-A to business-Z). If latency spike happens and traffic surges, how can we detect which business is responsible for this and control this emergency situation rapidly? To meet this demand, we need to do some works to identity the client. The scope of Throttle Quota can be implemented from these two alternative dimensions: `User` or `Client.id`
   
   ##### 2.2.1 User:
   ZooKeeper doesn't have a thorough user management component. For simplification, we can take advantage of the `username:password` of ACL digest auth.
   
   ```
   String auth = userName + ":" +password;
   zk.addAuthInfo("digest", auth.getBytes());
   ```
   
   ##### 2.2.2 Client.id:
   `client.id` is a logical grouping of clients with a meaningful name chosen by the client application. 
   `client.id` are defined in the application using the `client.id` property. A client ID identifies an application making a request.
   
   ##### 2.2.2.1 How to generate the client.id?
   
     - If using a unauthorized client, the client id is hard-coded by a "client.id" property in the `ZKClientConfig`. 
     - If using an authorized client, we can obtain the secured identity from client certificate as the client.id. 
         For a Kerberos' example:
   
         ```
         Client {
           xxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxx
           principal="businessA/cluster2-host1@HADOOP.COM";
         };
         ```
   
         Parse the keyword: "businessA" as the client.id. Of course, an authorized client can also use the hard-coded way as the unauthorized client did. Some discussions in [5].
   
   
   ##### 2.2.3 Hard/Soft Throttle Quota
     - For the hard Throttle Quota, the server will reject requests and reply the client a `QuotaExceededException` by force. 
     - However, for the soft Throttle Quota, we should not do nothing, just log the warnings as count or bytes space quota did. At this case, when server detects this quota violation, the server slows down a client exceeding its quota. It computes the amount of delay needed to bring a guilty client under it's quota and delays the response for that time. This approach keeps the quota violation transparent to clients. This also keeps them from having to implement any special backoff and retry behavior which can get tricky. In fact, bad client behavior (retry without backoff) can exacerbate the very problem quotas are trying to solve. Of course at this case, users need to increase the value of `zookeeper.request.timeout` to avoid too much client's timeout
   
   
   ### 3 Some existing designs
   
   #### 3.1
   In the original implementation of ZOOKEEPER-1383, it counted the throughput from `ServerCnxn#packetReceived` based on a simple time-window approach. However it did not distinguish between read and write requests, and it's not easy and appropriate to do it at that place: `ServerCnxn`. It used the `ServerCnxn#resetStats` to split/divide time windows, it's not good for controlling the traffic more precisely in a fine-grained manner.
   
   ```
   A Simplified request workflow:
   request ---> ServerCnxn ---> RequestThrottler ---> PrepRequestProcessor --->......---> FinalRequestProcessor
   ```
   
   #### 3.2
   In the FB's design[1], they combine with `RequestThrottler` based on the `Rolling-window rate throttling` approach. When request arrives at the `RequestThrottler`, asks for the `RequestOracle` to update/check Quotas and make a throttling decision, then pipeline requests in the RequestProcessors. more details in [1],[2]
   
   
   #### Reference:
     - [1] https://www.facebook.com/zkmeetup/videos/2984998221531756/
     - [2] https://engineering.fb.com/open-source/zookeeper-meetup/
     - [3] https://kafka.apache.org/documentation/#design_quotas
     - [4] https://kafka.apache.org/documentation/#quotas
     - [5] https://issues.apache.org/jira/browse/ZOOKEEPER-3467
     - [6] https://tech.domain.com.au/2017/11/protect-your-api-resources-with-rate-limiting/


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



[GitHub] [zookeeper] ztzg commented on pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
ztzg commented on pull request #934:
URL: https://github.com/apache/zookeeper/pull/934#issuecomment-715887848


   > @ztzg You overrate me(feel an obligation to do it). We still leave some other works about quota in the future, please join it.
   
   Sorry! My goal was not to "push" you at all—quite the opposite. You have already done a huge amount of work on this, and I don't think GitHub's workflows are sufficient here.
   
   As far as I know, there is no easy way for multiple people to contribute to a PR, besides reviews and comments. But those could still be useful if, for example, subsets of the patch are difficult to "rebase"; I would gladly take care of some of the files and "ship" the results to you if that helps.
   
   But okay, I'll keep quiet from now on—unless, of course, you decide to take me up on the offer :)
   
   (And yes, I am also planning to look into this and other topics in the future.)
   
   > > To make quota enforcement effective. I think,
   > 
   > @arshadmohammad Yes, I really think about the permission issue in the original github description(maybe we can take an example from what we have done for `/zookeeper/config`). I will not do it in this PR to make it bigger. Please create another ticket to do it.
   
   Right.  I 100% agree with @arshadmohammad's suggestion of the feature, but I also 100% think it should be done later on.
   


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



[GitHub] [zookeeper] arshadmohammad commented on pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
arshadmohammad commented on pull request #934:
URL: https://github.com/apache/zookeeper/pull/934#issuecomment-714318406


   To make quota enforcement effective. I think, 
   1. set quota and delete quota should be admin(super user) operation
   2. No User, admin or normal user, should be allowed to update the quota znode(/zookeeper/quota/xyz) nodes directly, All updates on quota znodes should be done through quota APIs only


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



[GitHub] [zookeeper] maoling commented on pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
maoling commented on pull request #934:
URL: https://github.com/apache/zookeeper/pull/934#issuecomment-755338236


   @ztzg
   Yes, we should create an umbrella JIRA tickets for the future works described in the design doc. 
   I'll rebase this PR tomorrow


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



[GitHub] [zookeeper] maoling commented on pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
maoling commented on pull request #934:
URL: https://github.com/apache/zookeeper/pull/934#issuecomment-761760613


   @ztzg @anmolnar  Thanks for your ping and work. I have rebased this PR, please help me recheck 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



[GitHub] [zookeeper] hanm commented on a change in pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
hanm commented on a change in pull request #934:
URL: https://github.com/apache/zookeeper/pull/934#discussion_r503653071



##########
File path: zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md
##########
@@ -856,6 +856,10 @@ property, when available, is noted below.
     commit log is triggered.
     Does not affect the limit defined by *flushDelay*.
     Default is 1000.
+* *enforeQuota* :
+    (Java system property: **zookeeper.enforeQuota**)
+    enfore the quota check. Enable this option to use the [quota feature](http://zookeeper.apache.org/doc/current/zookeeperQuotas.html).
+    the default value:true.

Review comment:
       +1 on disabling the feature by default 

##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/cli/SetQuotaCommand.java
##########
@@ -158,39 +220,79 @@ public static boolean createQuota(
         for (int i = 1; i < splits.length; i++) {
             sb.append("/").append(splits[i]);
             quotaPath = sb.toString();
-            try {
-                zk.create(quotaPath, null, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-            } catch (KeeperException.NodeExistsException ne) {
-                //do nothing
+            if (zk.exists(quotaPath, false) == null) {
+                try {
+                    zk.create(quotaPath, null, ZooDefs.Ids.OPEN_ACL_UNSAFE,
+                            CreateMode.PERSISTENT);
+                } catch (KeeperException.NodeExistsException ne) {
+                    //do nothing
+                }
             }
         }
         String statPath = quotaPath + "/" + Quotas.statNode;
         quotaPath = quotaPath + "/" + Quotas.limitNode;
-        StatsTrack strack = new StatsTrack(null);
-        strack.setBytes(bytes);
-        strack.setCount(numNodes);
-        try {
-            zk.create(quotaPath, strack.toString().getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-            StatsTrack stats = new StatsTrack(null);
-            stats.setBytes(0L);
+        byte[] data;
+
+        if (zk.exists(quotaPath, false) == null) {
+            zk.create(quotaPath, quota.toString().getBytes(),
+                    ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+
+            StatsTrack stats = new StatsTrack();
             stats.setCount(0);
-            zk.create(statPath, stats.toString().getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-        } catch (KeeperException.NodeExistsException ne) {
-            byte[] data = zk.getData(quotaPath, false, new Stat());
-            StatsTrack strackC = new StatsTrack(new String(data));
-            if (bytes != -1L) {
-                strackC.setBytes(bytes);
+            stats.setBytes(0L);
+
+            zk.create(statPath, stats.toString().getBytes(),
+                    ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+
+            data = zk.getData(quotaPath, false, new Stat());
+            StatsTrack quotaStrack = new StatsTrack(data);
+
+            data = zk.getData(statPath, false, new Stat());
+            StatsTrack statStrack = new StatsTrack(data);
+            if ((quotaStrack.getCount() > -1 && quotaStrack.getCount() < statStrack.getCount()) || (quotaStrack.getCountHardLimit() > -1

Review comment:
       There are quite a few checks in this code block that can be extracted into some sort of `checkQuota` abstraction




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



[GitHub] [zookeeper] symat commented on pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
symat commented on pull request #934:
URL: https://github.com/apache/zookeeper/pull/934#issuecomment-762090660


   I re-triggered the  github CI checks


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



[GitHub] [zookeeper] ztzg commented on pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
ztzg commented on pull request #934:
URL: https://github.com/apache/zookeeper/pull/934#issuecomment-707270267


   Hi, @maoling,
   
   For the record, I asked this question [on the `dev` mailing list](https://mail-archives.apache.org/mod_mbox/zookeeper-dev/202010.mbox/%3C87h7qzcqim.fsf%40crosstwine.com%3E):
   
   > [Is] something specific blocking your pull request, or is it just a matter of finding time to work on it?  Is there a way I can help?  It would be great to push this topic over the hump (it started with ZOOKEEPER-451!).
   
   (The whole discussion about 3.7 is visible [on this page](https://mail-archives.apache.org/mod_mbox/zookeeper-dev/202010.mbox/browser).)


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



[GitHub] [zookeeper] ztzg commented on pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
ztzg commented on pull request #934:
URL: https://github.com/apache/zookeeper/pull/934#issuecomment-761836256


   > @ztzg @anmolnar Thanks for your ping and work. I have rebased this PR, please help me recheck it.
   
   @maoling: Fantastic, thanks!  Will do!


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



[GitHub] [zookeeper] maoling closed pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
maoling closed pull request #934:
URL: https://github.com/apache/zookeeper/pull/934


   


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



[GitHub] [zookeeper] maoling closed pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
maoling closed pull request #934:
URL: https://github.com/apache/zookeeper/pull/934


   


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



[GitHub] [zookeeper] anmolnar commented on pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
anmolnar commented on pull request #934:
URL: https://github.com/apache/zookeeper/pull/934#issuecomment-760784692


   @maoling Hopefully you don't mind that I copied your great write up about Quotas to our wiki pages:
   https://cwiki.apache.org/confluence/display/ZOOKEEPER/Quotas
   
   I hope I can get back to this patch sooner rather than later, because I'd really love to see this submitted, but way too busy with other stuff.


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



[GitHub] [zookeeper] maoling commented on pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
maoling commented on pull request #934:
URL: https://github.com/apache/zookeeper/pull/934#issuecomment-715883237


   @ztzg  You overrate me(feel an obligation to do it). We still leave some other works about quota in the future, please join it.
   
   > To make quota enforcement effective. I think,
   
   @arshadmohammad Yes, I really think about the permission issue in the original github description(maybe we can take an example from what we have done for `/zookeeper/config`). I will not do it in this PR to make it bigger. Please create another ticket to do it.
   
   > All updates on quota znodes should be done through quota APIs only
   
   Maybe we can expose these new apis about quota operations in the `ZooKeeperAdmin.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



[GitHub] [zookeeper] anmolnar edited a comment on pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
anmolnar edited a comment on pull request #934:
URL: https://github.com/apache/zookeeper/pull/934#issuecomment-760784692


   @maoling Hopefully you don't mind that I copied your great write up about Quotas to our wiki pages:
   https://cwiki.apache.org/confluence/display/ZOOKEEPER/Quotas
   
   I hope I can get back to this patch sooner rather than later, because I'd really love to see this submitted, but way too busy with other stuff.
   
   Would you mind rebasing it first?


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



[GitHub] [zookeeper] ztzg closed pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
ztzg closed pull request #934:
URL: https://github.com/apache/zookeeper/pull/934


   


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



[GitHub] [zookeeper] ztzg commented on pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
ztzg commented on pull request #934:
URL: https://github.com/apache/zookeeper/pull/934#issuecomment-757543115


   > A little busy recently, just basically solve the conflicts in this PR. I will have a booked time at next weekend(01-16 and 01-17) to recheck this patch and give it another push
   
   Sure, no problem.
   
   FYI, I had also rebased the PR on top of current `master`, 1/ to see if there was anything problematic and 2/ because I will probably have to carry it as a patch on top of 3.7.0 anyway.
   
   I have pushed the results there: https://github.com/ztzg/zookeeper/commit/87fa21fc49728cac14518d5782d49123d51778e3 . I haven't spotted anything unexpected in the "interdiff"; feel free to grab (and check) my tree if it helps.
   


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



[GitHub] [zookeeper] ztzg commented on pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
ztzg commented on pull request #934:
URL: https://github.com/apache/zookeeper/pull/934#issuecomment-713732624


   @maoling: Thank you for your heroic efforts on this huge & historic PR! Let us know if there is a way we can help.
   


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



[GitHub] [zookeeper] ztzg commented on pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
ztzg commented on pull request #934:
URL: https://github.com/apache/zookeeper/pull/934#issuecomment-762681757


   Hi @maoling,
   
   Enforceable quota support is now in `master`, `branch-3.7` and `branch-3.7.0`.  Thank you very, very much for your work on this!  This comes on the heels of your promotion to committer status :)
   
   As you and other mentioned, there are more features to come.  I hope this constitutes a good basis for @hanm to refactor, and hopefully, unload, his internal patches.  @arshadmohammad had important suggestions, which I have duly noted.  As per our discussions above, I also have a few minor followups I have to take care of.
   
   Thank you @anmolnar for taking care of transferring @maoling's notes; @eolivelli, @hanm and others for the early/intermediate reviews, and @symat for the closing one—as well as re-running CI, which allowed the Jenkins test suite to succeed.
   
   


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



[GitHub] [zookeeper] muse-dev[bot] commented on a change in pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
muse-dev[bot] commented on a change in pull request #934:
URL: https://github.com/apache/zookeeper/pull/934#discussion_r559097140



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/PrepRequestProcessor.java
##########
@@ -394,6 +394,7 @@ protected void pRequest2Txn(int type, long zxid, Request request, Record record,
             validatePath(path, request.sessionId);
             nodeRecord = getRecordForPath(path);
             zks.checkACL(request.cnxn, nodeRecord.acl, ZooDefs.Perms.WRITE, request.authInfo, path, null);
+            zks.checkQuota(path, setDataRequest.getData(), OpCode.setData);

Review comment:
       *THREAD_SAFETY_VIOLATION:*  Unprotected write. Non-private method `PrepRequestProcessor.pRequest2Txn(...)` indirectly writes to field `server.ZooKeeperServer.RATE_LOGGER.msg` outside of synchronization.
    Reporting because this access may occur on a background thread.

##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java
##########
@@ -2002,6 +2016,123 @@ public void checkACL(ServerCnxn cnxn, List<ACL> acl, int perm, List<Id> ids, Str
         throw new KeeperException.NoAuthException();
     }
 
+    /**
+     * check a path whether exceeded the quota.
+     *
+     * @param path
+     *            the path of the node
+     * @param data
+     *            the data of the path
+     * @param type
+     *            currently, create and setData need to check quota
+     */
+
+    public void checkQuota(String path, byte[] data, int type) throws KeeperException.QuotaExceededException {
+        if (!enforceQuota) {
+            return;
+        }
+        long dataBytes = (data == null) ? 0 : data.length;
+        ZKDatabase zkDatabase = getZKDatabase();
+        String lastPrefix = zkDatabase.getDataTree().getMaxPrefixWithQuota(path);
+        if (StringUtils.isEmpty(lastPrefix)) {
+            return;
+        }
+
+        switch (type) {
+            case OpCode.create:
+                checkQuota(lastPrefix, dataBytes, 1);

Review comment:
       *THREAD_SAFETY_VIOLATION:*  Read/Write race. Non-private method `ZooKeeperServer.checkQuota(...)` indirectly reads without synchronization from `server.ZooKeeperServer.RATE_LOGGER.count`. Potentially races with write in method `ZooKeeperServer.checkQuota(...)`.
    Reporting because this access may occur on a background thread.




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



[GitHub] [zookeeper] maoling commented on pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
maoling commented on pull request #934:
URL: https://github.com/apache/zookeeper/pull/934#issuecomment-757106048


   A little busy recently, just basic


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



[GitHub] [zookeeper] maoling commented on pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
maoling commented on pull request #934:
URL: https://github.com/apache/zookeeper/pull/934#issuecomment-919116253


   I found another issue/bug introduced by this PR: [ZOOKEEPER-4375](https://issues.apache.org/jira/browse/ZOOKEEPER-4375) Quota cannot limit the specify value when multiply clients create/set znodes. I will fix this bug asap !!!!


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

To unsubscribe, e-mail: notifications-unsubscribe@zookeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [zookeeper] ztzg edited a comment on pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
ztzg edited a comment on pull request #934:
URL: https://github.com/apache/zookeeper/pull/934#issuecomment-761836256


   > @ztzg @anmolnar Thanks for your ping and work. I have rebased this PR, please help me recheck it.
   
   @maoling: Fantastic, thanks!  Will do!
   
   @anmolnar, @eolivelli, @hanm, @nkalmar, @symat: I need (at least) a second approval for merging this.  Is any of you up to it, so that I can pull this feature into 3.7?


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



[GitHub] [zookeeper] ztzg commented on pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
ztzg commented on pull request #934:
URL: https://github.com/apache/zookeeper/pull/934#issuecomment-715945523


   @maoling: Well, English is not my mother tongue either (I'm a French-speaking Belgian), so things can get weird sometimes :)
   
   For what it's worth, I believe the "modesty and introspection" model is a pretty good one to live by, in general!
   
   And I could perhaps have used smaller words in this case. But I really think this PR is "historic": as you know, it has its roots in ZOOKEEPER-451, if not earlier. Plus, you were the one to work, mostly alone, to bring it to its current state, so there's that. No matter what we call it, thank you!
   
   (Now, I understand how that could put some people under pressure, and I was afraid I had done so. I'm happy to learn that it was just a misunderstanding!)
   
   I have noticed your latest push, and will be happy to read & consider your notes.
   
   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



[GitHub] [zookeeper] ztzg edited a comment on pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
ztzg edited a comment on pull request #934:
URL: https://github.com/apache/zookeeper/pull/934#issuecomment-715887848


   > @ztzg You overrate me(feel an obligation to do it). We still leave some other works about quota in the future, please join it.
   
   Sorry! My goal was not to "push" you at all—quite the opposite. You have already done a huge amount of work on this, and I don't think GitHub's workflows are sufficient here.
   
   As far as I know, there is no easy way for multiple people to contribute to a PR, besides reviews and comments. But those could still be useful if, for example, subsets of the patch are difficult to "rebase"; I would gladly take care of some of the files and "ship" the results to you if that helps.
   
   But okay, I'll keep quiet from now on—unless, of course, you decide to take me up on the offer :)
   
   (And yes, I am also planning to look into this and other topics in the future.)
   
   > > To make quota enforcement effective. I think,
   > 
   > @arshadmohammad Yes, I really think about the permission issue in the original github description(maybe we can take an example from what we have done for `/zookeeper/config`). I will not do it in this PR to make it bigger. Please create another ticket to do it.
   
   Right.  I 100% agree with @arshadmohammad's suggestion of the feature (and using `/zookeeper/config` as a guide), but I also 100% think it should be done later on.
   


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



[GitHub] [zookeeper] ztzg commented on pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
ztzg commented on pull request #934:
URL: https://github.com/apache/zookeeper/pull/934#issuecomment-754882022


   Hi @maoling, all,
   
   There are quite a few outstanding good ideas in the comments above.  But I am under the impression that the PR itself is ready to be merged, and that the other items, including your design document, should be lifted into tickets and/or wiki pages.  Would you agree?  If you do, I would like to give this a last push so that it lands in 3.7.0.
   
   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



[GitHub] [zookeeper] maoling edited a comment on pull request #934: ZOOKEEPER-3301:Enforce the quota limit

Posted by GitBox <gi...@apache.org>.
maoling edited a comment on pull request #934:
URL: https://github.com/apache/zookeeper/pull/934#issuecomment-757106048


   A little busy recently, just basically solve the conflicts in this PR. I will have a booked time at next weekend(01-16 and 01-17) to recheck this patch and give it another push


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