You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by ph...@apache.org on 2012/12/12 07:51:41 UTC

svn commit: r1420540 - in /zookeeper/branches/branch-3.4: ./ src/java/main/org/apache/jute/ src/java/test/data/buffersize/ src/java/test/data/buffersize/create/ src/java/test/data/buffersize/create/version-2/ src/java/test/data/buffersize/set/ src/java...

Author: phunt
Date: Wed Dec 12 06:51:38 2012
New Revision: 1420540

URL: http://svn.apache.org/viewvc?rev=1420540&view=rev
Log:
ZOOKEEPER-1513. "Unreasonable length" exception while starting a server (Skye W-M via phunt)

Added:
    zookeeper/branches/branch-3.4/src/java/test/data/buffersize/
    zookeeper/branches/branch-3.4/src/java/test/data/buffersize/create/
    zookeeper/branches/branch-3.4/src/java/test/data/buffersize/create/version-2/
    zookeeper/branches/branch-3.4/src/java/test/data/buffersize/create/version-2/log.1   (with props)
    zookeeper/branches/branch-3.4/src/java/test/data/buffersize/create/version-2/snapshot.0   (with props)
    zookeeper/branches/branch-3.4/src/java/test/data/buffersize/set/
    zookeeper/branches/branch-3.4/src/java/test/data/buffersize/set/version-2/
    zookeeper/branches/branch-3.4/src/java/test/data/buffersize/set/version-2/log.1   (with props)
    zookeeper/branches/branch-3.4/src/java/test/data/buffersize/set/version-2/snapshot.0   (with props)
    zookeeper/branches/branch-3.4/src/java/test/data/buffersize/snapshot/
    zookeeper/branches/branch-3.4/src/java/test/data/buffersize/snapshot/version-2/
    zookeeper/branches/branch-3.4/src/java/test/data/buffersize/snapshot/version-2/log.1   (with props)
    zookeeper/branches/branch-3.4/src/java/test/data/buffersize/snapshot/version-2/snapshot.0   (with props)
    zookeeper/branches/branch-3.4/src/java/test/data/buffersize/snapshot/version-2/snapshot.2   (with props)
    zookeeper/branches/branch-3.4/src/java/test/org/apache/zookeeper/test/BufferSizeTest.java
Modified:
    zookeeper/branches/branch-3.4/CHANGES.txt
    zookeeper/branches/branch-3.4/build.xml
    zookeeper/branches/branch-3.4/src/java/main/org/apache/jute/BinaryInputArchive.java

Modified: zookeeper/branches/branch-3.4/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/branches/branch-3.4/CHANGES.txt?rev=1420540&r1=1420539&r2=1420540&view=diff
==============================================================================
--- zookeeper/branches/branch-3.4/CHANGES.txt (original)
+++ zookeeper/branches/branch-3.4/CHANGES.txt Wed Dec 12 06:51:38 2012
@@ -10,6 +10,9 @@ BUGFIXES:
   ZOOKEEPER-1596. Zab1_0Test should ensure that the file is closed
   (Enis Soztutar via phunt)
 
+  ZOOKEEPER-1513. "Unreasonable length" exception while starting a
+  server (Skye W-M via phunt)
+
 IMPROVEMENTS:
 
   ZOOKEEPER-1564. Allow JUnit test build with IBM Java

Modified: zookeeper/branches/branch-3.4/build.xml
URL: http://svn.apache.org/viewvc/zookeeper/branches/branch-3.4/build.xml?rev=1420540&r1=1420539&r2=1420540&view=diff
==============================================================================
--- zookeeper/branches/branch-3.4/build.xml (original)
+++ zookeeper/branches/branch-3.4/build.xml Wed Dec 12 06:51:38 2012
@@ -68,6 +68,7 @@
     <property name="test.data.dir" value="${test.java.build.dir}/data" />
     <property name="test.data.upgrade.dir" value="${test.data.dir}/upgrade" />
     <property name="test.data.invalid.dir" value="${test.data.dir}/invalidsnap" />
+    <property name="test.data.buffersize.dir" value="${test.data.dir}/buffersize" />
     <property name="test.cppunit.dir" value="${test.java.build.dir}/test-cppunit"/>
     <property name="test.tmp.dir" value="${test.java.build.dir}/tmp" />
     <property name="test.output" value="no" />
@@ -1126,6 +1127,7 @@
         <delete dir="${test.tmp.dir}" />
         <delete dir="${test.data.upgrade.dir}" />
         <delete dir="${test.data.invalid.dir}" />
+        <delete dir="${test.data.buffersize.dir}" />
         <delete dir="${test.data.dir}" />
         <mkdir dir="${test.log.dir}" />
         <mkdir dir="${test.tmp.dir}" />
