You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by ha...@apache.org on 2019/09/24 19:31:38 UTC

[zookeeper] branch branch-3.5 updated: ZOOKEEPER-3496: Transaction larger than jute.maxbuffer makes ZooKeeper service unavailable

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

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


The following commit(s) were added to refs/heads/branch-3.5 by this push:
     new 7d263d0  ZOOKEEPER-3496: Transaction larger than jute.maxbuffer makes ZooKeeper service unavailable
7d263d0 is described below

commit 7d263d0af62fbba6fa6e2e2464129bd959d81067
Author: Mohammad Arshad <ar...@apache.org>
AuthorDate: Tue Sep 24 12:31:26 2019 -0700

    ZOOKEEPER-3496: Transaction larger than jute.maxbuffer makes ZooKeeper service unavailable
    
    Author: Mohammad Arshad <ar...@apache.org>
    
    Reviewers: Enrico Olivelli <eo...@gmail.com>, Michael Han <ha...@apache.org>, Allan Lyu <fa...@apache.org>
    
    Closes #1080 from arshadmohammad/ZOOKEEPER-3496
---
 .../src/main/resources/markdown/zookeeperAdmin.md  | 11 ++++
 .../java/org/apache/jute/BinaryInputArchive.java   | 27 ++++++++--
 .../org/apache/jute/BinaryInputArchiveTest.java    | 59 +++++++++++++++++++++-
 3 files changed, 92 insertions(+), 5 deletions(-)

diff --git a/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md b/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md
index 73a6f9d..77a1c62 100644
--- a/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md
+++ b/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md
@@ -1077,6 +1077,17 @@ the variable does.
     problems will arise. This is really a sanity check. ZooKeeper is
     designed to store data on the order of kilobytes in size.
 
+* *jute.maxbuffer.extrasize*:
+    (Java system property: **zookeeper.jute.maxbuffer.extrasize**)
+    **New in 3.5.7:**
+    While processing client requests ZooKeeper server adds some additional information into 
+    the requests before persisting it as a transaction. Earlier this additional information size 
+    was fixed to 1024 bytes. For many scenarios, specially scenarios where jute.maxbuffer value
+    is more than 1 MB and request type is multi, this fixed size was insufficient.
+    To handle all the scenarios additional information size is increased from 1024 byte 
+    to same as jute.maxbuffer size and also it is made configurable through jute.maxbuffer.extrasize.
+    Generally this property is not required to be configured as default value is the most optimal value.
+
 * *skipACL* :
     (Java system property: **zookeeper.skipACL**)
     Skips ACL checks. This results in a boost in throughput,
