You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by rg...@apache.org on 2015/06/11 20:14:22 UTC

svn commit: r1684958 - in /zookeeper/branches/branch-3.4: CHANGES.txt src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java src/java/test/org/apache/zookeeper/server/PrepRequestProcessorTest.java

Author: rgs
Date: Thu Jun 11 18:14:21 2015
New Revision: 1684958

URL: http://svn.apache.org/r1684958
Log:
ZOOKEEPER-2213: Empty path in Set crashes server and prevents restart
(Hongchao Deng via rgs)

Modified:
    zookeeper/branches/branch-3.4/CHANGES.txt
    zookeeper/branches/branch-3.4/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java
    zookeeper/branches/branch-3.4/src/java/test/org/apache/zookeeper/server/PrepRequestProcessorTest.java

Modified: zookeeper/branches/branch-3.4/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/branches/branch-3.4/CHANGES.txt?rev=1684958&r1=1684957&r2=1684958&view=diff
==============================================================================
--- zookeeper/branches/branch-3.4/CHANGES.txt (original)
+++ zookeeper/branches/branch-3.4/CHANGES.txt Thu Jun 11 18:14:21 2015
@@ -90,6 +90,9 @@ BUGFIXES:
   ZOOKEEPER-2201: Network issues can cause cluster to hang due to near-deadlock
   (Donny Nadolny via rgs)
 
+  ZOOKEEPER-2213: Empty path in Set crashes server and prevents restart
+  (Hongchao Deng via rgs)
+
 IMPROVEMENTS:
 
   ZOOKEEPER-1575. adding .gitattributes to prevent CRLF and LF mismatches for

Modified: zookeeper/branches/branch-3.4/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java
URL: http://svn.apache.org/viewvc/zookeeper/branches/branch-3.4/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java?rev=1684958&r1=1684957&r2=1684958&view=diff
==============================================================================
--- zookeeper/branches/branch-3.4/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java (original)
+++ zookeeper/branches/branch-3.4/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java Thu Jun 11 18:14:21 2015
@@ -39,6 +39,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.KeeperException.BadArgumentsException;
 import org.apache.zookeeper.MultiTransactionRecord;
 import org.apache.zookeeper.Op;
 import org.apache.zookeeper.ZooDefs;
@@ -351,13 +352,7 @@ public class PrepRequestProcessor extend
                 if (createMode.isSequential()) {
                     path = path + String.format(Locale.ENGLISH, "%010d", parentCVersion);
                 }
