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 2014/07/25 01:31:58 UTC

svn commit: r1613328 - in /zookeeper/trunk: CHANGES.txt src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java src/java/test/org/apache/zookeeper/test/ClientSkipACLTest.java src/java/test/org/apache/zookeeper/test/ClientTest.java

Author: phunt
Date: Thu Jul 24 23:31:57 2014
New Revision: 1613328

URL: http://svn.apache.org/r1613328
Log:
ZOOKEEPER-1877. Malformed ACL Id can crash server with skipACL=yes (Chris Chen via phunt)

Added:
    zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientSkipACLTest.java
Modified:
    zookeeper/trunk/CHANGES.txt
    zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java
    zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java

Modified: zookeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/trunk/CHANGES.txt?rev=1613328&r1=1613327&r2=1613328&view=diff
==============================================================================
--- zookeeper/trunk/CHANGES.txt (original)
+++ zookeeper/trunk/CHANGES.txt Thu Jul 24 23:31:57 2014
@@ -738,6 +738,9 @@ BUGFIXES:
   ZOOKEEPER-1937. init script needs fixing for ZOOKEEPER-1719
   (Marshall McMullen via phunt)
 
+  ZOOKEEPER-1877. Malformed ACL Id can crash server with skipACL=yes
+  (Chris Chen via phunt)
+
 IMPROVEMENTS:
 
   ZOOKEEPER-1170. Fix compiler (eclipse) warnings: unused imports,

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java?rev=1613328&r1=1613327&r2=1613328&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java Thu Jul 24 23:31:57 2014
@@ -376,10 +376,7 @@ public class PrepRequestProcessor extend
                 String path = createRequest.getPath();
                 String parentPath = validatePathForCreate(path, request.sessionId);
 
-                List<ACL> listACL = removeDuplicates(createRequest.getAcl());
-                if (!fixupACL(request.authInfo, listACL)) {
-                    throw new KeeperException.InvalidACLException(path);
-                }
+                List<ACL> listACL = fixupACL(path, request.authInfo, createRequest.getAcl());
                 ChangeRecord parentRecord = getRecordForPath(parentPath);
 
                 checkACL(zks, parentRecord.acl, ZooDefs.Perms.CREATE, request.authInfo);
@@ -429,10 +426,7 @@ public class PrepRequestProcessor extend
                 String path = createRequest.getPath();
                 String parentPath = validatePathForCreate(path, request.sessionId);
 
-                List<ACL> listACL = removeDuplicates(createRequest.getAcl());
-                if (!fixupACL(request.authInfo, listACL)) {
-                    throw new KeeperException.InvalidACLException(path);
-                }
+                List<ACL> listACL = fixupACL(path, request.authInfo, createRequest.getAcl());
                 ChangeRecord parentRecord = getRecordForPath(parentPath);
 
                 checkACL(zks, parentRecord.acl, ZooDefs.Perms.CREATE, request.authInfo);
@@ -633,10 +627,7 @@ public class PrepRequestProcessor extend
                 if(deserialize)
                     ByteBufferInputStream.byteBuffer2Record(request.request, setAclRequest);
                 path = setAclRequest.getPath();
-                List<ACL> listACL = removeDuplicates(setAclRequest.getAcl());
-                if (!fixupACL(request.authInfo, listACL)) {
-                    throw new KeeperException.InvalidACLException(path);
-                }
+                List<ACL> listACL = fixupACL(path, request.authInfo, setAclRequest.getAcl());
                 nodeRecord = getRecordForPath(path);
                 checkACL(zks, nodeRecord.acl, ZooDefs.Perms.ADMIN, request.authInfo);
                 newVersion = checkAndIncVersion(nodeRecord.stat.getAversion(), setAclRequest.getVersion(), path);
@@ -921,62 +912,57 @@ public class PrepRequestProcessor extend
      * depend on the requestor's authentication information.
      *
      * @param authInfo list of ACL IDs associated with the client connection
-     * @param acl list of ACLs being assigned to the node (create or setACL operation)
-     * @return
+     * @param acls list of ACLs being assigned to the node (create or setACL operation)
+     * @return verified and expanded ACLs
+     * @throws KeeperException.InvalidACLException
      */
