You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@curator.apache.org by ra...@apache.org on 2013/11/08 17:40:35 UTC

[3/3] git commit: CURATOR-58

CURATOR-58

EnsurePath and creatingParentsIfNeeded() were not using the ACLProvider to create parent nodes. The fix for this
is complicated by the fact that ACLProvider is in the framework module but ZKPaths is in the client module. Workaround
by introducing a InternalACLProvider and making ACLProvider a "synonym" for it. It can now be passed to ZKPaths.mkdirs().


Project: http://git-wip-us.apache.org/repos/asf/curator/repo
Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/16b9438f
Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/16b9438f
Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/16b9438f

Branch: refs/heads/CURATOR-58
Commit: 16b9438fe34dc00cdd527e53dc7e71b9cf13b695
Parents: c5a7ee4
Author: randgalt <ra...@apache.org>
Authored: Fri Nov 8 08:38:57 2013 -0800
Committer: randgalt <ra...@apache.org>
Committed: Fri Nov 8 08:38:57 2013 -0800

----------------------------------------------------------------------
 .../org/apache/curator/utils/EnsurePath.java    | 77 ++++++++-------
 .../curator/utils/InternalACLProvider.java      | 42 +++++++++
 .../java/org/apache/curator/utils/ZKPaths.java  | 36 ++++++-
 .../curator/framework/api/ACLProvider.java      |  3 +-
 .../framework/imps/CreateBuilderImpl.java       |  4 +-
 .../curator/framework/imps/NamespaceImpl.java   |  2 +-
 .../framework/recipes/locks/TestLockACLs.java   | 98 ++++++++++++++++++++
 7 files changed, 224 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/16b9438f/curator-client/src/main/java/org/apache/curator/utils/EnsurePath.java
----------------------------------------------------------------------
diff --git a/curator-client/src/main/java/org/apache/curator/utils/EnsurePath.java b/curator-client/src/main/java/org/apache/curator/utils/EnsurePath.java
index 2a6c0e9..15687c4 100644
--- a/curator-client/src/main/java/org/apache/curator/utils/EnsurePath.java
+++ b/curator-client/src/main/java/org/apache/curator/utils/EnsurePath.java
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.curator.utils;
 
 import org.apache.curator.CuratorZookeeperClient;
