You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by nk...@apache.org on 2021/03/01 07:39:54 UTC

[zookeeper] branch branch-3.7 updated: ZOOKEEPER-4219: Quota checks break setData in multi transactions

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

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


The following commit(s) were added to refs/heads/branch-3.7 by this push:
     new fd8b6fc  ZOOKEEPER-4219: Quota checks break setData in multi transactions
fd8b6fc is described below

commit fd8b6fcdca52285a04b2f60c019d868e1cb30d68
Author: Damien Diederen <dd...@crosstwine.com>
AuthorDate: Mon Mar 1 08:36:33 2021 +0100

    ZOOKEEPER-4219: Quota checks break setData in multi transactions
    
    Without this patch, a multi() transaction such as the one implemented
    in ZooKeeperQuotaTest.testMultiCreateThenSetDataShouldWork fails with
    MarshallingError when 'enforceQuota' is enabled.
    
    This happens whenever the node has an associated quota, whether it was
    exceeded or not.
    
    This is due to the server encountering null while trying to access a
    database node by path--whereas that node only exists as a ChangeRecord
    in the server's 'outstandingChanges' list:
    
        java.lang.NullPointerException
            at org.apache.zookeeper.server.ZooKeeperServer.checkQuota(ZooKeeperServer.java:2048)
            at org.apache.zookeeper.server.PrepRequestProcessor.pRequest2Txn(PrepRequestProcessor.java:397)
    
    The patch adds an additional 'lastData' parameter to the quota
    checking function, and passes the data from the ChangeRecord during
    'setData' operations.
    
    Author: Damien Diederen <dd...@crosstwine.com>
    
    Reviewers: Enrico Olivelli <eo...@apache.org>, Mate Szalay-Beko <sy...@apache.org>, Norbert Kalmar <nk...@apache.org>
    
    Closes #1611 from ztzg/ZOOKEEPER-4219-quota-multi-setdata
    
    (cherry picked from commit eb1569e4fbb55d12fa6211fbd997aa592fcb01af)
    Signed-off-by: Norbert Kalmar <nk...@apache.org>
---
 .../zookeeper/server/PrepRequestProcessor.java     |  4 +-
 .../apache/zookeeper/server/ZooKeeperServer.java   | 14 ++----
 .../apache/zookeeper/test/ZooKeeperQuotaTest.java  | 53 +++++++++++++++++++++-
 3 files changed, 59 insertions(+), 12 deletions(-)

diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/PrepRequestProcessor.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/PrepRequestProcessor.java
index 3e6d479..355c41e 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/PrepRequestProcessor.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/PrepRequestProcessor.java
@@ -394,7 +394,7 @@ public class PrepRequestProcessor extends ZooKeeperCriticalThread implements Req
             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);
+            zks.checkQuota(path, nodeRecord.data, setDataRequest.getData(), OpCode.setData);
             int newVersion = checkAndIncVersion(nodeRecord.stat.getVersion(), setDataRequest.getVersion(), path);
             request.setTxn(new SetDataTxn(path, setDataRequest.getData(), newVersion));
             nodeRecord = nodeRecord.duplicate(request.getHdr().getZxid());
@@ -698,7 +698,7 @@ public class PrepRequestProcessor extends ZooKeeperCriticalThread implements Req
             throw new KeeperException.NoChildrenForEphemeralsException(path);
         }
         int newCversion = parentRecord.stat.getCversion() + 1;
-        zks.checkQuota(path, data, OpCode.create);
+        zks.checkQuota(path, null, data, OpCode.create);
         if (type == OpCode.createContainer) {
             request.setTxn(new CreateContainerTxn(path, data, listACL, newCversion));
         } else if (type == OpCode.createTTL) {
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java
index 5c3f5f4..1141e45 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java
@@ -2020,14 +2020,15 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
      * check a path whether exceeded the quota.
      *
      * @param path
-     *            the path of the node
+     *            the path of the node, used for the quota prefix check
+     * @param lastData
+     *            the current node data, {@code null} for none
      * @param data
-     *            the data of the path
+     *            the data to be set, or {@code null} for none
      * @param type
      *            currently, create and setData need to check quota
      */
-
-    public void checkQuota(String path, byte[] data, int type) throws KeeperException.QuotaExceededException {
+    public void checkQuota(String path, byte[] lastData, byte[] data, int type) throws KeeperException.QuotaExceededException {
         if (!enforceQuota) {
             return;
         }
@@ -2043,11 +2044,6 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
                 checkQuota(lastPrefix, dataBytes, 1);
                 break;
             case OpCode.setData:
-                DataNode node = zkDatabase.getDataTree().getNode(path);
-                byte[] lastData;
-                synchronized (node) {
-                    lastData = node.getData();
-                }
                 checkQuota(lastPrefix, dataBytes - (lastData == null ? 0 : lastData.length), 0);
                 break;
              default:
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ZooKeeperQuotaTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ZooKeeperQuotaTest.java
index 67b524a..aacd59e 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ZooKeeperQuotaTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ZooKeeperQuotaTest.java
@@ -20,13 +20,17 @@ package org.apache.zookeeper.test;
 
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.junit.jupiter.api.Assertions.fail;
 import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
 import java.util.List;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.QuotaExceededException;
+import org.apache.zookeeper.Op;
 import org.apache.zookeeper.Quotas;
 import org.apache.zookeeper.StatsTrack;
 import org.apache.zookeeper.ZooDefs.Ids;
@@ -397,6 +401,53 @@ public class ZooKeeperQuotaTest extends ClientBase {
     }
 
     @Test
+    public void testMultiCreateThenSetDataShouldWork() throws Exception {
+        final String path = "/a";
+        final String subPath = "/a/b";
+
+        zk.create(path, null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+
+        final byte[] data13b = "Hello, World!".getBytes(StandardCharsets.UTF_8);
+
+        final StatsTrack st = new StatsTrack();
+        st.setByteHardLimit(data13b.length);
+        SetQuotaCommand.createQuota(zk, path, st);
+
+        final List<Op> ops = Arrays.asList(
+            Op.create(subPath, null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT),
+            Op.setData(subPath, data13b, -1));
+
+        zk.multi(ops);
+    }
+
+    @Test
+    public void testMultiCreateThenSetDataShouldFail() throws Exception {
+        final String path = "/a";
+        final String subPath = "/a/b";
+
+        zk.create(path, null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+
+        final byte[] data13b = "Hello, World!".getBytes(StandardCharsets.UTF_8);
+
+        final StatsTrack st = new StatsTrack();
+        st.setByteHardLimit(data13b.length - 1);
+        SetQuotaCommand.createQuota(zk, path, st);
+
+        final List<Op> ops = Arrays.asList(
+            Op.create(subPath, null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT),
+            Op.setData(subPath, data13b, -1));
+
+        try {
+            zk.multi(ops);
+            fail("should fail transaction when hard quota is exceeded");
+        } catch (QuotaExceededException e) {
+            //expected
+        }
+
+        assertNull(zk.exists(subPath, null));
+    }
+
+    @Test
     public void testDeleteBytesQuota() throws Exception {
 
         final String path = "/c1";
@@ -501,4 +552,4 @@ public class ZooKeeperQuotaTest extends ClientBase {
             }
         }
     }
-}
\ No newline at end of file
+}