-                try {
-                    PathUtils.validatePath(path);
-                } catch(IllegalArgumentException ie) {
-                    LOG.info("Invalid path " + path + " with session 0x" +
-                            Long.toHexString(request.sessionId));
-                    throw new KeeperException.BadArgumentsException(path);
-                }
+                validatePath(path, request.sessionId);
                 try {
                     if (getRecordForPath(path) != null) {
                         throw new KeeperException.NodeExistsException(path);
@@ -420,6 +415,7 @@ public class PrepRequestProcessor extend
                 if(deserialize)
                     ByteBufferInputStream.byteBuffer2Record(request.request, setDataRequest);
                 path = setDataRequest.getPath();
+                validatePath(path, request.sessionId);
                 nodeRecord = getRecordForPath(path);
                 checkACL(zks, nodeRecord.acl, ZooDefs.Perms.WRITE,
                         request.authInfo);
@@ -440,6 +436,7 @@ public class PrepRequestProcessor extend
                 if(deserialize)
                     ByteBufferInputStream.byteBuffer2Record(request.request, setAclRequest);
                 path = setAclRequest.getPath();
+                validatePath(path, request.sessionId);
                 listACL = removeDuplicates(setAclRequest.getAcl());
                 if (!fixupACL(request.authInfo, listACL)) {
                     throw new KeeperException.InvalidACLException(path);
@@ -499,6 +496,7 @@ public class PrepRequestProcessor extend
                 if(deserialize)
                     ByteBufferInputStream.byteBuffer2Record(request.request, checkVersionRequest);
                 path = checkVersionRequest.getPath();
+                validatePath(path, request.sessionId);
                 nodeRecord = getRecordForPath(path);
                 checkACL(zks, nodeRecord.acl, ZooDefs.Perms.READ,
                         request.authInfo);
@@ -513,6 +511,16 @@ public class PrepRequestProcessor extend
         }
     }
 
+    private void validatePath(String path, long sessionId) throws BadArgumentsException {
+        try {
+            PathUtils.validatePath(path);
+        } catch(IllegalArgumentException ie) {
+            LOG.info("Invalid path " +  path + " with session 0x" + Long.toHexString(sessionId) +
+                    ", reason: " + ie.getMessage());
+            throw new BadArgumentsException(path);
+        }
+    }
+
     /**
      * This method will be called inside the ProcessRequestThread, which is a
      * singleton, so there will be a single thread calling this code.

Modified: zookeeper/branches/branch-3.4/src/java/test/org/apache/zookeeper/server/PrepRequestProcessorTest.java
URL: http://svn.apache.org/viewvc/zookeeper/branches/branch-3.4/src/java/test/org/apache/zookeeper/server/PrepRequestProcessorTest.java?rev=1684958&r1=1684957&r2=1684958&view=diff
==============================================================================
--- zookeeper/branches/branch-3.4/src/java/test/org/apache/zookeeper/server/PrepRequestProcessorTest.java (original)
+++ zookeeper/branches/branch-3.4/src/java/test/org/apache/zookeeper/server/PrepRequestProcessorTest.java Thu Jun 11 18:14:21 2015
@@ -29,6 +29,7 @@ import org.apache.zookeeper.PortAssignme
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.ZooDefs.OpCode;
 import org.apache.zookeeper.data.Id;
+import org.apache.zookeeper.proto.SetDataRequest;
 import org.apache.zookeeper.server.ZooKeeperServer.ChangeRecord;
 import org.apache.zookeeper.test.ClientBase;
 import org.apache.zookeeper.txn.ErrorTxn;
@@ -47,6 +48,7 @@ import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.List;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
 
 public class PrepRequestProcessorTest extends ClientBase {
     private static final Logger LOG = LoggerFactory.getLogger(PrepRequestProcessorTest.class);
@@ -93,13 +95,10 @@ public class PrepRequestProcessorTest ex
 
         Assert.assertEquals("Request should have marshalling error", new ErrorTxn(KeeperException.Code.MARSHALLINGERROR.intValue()),
                 outcome.txn);
-        Assert.assertTrue("request hasn't been processed in chain",
-                pLatch.await(5, java.util.concurrent.TimeUnit.SECONDS));
+        Assert.assertTrue("request hasn't been processed in chain", pLatch.await(5, TimeUnit.SECONDS));
     }
 
-    private Request createMultiRequest(List<Op> ops) throws IOException {
-        Record record = new MultiTransactionRecord(ops);
-
+    private Request createRequest(Record record, int opCode) throws IOException {
         // encoding
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
         BinaryOutputArchive boa = BinaryOutputArchive.getArchive(baos);
@@ -109,18 +108,18 @@ public class PrepRequestProcessorTest ex
         // Id
         List<Id> ids = Arrays.asList(Ids.ANYONE_ID_UNSAFE);
 
-        return new Request(null, 1l, 0, OpCode.multi, ByteBuffer.wrap(baos.toByteArray()), ids);
+        return new Request(null, 1l, 0, opCode, ByteBuffer.wrap(baos.toByteArray()), ids);
     }
 
     private void process(List<Op> ops) throws Exception {
         pLatch = new CountDownLatch(1);
         processor = new PrepRequestProcessor(zks, new MyRequestProcessor());
 
-        Request req = createMultiRequest(ops);
+        Record record = new MultiTransactionRecord(ops);
+        Request req = createRequest(record, OpCode.multi);
 
         processor.pRequest(req);
-        Assert.assertTrue("request hasn't been processed in chain",
-                pLatch.await(5, java.util.concurrent.TimeUnit.SECONDS));
+        Assert.assertTrue("request hasn't been processed in chain", pLatch.await(5, TimeUnit.SECONDS));
     }
 
     /**
@@ -168,9 +167,6 @@ public class PrepRequestProcessorTest ex
         zks.getZKDatabase().dataTree.createNode("/foo", new byte[0], Ids.OPEN_ACL_UNSAFE, 0, 0, 0, 0);
         zks.getZKDatabase().dataTree.createNode("/foo/bar", new byte[0], Ids.OPEN_ACL_UNSAFE, 0, 0, 0, 0);
 
-        pLatch = new CountDownLatch(1);
-        processor = new PrepRequestProcessor(zks, new MyRequestProcessor());
-
         Assert.assertNull(zks.outstandingChangesForPath.get("/foo"));
 
         // multi record:
@@ -184,6 +180,23 @@ public class PrepRequestProcessorTest ex
         Assert.assertNull(zks.outstandingChangesForPath.get("/foo"));
     }
 
+    /**
+     * It tests that PrepRequestProcessor will return BadArgument KeeperException
+     * if the request path (if it exists) is not valid, e.g. empty string.
+     */
+    @Test
+    public void testInvalidPath() throws Exception {
+        pLatch = new CountDownLatch(1);
+        processor = new PrepRequestProcessor(zks, new MyRequestProcessor());
+
+        SetDataRequest record = new SetDataRequest("", new byte[0], -1);
+        Request req = createRequest(record, OpCode.setData);
+        processor.pRequest(req);
+        pLatch.await();
+        Assert.assertEquals(outcome.hdr.getType(), OpCode.error);
+        Assert.assertEquals(outcome.getException().code(), KeeperException.Code.BADARGUMENTS);
+    }
+
     private class MyRequestProcessor implements RequestProcessor {
         @Override
         public void processRequest(Request request) {