@@ -26,18 +27,18 @@ import java.util.concurrent.atomic.AtomicReference;
 
 /**
  * <p>
- *     Utility to ensure that a particular path is created.
+ * Utility to ensure that a particular path is created.
  * </p>
- *
+ * <p/>
  * <p>
- *     The first time it is used, a synchronized call to {@link ZKPaths#mkdirs(ZooKeeper, String)} is made to
- *     ensure that the entire path has been created (with an empty byte array if needed). Subsequent
- *     calls with the instance are un-synchronized NOPs.
+ * The first time it is used, a synchronized call to {@link ZKPaths#mkdirs(ZooKeeper, String)} is made to
+ * ensure that the entire path has been created (with an empty byte array if needed). Subsequent
+ * calls with the instance are un-synchronized NOPs.
  * </p>
- *
+ * <p/>
  * <p>
- *     Usage:<br/>
- *     <code><pre>
+ * Usage:<br/>
+ * <code><pre>
  *         EnsurePath       ensurePath = new EnsurePath(aFullPathToEnsure);
  *         ...
  *         String           nodePath = aFullPathToEnsure + "/foo";
@@ -51,11 +52,12 @@ import java.util.concurrent.atomic.AtomicReference;
  */
 public class EnsurePath
 {
-    private final String                    path;
-    private final boolean                   makeLastNode;
-    private final AtomicReference<Helper>   helper;
+    private final String path;
+    private final boolean makeLastNode;
+    private final InternalACLProvider aclProvider;
+    private final AtomicReference<Helper> helper;
 
-    private static final Helper             doNothingHelper = new Helper()
+    private static final Helper doNothingHelper = new Helper()
     {
         @Override
         public void ensure(CuratorZookeeperClient client, String path, final boolean makeLastNode) throws Exception
@@ -66,7 +68,7 @@ public class EnsurePath
 
     private interface Helper
     {
-        public void     ensure(CuratorZookeeperClient client, String path, final boolean makeLastNode) throws Exception;
+        public void ensure(CuratorZookeeperClient client, String path, final boolean makeLastNode) throws Exception;
     }
 
     /**
@@ -74,7 +76,16 @@ public class EnsurePath
      */
     public EnsurePath(String path)
     {
-        this(path, null, true);
+        this(path, null, true, null);
+    }
+
+    /**
+     * @param path the full path to ensure
+     * @param aclProvider if not null, the ACL provider to use when creating parent nodes
+     */
+    public EnsurePath(String path, InternalACLProvider aclProvider)
+    {
+        this(path, null, true, aclProvider);
     }
 
     /**
@@ -84,9 +95,9 @@ public class EnsurePath
      * @param client ZK client
      * @throws Exception ZK errors
      */
-    public void     ensure(CuratorZookeeperClient client) throws Exception
+    public void ensure(CuratorZookeeperClient client) throws Exception
     {
-        Helper  localHelper = helper.get();
+        Helper localHelper = helper.get();
         localHelper.ensure(client, path, makeLastNode);
     }
 
@@ -98,13 +109,14 @@ public class EnsurePath
      */
     public EnsurePath excludingLast()
     {
-        return new EnsurePath(path, helper, false);
+        return new EnsurePath(path, helper, false, aclProvider);
     }
 
-    private EnsurePath(String path, AtomicReference<Helper> helper, boolean makeLastNode)
+    private EnsurePath(String path, AtomicReference<Helper> helper, boolean makeLastNode, InternalACLProvider aclProvider)
     {
         this.path = path;
         this.makeLastNode = makeLastNode;
+        this.aclProvider = aclProvider;
         this.helper = (helper != null) ? helper : new AtomicReference<Helper>(new InitialHelper());
     }
 
@@ -113,13 +125,14 @@ public class EnsurePath
      *
      * @return the path being ensured
      */
-    public String getPath() {
+    public String getPath()
+    {
         return this.path;
     }
 
     private class InitialHelper implements Helper
     {
-        private boolean         isSet = false;  // guarded by synchronization
+        private boolean isSet = false;  // guarded by synchronization
 
         @Override
         public synchronized void ensure(final CuratorZookeeperClient client, final String path, final boolean makeLastNode) throws Exception
@@ -127,20 +140,20 @@ public class EnsurePath
             if ( !isSet )
             {
                 RetryLoop.callWithRetry
-                (
-                    client,
-                    new Callable<Object>()
-                    {
-                        @Override
-                        public Object call() throws Exception
+                    (
+                        client,
+                        new Callable<Object>()
                         {
-                            ZKPaths.mkdirs(client.getZooKeeper(), path, makeLastNode);
-                            helper.set(doNothingHelper);
-                            isSet = true;
-                            return null;
+                            @Override
+                            public Object call() throws Exception
+                            {
+                                ZKPaths.mkdirs(client.getZooKeeper(), path, makeLastNode, aclProvider);
+                                helper.set(doNothingHelper);
+                                isSet = true;
+                                return null;
+                            }
                         }
-                    }
-                );
+                    );
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/curator/blob/16b9438f/curator-client/src/main/java/org/apache/curator/utils/InternalACLProvider.java
----------------------------------------------------------------------
diff --git a/curator-client/src/main/java/org/apache/curator/utils/InternalACLProvider.java b/curator-client/src/main/java/org/apache/curator/utils/InternalACLProvider.java
new file mode 100644
index 0000000..661a55f
--- /dev/null
+++ b/curator-client/src/main/java/org/apache/curator/utils/InternalACLProvider.java
@@ -0,0 +1,42 @@
+/**
+ * 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.curator.utils;
+
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import java.util.List;
+
+public interface InternalACLProvider
+{
+    /**
+     * Return the ACL list to use by default (usually {@link ZooDefs.Ids#OPEN_ACL_UNSAFE}).
+     *
+     * @return default ACL list
+     */
+    public List<ACL> getDefaultAcl();
+
+    /**
+     * Return the ACL list to use for the given path
+     *
+     * @param path path (NOTE: might be null)
+     * @return ACL list
+     */
+    public List<ACL>        getAclForPath(String path);
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/16b9438f/curator-client/src/main/java/org/apache/curator/utils/ZKPaths.java
----------------------------------------------------------------------
diff --git a/curator-client/src/main/java/org/apache/curator/utils/ZKPaths.java b/curator-client/src/main/java/org/apache/curator/utils/ZKPaths.java
index 4f178d5..04b2141 100644
--- a/curator-client/src/main/java/org/apache/curator/utils/ZKPaths.java
+++ b/curator-client/src/main/java/org/apache/curator/utils/ZKPaths.java
@@ -23,6 +23,7 @@ import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.ACL;
 import java.util.Collections;
 import java.util.List;
 
@@ -123,7 +124,7 @@ public class ZKPaths
     public static void mkdirs(ZooKeeper zookeeper, String path)
         throws InterruptedException, KeeperException
     {
-        mkdirs(zookeeper, path, true);
+        mkdirs(zookeeper, path, true, null);
     }
 
     /**
@@ -140,6 +141,24 @@ public class ZKPaths
     public static void mkdirs(ZooKeeper zookeeper, String path, boolean makeLastNode)
         throws InterruptedException, KeeperException
     {
+        mkdirs(zookeeper, path, makeLastNode, null);
+    }
+
+    /**
+     * Make sure all the nodes in the path are created. NOTE: Unlike File.mkdirs(), Zookeeper doesn't distinguish
+     * between directories and files. So, every node in the path is created. The data for each node is an empty blob
+     *
+     * @param zookeeper the client
+     * @param path      path to ensure
+     * @param makeLastNode if true, all nodes are created. If false, only the parent nodes are created
+     * @throws InterruptedException thread interruption
+     * @throws org.apache.zookeeper.KeeperException
+     *                              Zookeeper errors
+     * @param aclProvider if not null, the ACL provider to use when creating parent nodes
+     */
+    public static void mkdirs(ZooKeeper zookeeper, String path, boolean makeLastNode, InternalACLProvider aclProvider)
+        throws InterruptedException, KeeperException
+    {
         PathUtils.validatePath(path);
 
         int pos = 1; // skip first slash, root is guaranteed to exist
@@ -164,7 +183,20 @@ public class ZKPaths
             {
                 try
                 {
-                    zookeeper.create(subPath, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+                    List<ACL> acl = null;
+                    if ( aclProvider != null )
+                    {
+                        acl = aclProvider.getAclForPath(path);
+                        if ( acl == null )
+                        {
+                            acl = aclProvider.getDefaultAcl();
+                        }
+                    }
+                    if ( acl == null )
+                    {
+                        acl = ZooDefs.Ids.OPEN_ACL_UNSAFE;
+                    }
+                    zookeeper.create(subPath, new byte[0], acl, CreateMode.PERSISTENT);
                 }
                 catch ( KeeperException.NodeExistsException e )
                 {

http://git-wip-us.apache.org/repos/asf/curator/blob/16b9438f/curator-framework/src/main/java/org/apache/curator/framework/api/ACLProvider.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLProvider.java b/curator-framework/src/main/java/org/apache/curator/framework/api/ACLProvider.java
index 4563e00..e0c5ba9 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLProvider.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/ACLProvider.java
@@ -18,11 +18,12 @@
  */
 package org.apache.curator.framework.api;
 
+import org.apache.curator.utils.InternalACLProvider;
 import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.data.ACL;
 import java.util.List;
 
-public interface ACLProvider
+public interface ACLProvider extends InternalACLProvider
 {
     /**
      * Return the ACL list to use by default (usually {@link ZooDefs.Ids#OPEN_ACL_UNSAFE}).

http://git-wip-us.apache.org/repos/asf/curator/blob/16b9438f/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java
index ebd342f..fa95b2c 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java
@@ -504,7 +504,7 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
             {
                 try
                 {
-                    ZKPaths.mkdirs(client.getZooKeeper(), mainOperationAndData.getData().getPath(), false);
+                    ZKPaths.mkdirs(client.getZooKeeper(), mainOperationAndData.getData().getPath(), false, client.getAclProvider());
                 }
                 catch ( KeeperException e )
                 {
@@ -679,7 +679,7 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
                             {
                                 if ( createParentsIfNeeded )
                                 {
-                                    ZKPaths.mkdirs(client.getZooKeeper(), path, false);
+                                    ZKPaths.mkdirs(client.getZooKeeper(), path, false, client.getAclProvider());
                                     createdPath = client.getZooKeeper().create(path, data, acling.getAclList(path), createMode);
                                 }
                                 else

http://git-wip-us.apache.org/repos/asf/curator/blob/16b9438f/curator-framework/src/main/java/org/apache/curator/framework/imps/NamespaceImpl.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/NamespaceImpl.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/NamespaceImpl.java
index 169ded5..3fc8e80 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/NamespaceImpl.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/NamespaceImpl.java
@@ -84,6 +84,6 @@ class NamespaceImpl
 
     EnsurePath newNamespaceAwareEnsurePath(String path)
     {
-        return new EnsurePath(fixForNamespace(path));
+        return new EnsurePath(fixForNamespace(path), client.getAclProvider());
     }
 }

http://git-wip-us.apache.org/repos/asf/curator/blob/16b9438f/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestLockACLs.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestLockACLs.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestLockACLs.java
new file mode 100644
index 0000000..008b399
--- /dev/null
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestLockACLs.java
@@ -0,0 +1,98 @@
+/**
+ * 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.curator.framework.recipes.locks;
+
+import com.google.common.io.Closeables;
+import org.apache.curator.RetryPolicy;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.api.ACLProvider;
+import org.apache.curator.framework.recipes.BaseClassForTests;
+import org.apache.curator.retry.ExponentialBackoffRetry;
+import org.apache.curator.test.TestingServer;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Id;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+import java.util.Collections;
+import java.util.List;
+
+public class TestLockACLs extends BaseClassForTests
+{
+    private static final List<ACL> ACLS = Collections.singletonList(new ACL(ZooDefs.Perms.ALL, new Id("ip", "127.0.0.1")));
+
+    private CuratorFramework createClient() throws Exception
+    {
+        RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3);
+        CuratorFramework client = CuratorFrameworkFactory.builder()
+            .namespace("ns")
+            .connectString(server.getConnectString())
+            .retryPolicy(retryPolicy)
+            .aclProvider(new MyACLProvider())
+            .build();
+        client.start();
+        return client;
+    }
+
+    @Test
+    public void testLockACLs() throws Exception
+    {
+        CuratorFramework client = createClient();
+        try
+        {
+            // Create a path directly and verify that MyACLProvider is being used
+            client.create().forPath("/foo");
+            Assert.assertNotNull(client.checkExists().forPath("/foo"));
+            Assert.assertEquals(ZooDefs.Perms.ALL, client.getACL().forPath("/foo").get(0).getPerms());
+            Assert.assertEquals("ip", client.getACL().forPath("/foo").get(0).getId().getScheme());
+            Assert.assertEquals("127.0.0.1", client.getACL().forPath("/foo").get(0).getId().getId());
+
+            // Now try creating a lock and we'll see that it incorrectly has the default world ACLs
+            InterProcessReadWriteLock lock = new InterProcessReadWriteLock(client, "/bar");
+            InterProcessMutex writeLock = lock.writeLock();
+            writeLock.acquire();
+            Assert.assertNotNull(client.checkExists().forPath("/bar"));
+            Assert.assertEquals(ZooDefs.Perms.ALL, client.getACL().forPath("/bar").get(0).getPerms());
+            Assert.assertEquals("ip", client.getACL().forPath("/bar").get(0).getId().getScheme());
+            Assert.assertEquals("127.0.0.1", client.getACL().forPath("/bar").get(0).getId().getId());
+        }
+        finally
+        {
+            Closeables.closeQuietly(client);
+        }
+    }
+
+    public class MyACLProvider implements ACLProvider
+    {
+
+        @Override
+        public List<ACL> getDefaultAcl()
+        {
+            return ACLS;
+        }
+
+        @Override
+        public List<ACL> getAclForPath(String path)
+        {
+            return ACLS;
+        }
+    }
+}
\ No newline at end of file