diff --git a/zookeeper-jute/src/main/java/org/apache/jute/BinaryInputArchive.java b/zookeeper-jute/src/main/java/org/apache/jute/BinaryInputArchive.java
index 7722bff..774cbac 100644
--- a/zookeeper-jute/src/main/java/org/apache/jute/BinaryInputArchive.java
+++ b/zookeeper-jute/src/main/java/org/apache/jute/BinaryInputArchive.java
@@ -27,8 +27,23 @@ import java.io.InputStream;
  *
  */
 public class BinaryInputArchive implements InputArchive {
-    static public final String UNREASONBLE_LENGTH= "Unreasonable length = ";
+    public static final String UNREASONBLE_LENGTH= "Unreasonable length = ";
+    public static final int maxBuffer = Integer.getInteger("jute.maxbuffer", 0xfffff);
+    private static final int extraMaxBuffer;
+
+    static {
+        final Integer configuredExtraMaxBuffer =
+            Integer.getInteger("zookeeper.jute.maxbuffer.extrasize", maxBuffer);
+        if (configuredExtraMaxBuffer < 1024) {
+            // Earlier hard coded value was 1024, So the value should not be less than that value
+            extraMaxBuffer = 1024;
+        } else {
+            extraMaxBuffer = configuredExtraMaxBuffer;
+        }
+    }
     private DataInput in;
+    private int maxBufferSize;
+    private int extraMaxBufferSize;
     
     static public BinaryInputArchive getArchive(InputStream strm) {
         return new BinaryInputArchive(new DataInputStream(strm));
@@ -48,7 +63,13 @@ public class BinaryInputArchive implements InputArchive {
     }
     /** Creates a new instance of BinaryInputArchive */
     public BinaryInputArchive(DataInput in) {
+        this(in, maxBuffer, extraMaxBuffer);
+    }
+
+    public BinaryInputArchive(DataInput in, int maxBufferSize, int extraMaxBufferSize) {
         this.in = in;
+        this.maxBufferSize = maxBufferSize;
+        this.extraMaxBufferSize = extraMaxBufferSize;
     }
     
     public byte readByte(String tag) throws IOException {
@@ -83,8 +104,6 @@ public class BinaryInputArchive implements InputArchive {
     	in.readFully(b);
     	return new String(b, "UTF8");
     }
-    
-    static public final int maxBuffer = Integer.getInteger("jute.maxbuffer", 0xfffff);
 
     public byte[] readBuffer(String tag) throws IOException {
         int len = readInt(tag);
@@ -123,7 +142,7 @@ public class BinaryInputArchive implements InputArchive {
     // make up for extra fields, etc. (otherwise e.g. clients may be able to
     // write buffers larger than we can read from disk!)
     private void checkLength(int len) throws IOException {
-        if (len < 0 || len > maxBuffer + 1024) {
+        if (len < 0 || len > maxBufferSize + extraMaxBufferSize) {
             throw new IOException(UNREASONBLE_LENGTH + len);
         }
     }
diff --git a/zookeeper-jute/src/test/java/org/apache/jute/BinaryInputArchiveTest.java b/zookeeper-jute/src/test/java/org/apache/jute/BinaryInputArchiveTest.java
index 52fdae9..c676f56 100644
--- a/zookeeper-jute/src/test/java/org/apache/jute/BinaryInputArchiveTest.java
+++ b/zookeeper-jute/src/test/java/org/apache/jute/BinaryInputArchiveTest.java
@@ -21,8 +21,10 @@ import org.junit.Assert;
 import org.junit.Test;
 
 import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
 import java.io.IOException;
-
+import java.nio.ByteBuffer;
+import java.security.SecureRandom;
 
 // TODO: introduce JuteTestCase as in ZKTestCase
 public class BinaryInputArchiveTest {
@@ -41,4 +43,59 @@ public class BinaryInputArchiveTest {
                     e.getMessage().startsWith(BinaryInputArchive.UNREASONBLE_LENGTH));
         }
     }
+
+    /**
+     * Record length is more than the maxbuffer + extrasize length
+     */
+    @Test
+    public void testReadStringForRecordsHavingLengthMoreThanMaxAllowedSize() {
+        int maxBufferSize = 2000;
+        int extraMaxBufferSize = 1025;
+        //this record size is more than the max allowed size
+        int recordSize = maxBufferSize + extraMaxBufferSize + 100;
+        BinaryInputArchive ia =
+            getBinaryInputArchive(recordSize, maxBufferSize, extraMaxBufferSize);
+        try {
+            ia.readString("");
+            Assert.fail("Should have thrown an IOException");
+        } catch (IOException e) {
+            Assert.assertTrue("Not 'Unreasonable length' exception: " + e,
+                e.getMessage().startsWith(BinaryInputArchive.UNREASONBLE_LENGTH));
+        }
+    }
+
+    /**
+     * Record length is less than then maxbuffer + extrasize length
+     */
+    @Test
+    public void testReadStringForRecordsHavingLengthLessThanMaxAllowedSize()
+        throws IOException {
+        int maxBufferSize = 2000;
+        int extraMaxBufferSize = 1025;
+        int recordSize = maxBufferSize + extraMaxBufferSize - 100;
+        //Exception is not expected as record size is less than the allowed size
+        BinaryInputArchive ia =
+            getBinaryInputArchive(recordSize, maxBufferSize, extraMaxBufferSize);
+        String s = ia.readString("");
+        Assert.assertNotNull(s);
+        Assert.assertEquals(recordSize, s.getBytes().length);
+    }
+
+    private BinaryInputArchive getBinaryInputArchive(int recordSize, int maxBufferSize,
+        int extraMaxBufferSize) {
+        byte[] data = getData(recordSize);
+        DataInputStream dis = new DataInputStream(new ByteArrayInputStream(data));
+        return new BinaryInputArchive(dis, maxBufferSize, extraMaxBufferSize);
+    }
+
+    private byte[] getData(int recordSize) {
+        ByteBuffer buf = ByteBuffer.allocate(recordSize + 4);
+        buf.putInt(recordSize);
+        byte[] bytes = new byte[recordSize];
+        for (int i = 0; i < recordSize; i++) {
+            bytes[i] = (byte) 'a';
+        }
+        buf.put(bytes);
+        return buf.array();
+    }
 }