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:36:42 UTC

[zookeeper] branch master 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 master
in repository https://gitbox.apache.org/repos/asf/zookeeper.git


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

commit eb1569e4fbb55d12fa6211fbd997aa592fcb01af
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
---
 .../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 55b070a..e71828d 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 3ffbb4b..8b3e492 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
+}