-    private boolean fixupACL(List<Id> authInfo, List<ACL> acl) {
-        if (skipACL) {
-            return true;
-        }
-        if (acl == null || acl.size() == 0) {
-            return false;
+    private List<ACL> fixupACL(String path, List<Id> authInfo, List<ACL> acls)
+        throws KeeperException.InvalidACLException {
+        // check for well formed ACLs
+        // This resolves https://issues.apache.org/jira/browse/ZOOKEEPER-1877
+        List<ACL> uniqacls = removeDuplicates(acls);
+        LinkedList<ACL> rv = new LinkedList<ACL>();
+        if (uniqacls == null || uniqacls.size() == 0) {
+            throw new KeeperException.InvalidACLException(path);
         }
-
-        Iterator<ACL> it = acl.iterator();
-        LinkedList<ACL> toAdd = null;
-        while (it.hasNext()) {
-            ACL a = it.next();
+        for (ACL a: uniqacls) {
+            LOG.debug("Processing ACL: {}", a);
+            if (a == null) {
+                throw new KeeperException.InvalidACLException(path);
+            }
             Id id = a.getId();
+            if (id == null || id.getScheme() == null) {
+                throw new KeeperException.InvalidACLException(path);
+            }
             if (id.getScheme().equals("world") && id.getId().equals("anyone")) {
-                // wide open
+                rv.add(a);
             } else if (id.getScheme().equals("auth")) {
                 // This is the "auth" id, so we have to expand it to the
                 // authenticated ids of the requestor
-                it.remove();
-                if (toAdd == null) {
-                    toAdd = new LinkedList<ACL>();
-                }
                 boolean authIdValid = false;
                 for (Id cid : authInfo) {
                     AuthenticationProvider ap =
                         ProviderRegistry.getProvider(cid.getScheme());
                     if (ap == null) {
                         LOG.error("Missing AuthenticationProvider for "
-                                + cid.getScheme());
+                            + cid.getScheme());
                     } else if (ap.isAuthenticated()) {
                         authIdValid = true;
-                        toAdd.add(new ACL(a.getPerms(), cid));
+                        rv.add(new ACL(a.getPerms(), cid));
                     }
                 }
                 if (!authIdValid) {
-                    return false;
+                    throw new KeeperException.InvalidACLException(path);
                 }
             } else {
                 AuthenticationProvider ap = ProviderRegistry.getProvider(id.getScheme());
-                if (ap == null) {
-                    return false;
-                }
-                if (!ap.isValid(id.getId())) {
-                    return false;
+                if (ap == null || !ap.isValid(id.getId())) {
+                    throw new KeeperException.InvalidACLException(path);
                 }
+                rv.add(a);
             }
         }
-        if (toAdd != null) {
-            for (ACL a : toAdd) {
-                acl.add(a);
-            }
-        }
-        return acl.size() > 0;
+        return rv;
     }
 
     public void processRequest(Request request) {

Added: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientSkipACLTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientSkipACLTest.java?rev=1613328&view=auto
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientSkipACLTest.java (added)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientSkipACLTest.java Thu Jul 24 23:31:57 2014
@@ -0,0 +1,34 @@
+/**
+ * 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 org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+public class ClientSkipACLTest extends ClientTest {
+    @BeforeClass
+    static public void setup() {
+        System.setProperty("zookeeper.skipACL", "yes");
+    }
+
+    @AfterClass
+    static public void teardown() {
+        System.clearProperty("zookeeper.skipACL");
+    }
+}
\ No newline at end of file

Modified: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java?rev=1613328&r1=1613327&r2=1613328&view=diff
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java (original)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java Thu Jul 24 23:31:57 2014
@@ -54,6 +54,7 @@ import org.slf4j.LoggerFactory;
 
 public class ClientTest extends ClientBase {
     protected static final Logger LOG = LoggerFactory.getLogger(ClientTest.class);
+    private boolean skipACL = System.getProperty("zookeeper.skipACL", "no").equals("yes");
 
     /** Verify that pings are sent, keeping the "idle" client alive */
     @Test
@@ -142,23 +143,43 @@ public class ClientTest extends ClientBa
                 LOG.info("Test successful, invalid acl received : "
                         + e.getMessage());
             }
+            try {
+                ArrayList<ACL> testACL = new ArrayList<ACL>();
+                testACL.add(new ACL(Perms.ALL | Perms.ADMIN, new Id()));
+                zk.create("/nullidtest", new byte[0], testACL, CreateMode.PERSISTENT);
+                Assert.fail("Should have received an invalid acl error");
+            } catch(InvalidACLException e) {
+                LOG.info("Test successful, invalid acl received : "
+                        + e.getMessage());
+            }
             zk.addAuthInfo("digest", "ben:passwd".getBytes());
-            zk.create("/acltest", new byte[0], Ids.CREATOR_ALL_ACL, CreateMode.PERSISTENT);
+            ArrayList<ACL> testACL = new ArrayList<ACL>();
+            testACL.add(new ACL(Perms.ALL, new Id("auth","")));
+            testACL.add(new ACL(Perms.WRITE, new Id("ip", "127.0.0.1")));
+            zk.create("/acltest", new byte[0], testACL, CreateMode.PERSISTENT);
             zk.close();
             zk = createClient();
             zk.addAuthInfo("digest", "ben:passwd2".getBytes());
-            try {
-                zk.getData("/acltest", false, new Stat());
-                Assert.fail("Should have received a permission error");
-            } catch (KeeperException e) {
-                Assert.assertEquals(Code.NOAUTH, e.code());
+            if (skipACL) {
+                try {
+                    zk.getData("/acltest", false, null);
+                } catch (KeeperException e) {
+                    Assert.fail("Badauth reads should succeed with skipACL.");
+                }
+            } else {
+                try {
+                    zk.getData("/acltest", false, null);
+                    Assert.fail("Should have received a permission error");
+                } catch (KeeperException e) {
+                    Assert.assertEquals(Code.NOAUTH, e.code());
+                }
             }
             zk.addAuthInfo("digest", "ben:passwd".getBytes());
-            zk.getData("/acltest", false, new Stat());
+            zk.getData("/acltest", false, null);
             zk.setACL("/acltest", Ids.OPEN_ACL_UNSAFE, -1);
             zk.close();
             zk = createClient();
-            zk.getData("/acltest", false, new Stat());
+            zk.getData("/acltest", false, null);
             List<ACL> acls = zk.getACL("/acltest", new Stat());
             Assert.assertEquals(1, acls.size());
             Assert.assertEquals(Ids.OPEN_ACL_UNSAFE, acls);
@@ -176,6 +197,48 @@ public class ClientTest extends ClientBa
         }
     }
 
+    @Test
+    public void testNullAuthId() throws Exception {
+        ZooKeeper zk = null;
+        try {
+            zk = createClient();
+            zk.addAuthInfo("digest", "ben:passwd".getBytes());
+            ArrayList<ACL> testACL = new ArrayList<ACL>();
+            testACL.add(new ACL(Perms.ALL, new Id("auth", null)));
+            zk.create("/acltest", new byte[0], testACL, CreateMode.PERSISTENT);
+            zk.close();
+            zk = createClient();
+            zk.addAuthInfo("digest", "ben:passwd2".getBytes());
+            if (skipACL) {
+                try {
+                    zk.getData("/acltest", false, null);
+                } catch (KeeperException e) {
+                    Assert.fail("Badauth reads should succeed with skipACL.");
+                }
+            } else {
+                try {
+                    zk.getData("/acltest", false, null);
+                    Assert.fail("Should have received a permission error");
+                } catch (KeeperException e) {
+                    Assert.assertEquals(Code.NOAUTH, e.code());
+                }
+            }
+            zk.addAuthInfo("digest", "ben:passwd".getBytes());
+            zk.getData("/acltest", false, null);
+            zk.setACL("/acltest", Ids.OPEN_ACL_UNSAFE, -1);
+            zk.close();
+            zk = createClient();
+            zk.getData("/acltest", false, null);
+            List<ACL> acls = zk.getACL("/acltest", new Stat());
+            Assert.assertEquals(1, acls.size());
+            Assert.assertEquals(Ids.OPEN_ACL_UNSAFE, acls);
+        } finally {
+            if (zk != null) {
+                zk.close();
+            }
+        }
+    }
+
     private class MyWatcher extends CountdownWatcher {
         LinkedBlockingQueue<WatchedEvent> events =
             new LinkedBlockingQueue<WatchedEvent>();