@@ -1138,6 +1140,10 @@
         <copy todir="${test.data.invalid.dir}">
             <fileset dir="${basedir}/src/java/test/data/invalidsnap"/>
         </copy>
+        <mkdir dir="${test.data.buffersize.dir}" />
+        <copy todir="${test.data.buffersize.dir}">
+            <fileset dir="${basedir}/src/java/test/data/buffersize"/>
+        </copy>
        
     </target>
 

Modified: zookeeper/branches/branch-3.4/src/java/main/org/apache/jute/BinaryInputArchive.java
URL: http://svn.apache.org/viewvc/zookeeper/branches/branch-3.4/src/java/main/org/apache/jute/BinaryInputArchive.java?rev=1420540&r1=1420539&r2=1420540&view=diff
==============================================================================
--- zookeeper/branches/branch-3.4/src/java/main/org/apache/jute/BinaryInputArchive.java (original)
+++ zookeeper/branches/branch-3.4/src/java/main/org/apache/jute/BinaryInputArchive.java Wed Dec 12 06:51:38 2012
@@ -83,20 +83,15 @@ public class BinaryInputArchive implemen
     	return new String(b, "UTF8");
     }
     
-    static public final int maxBuffer = determineMaxBuffer();
-    private static int determineMaxBuffer() {
-        String maxBufferString = System.getProperty("jute.maxbuffer");
-        try {
-            return Integer.parseInt(maxBufferString);
-        } catch(Exception e) {
-            return 0xfffff;
-        }
-        
-    }
+    static public final int maxBuffer = Integer.getInteger("jute.maxbuffer", 0xfffff);
+
     public byte[] readBuffer(String tag) throws IOException {
         int len = readInt(tag);
         if (len == -1) return null;
-        if (len < 0 || len > maxBuffer) {
+        // Since this is a rough sanity check, add some padding to maxBuffer to
+        // make up for extra fields, etc. (otherwise e.g. clients may be able to
+        // write buffers larger than we can read from disk!)
+        if (len < 0 || len > maxBuffer + 1024) {
             throw new IOException("Unreasonable length = " + len);
         }
         byte[] arr = new byte[len];

Added: zookeeper/branches/branch-3.4/src/java/test/data/buffersize/create/version-2/log.1
URL: http://svn.apache.org/viewvc/zookeeper/branches/branch-3.4/src/java/test/data/buffersize/create/version-2/log.1?rev=1420540&view=auto
==============================================================================
Binary file - no diff available.

Propchange: zookeeper/branches/branch-3.4/src/java/test/data/buffersize/create/version-2/log.1
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: zookeeper/branches/branch-3.4/src/java/test/data/buffersize/create/version-2/snapshot.0
URL: http://svn.apache.org/viewvc/zookeeper/branches/branch-3.4/src/java/test/data/buffersize/create/version-2/snapshot.0?rev=1420540&view=auto
==============================================================================
Binary file - no diff available.

Propchange: zookeeper/branches/branch-3.4/src/java/test/data/buffersize/create/version-2/snapshot.0
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: zookeeper/branches/branch-3.4/src/java/test/data/buffersize/set/version-2/log.1
URL: http://svn.apache.org/viewvc/zookeeper/branches/branch-3.4/src/java/test/data/buffersize/set/version-2/log.1?rev=1420540&view=auto
==============================================================================
Binary file - no diff available.

Propchange: zookeeper/branches/branch-3.4/src/java/test/data/buffersize/set/version-2/log.1
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: zookeeper/branches/branch-3.4/src/java/test/data/buffersize/set/version-2/snapshot.0
URL: http://svn.apache.org/viewvc/zookeeper/branches/branch-3.4/src/java/test/data/buffersize/set/version-2/snapshot.0?rev=1420540&view=auto
==============================================================================
Binary file - no diff available.

Propchange: zookeeper/branches/branch-3.4/src/java/test/data/buffersize/set/version-2/snapshot.0
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: zookeeper/branches/branch-3.4/src/java/test/data/buffersize/snapshot/version-2/log.1
URL: http://svn.apache.org/viewvc/zookeeper/branches/branch-3.4/src/java/test/data/buffersize/snapshot/version-2/log.1?rev=1420540&view=auto
==============================================================================
Binary file - no diff available.

Propchange: zookeeper/branches/branch-3.4/src/java/test/data/buffersize/snapshot/version-2/log.1
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: zookeeper/branches/branch-3.4/src/java/test/data/buffersize/snapshot/version-2/snapshot.0
URL: http://svn.apache.org/viewvc/zookeeper/branches/branch-3.4/src/java/test/data/buffersize/snapshot/version-2/snapshot.0?rev=1420540&view=auto
==============================================================================
Binary file - no diff available.

Propchange: zookeeper/branches/branch-3.4/src/java/test/data/buffersize/snapshot/version-2/snapshot.0
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: zookeeper/branches/branch-3.4/src/java/test/data/buffersize/snapshot/version-2/snapshot.2
URL: http://svn.apache.org/viewvc/zookeeper/branches/branch-3.4/src/java/test/data/buffersize/snapshot/version-2/snapshot.2?rev=1420540&view=auto
==============================================================================
Binary file - no diff available.

Propchange: zookeeper/branches/branch-3.4/src/java/test/data/buffersize/snapshot/version-2/snapshot.2
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: zookeeper/branches/branch-3.4/src/java/test/org/apache/zookeeper/test/BufferSizeTest.java
URL: http://svn.apache.org/viewvc/zookeeper/branches/branch-3.4/src/java/test/org/apache/zookeeper/test/BufferSizeTest.java?rev=1420540&view=auto
==============================================================================
--- zookeeper/branches/branch-3.4/src/java/test/org/apache/zookeeper/test/BufferSizeTest.java (added)
+++ zookeeper/branches/branch-3.4/src/java/test/org/apache/zookeeper/test/BufferSizeTest.java Wed Dec 12 06:51:38 2012
@@ -0,0 +1,132 @@
+/**
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.jute.BinaryInputArchive;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.ZooKeeper;
+import org.junit.Before;
+import org.junit.Test;
+
+public class BufferSizeTest extends ClientBase {
+    public static final int TEST_MAXBUFFER = 100;
+    private static final File TEST_DATA = new File(
+            System.getProperty("test.data.dir", "build/test/data"),
+            "buffersize");
+    
+    private ZooKeeper zk;
+
+    @Before
+    public void setMaxBuffer() throws IOException, InterruptedException {
+        System.setProperty("jute.maxbuffer", "" + TEST_MAXBUFFER);
+        assertEquals("Can't set jute.maxbuffer!", TEST_MAXBUFFER, BinaryInputArchive.maxBuffer);
+        zk = createClient();
+    }
+    
+    @Test
+    public void testCreatesReqs() throws Exception {
+        testRequests(new ClientOp() {
+            @Override
+            public void execute(byte[] data) throws Exception {
+                zk.create("/create_test", data, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT_SEQUENTIAL);
+            }
+        });
+    }
+    
+    @Test
+    public void testSetReqs() throws Exception {
+        final String path = "/set_test";
+        zk.create(path, new byte[1], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        testRequests(new ClientOp() {
+            @Override
+            public void execute(byte[] data) throws Exception {
+                zk.setData(path, data, -1);
+            }
+        });
+    }
+    
+    /** Issues requests containing data smaller, equal, and greater than TEST_MAXBUFFER. */
+    private void testRequests(ClientOp clientOp) throws Exception {
+        clientOp.execute(new byte[TEST_MAXBUFFER - 60]);
+        try {
+            // This should fail since the buffer size > the data size due to extra fields
+            clientOp.execute(new byte[TEST_MAXBUFFER]);
+            fail("Request exceeding jute.maxbuffer succeeded!");
+        } catch (KeeperException.ConnectionLossException e) {}
+        try {
+            clientOp.execute(new byte[TEST_MAXBUFFER + 10]);
+            fail("Request exceeding jute.maxbuffer succeeded!");
+        } catch (KeeperException.ConnectionLossException e) {}
+    }
+
+    private interface ClientOp {
+        void execute(byte[] data) throws Exception;
+    }
+
+    @Test
+    public void testStartup() throws Exception {
+        final String path = "/test_node";
+        zk.create(path, new byte[TEST_MAXBUFFER - 60], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        zk.setData(path, new byte[TEST_MAXBUFFER - 50], -1);
+
+        stopServer();
+        startServer();
+    }
+    
+    @Test
+    public void testStartupFailureCreate() throws Exception {
+        // Empty snapshot and logfile containing a 5000-byte create
+        testStartupFailure(new File(TEST_DATA, "create"),
+                "Server started despite create exceeding jute.maxbuffer!");
+    }
+    
+    @Test
+    public void testStartupFailureSet() throws Exception {
+        // Empty snapshot and logfile containing a 1-byte create and 5000-byte set
+        testStartupFailure(new File(TEST_DATA, "set"),
+                "Server started despite set exceeding jute.maxbuffer!");
+    }
+    
+    @Test
+    public void testStartupFailureSnapshot() throws Exception {
+        // Snapshot containing 5000-byte znode and logfile containing create txn
+        testStartupFailure(new File(TEST_DATA, "snapshot"),
+                "Server started despite znode exceeding jute.maxbuffer!");
+    }
+    
+    private void testStartupFailure(File testDir, String failureMsg) throws Exception {
+        stopServer();
+        // Point server at testDir
+        tmpDir = testDir;
+        try {
+            startServer();
+            fail(failureMsg);
+        } catch (IOException e) {
+            LOG.info("Successfully caught IOException: " + e);
+        }
+    }
+}