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 2017/07/19 00:48:55 UTC

curator git commit: CURATOR-362 - Use provided ACL when creating parent directories

Repository: curator
Updated Branches:
  refs/heads/CURATOR-362 [created] 7e611bd1c


CURATOR-362 - Use provided ACL when creating parent directories

CURATOR-362 - Cover case when no ACL list is provided to CreateBuilder

CURATOR-362 - Use provided ACL for creating parents in background operation

CURATOR-362 - Use provided ACL for creating parents in background operation


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

Branch: refs/heads/CURATOR-362
Commit: 7e611bd1c7bbdea3f220af0a17d12bdf1b49b8c0
Parents: 81fc3d9
Author: szekizoli <sz...@gmail.com>
Authored: Tue May 16 23:27:56 2017 +0100
Committer: szekizoli <sz...@gmail.com>
Committed: Wed Jul 19 00:55:05 2017 +0100

----------------------------------------------------------------------
 .../api/ACLBackgroundPathAndBytesable.java      |   2 +-
 .../framework/api/ACLPathAndBytesable.java      |   2 +-
 .../apache/curator/framework/api/ACLable.java   |   2 +-
 .../framework/api/ACLableExistBuilderMain.java  |  26 ++
 .../curator/framework/api/ExistsBuilder.java    |   4 +-
 .../curator/framework/api/ParentACLable.java    |  40 +++
 .../apache/curator/framework/imps/ACLing.java   |  43 ++--
 .../framework/imps/CreateBuilderImpl.java       |  89 ++++++-
 .../framework/imps/ExistsBuilderImpl.java       |  27 +-
 .../framework/imps/SetACLBuilderImpl.java       |   2 +-
 .../curator/framework/imps/TestCreate.java      | 249 +++++++++++++++++++
 .../framework/imps/TestExistsBuilder.java       | 109 ++++++++
 12 files changed, 556 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/7e611bd1/curator-framework/src/main/java/org/apache/curator/framework/api/ACLBackgroundPathAndBytesable.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLBackgroundPathAndBytesable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/ACLBackgroundPathAndBytesable.java
index 3f9fe27..78cfab4 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLBackgroundPathAndBytesable.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/ACLBackgroundPathAndBytesable.java
@@ -19,7 +19,7 @@
 package org.apache.curator.framework.api;
 
 public interface ACLBackgroundPathAndBytesable<T> extends
-    ACLable<BackgroundPathAndBytesable<T>>,
+    ParentACLable<BackgroundPathAndBytesable<T>>,
     BackgroundPathAndBytesable<T>
 {
 }

http://git-wip-us.apache.org/repos/asf/curator/blob/7e611bd1/curator-framework/src/main/java/org/apache/curator/framework/api/ACLPathAndBytesable.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLPathAndBytesable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/ACLPathAndBytesable.java
index 2342186..8d16d50 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLPathAndBytesable.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/ACLPathAndBytesable.java
@@ -19,7 +19,7 @@
 package org.apache.curator.framework.api;
 
 public interface ACLPathAndBytesable<T> extends
-    ACLable<PathAndBytesable<T>>,
+    ParentACLable<PathAndBytesable<T>>,
     PathAndBytesable<T>
 {
 }

http://git-wip-us.apache.org/repos/asf/curator/blob/7e611bd1/curator-framework/src/main/java/org/apache/curator/framework/api/ACLable.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/ACLable.java
index afb0e02..e11f299 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLable.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/ACLable.java
@@ -30,5 +30,5 @@ public interface ACLable<T>
      * @param aclList the ACL list to use
      * @return this
      */
-    public T withACL(List<ACL> aclList);
+    T withACL(List<ACL> aclList);
 }

http://git-wip-us.apache.org/repos/asf/curator/blob/7e611bd1/curator-framework/src/main/java/org/apache/curator/framework/api/ACLableExistBuilderMain.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLableExistBuilderMain.java b/curator-framework/src/main/java/org/apache/curator/framework/api/ACLableExistBuilderMain.java
new file mode 100644
index 0000000..c16f4fe
--- /dev/null
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/ACLableExistBuilderMain.java
@@ -0,0 +1,26 @@
+/**
+ * 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.api;
+
+public interface ACLableExistBuilderMain extends
+        ExistsBuilderMain,
+        ACLable<ExistsBuilderMain>
+{
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/7e611bd1/curator-framework/src/main/java/org/apache/curator/framework/api/ExistsBuilder.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ExistsBuilder.java b/curator-framework/src/main/java/org/apache/curator/framework/api/ExistsBuilder.java
index 28b9e71..2c0cb47 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/ExistsBuilder.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/ExistsBuilder.java
@@ -29,7 +29,7 @@ public interface ExistsBuilder extends
      *
      * @return this
      */
-    ExistsBuilderMain creatingParentsIfNeeded();
+    ACLableExistBuilderMain creatingParentsIfNeeded();
 
     /**
      * Causes any parent nodes to get created using {@link CreateMode#CONTAINER} if they haven't already been.
@@ -39,5 +39,5 @@ public interface ExistsBuilder extends
      *
      * @return this
      */
-    ExistsBuilderMain creatingParentContainersIfNeeded();
+    ACLableExistBuilderMain creatingParentContainersIfNeeded();
 }

http://git-wip-us.apache.org/repos/asf/curator/blob/7e611bd1/curator-framework/src/main/java/org/apache/curator/framework/api/ParentACLable.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ParentACLable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/ParentACLable.java
new file mode 100644
index 0000000..023c9c1
--- /dev/null
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/ParentACLable.java
@@ -0,0 +1,40 @@
+/**
+ * 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.api;
+
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+
+import java.util.List;
+
+public interface ParentACLable<T> extends ACLable<T> {
+
+    /**
+     * Set an ACL list (default is {@link ZooDefs.Ids#OPEN_ACL_UNSAFE}).
+     *
+     * If applyToParents is true, then the aclList is applied to the created parents.
+     * Existing parent nodes are not affected.
+     *
+     * @param aclList the ACL list to use
+     * @param applyToParents if true, then the aclList is applied to the created parents.
+     * @return this
+     */
+    T withACL(List<ACL> aclList, boolean applyToParents);
+
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/7e611bd1/curator-framework/src/main/java/org/apache/curator/framework/imps/ACLing.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/ACLing.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/ACLing.java
index 3f367cc..84c2244 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/ACLing.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/ACLing.java
@@ -20,13 +20,15 @@ package org.apache.curator.framework.imps;
 
 import com.google.common.collect.ImmutableList;
 import org.apache.curator.framework.api.ACLProvider;
+import org.apache.curator.utils.InternalACLProvider;
 import org.apache.zookeeper.data.ACL;
 import java.util.List;
 
-class ACLing
+class ACLing implements InternalACLProvider
 {
     private final List<ACL>     aclList;
     private final ACLProvider   aclProvider;
+    private final boolean       applyToParents;
 
     ACLing(ACLProvider aclProvider)
     {
@@ -35,31 +37,42 @@ class ACLing
 
     ACLing(ACLProvider aclProvider, List<ACL> aclList)
     {
+        this(aclProvider, aclList, false);
+    }
+
+    ACLing(ACLProvider aclProvider, List<ACL> aclList, boolean applyToParents)
+    {
         this.aclProvider = aclProvider;
         this.aclList = (aclList != null) ? ImmutableList.copyOf(aclList) : null;
+        this.applyToParents = applyToParents;
+    }
+
+    InternalACLProvider getACLProviderForParents()
+    {
+        return applyToParents ? this : aclProvider;
     }
 
     List<ACL> getAclList(String path)
     {
-        List<ACL> localAclList = aclList;
-        do
+        if ( aclList != null ) return aclList;
+        if ( path != null )
         {
+            List<ACL> localAclList = aclProvider.getAclForPath(path);
             if ( localAclList != null )
             {
-                break;
+                return localAclList;
             }
+        }
+        return aclProvider.getDefaultAcl();
+    }
 
-            if ( path != null )
-            {
-                localAclList = aclProvider.getAclForPath(path);
-                if ( localAclList != null )
-                {
-                    break;
-                }
-            }
+    @Override
+    public List<ACL> getDefaultAcl() {
+        return aclProvider.getDefaultAcl();
+    }
 
-            localAclList = aclProvider.getDefaultAcl();
-        } while ( false );
-        return localAclList;
+    @Override
+    public List<ACL> getAclForPath(String path) {
+        return getAclList(path);
     }
 }

http://git-wip-us.apache.org/repos/asf/curator/blob/7e611bd1/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 406d972..416485a 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
@@ -28,6 +28,7 @@ import org.apache.curator.framework.api.*;
 import org.apache.curator.framework.api.transaction.OperationType;
 import org.apache.curator.framework.api.transaction.TransactionCreateBuilder;
 import org.apache.curator.framework.api.transaction.TransactionCreateBuilder2;
+import org.apache.curator.utils.InternalACLProvider;
 import org.apache.curator.utils.ThreadUtils;
 import org.apache.curator.utils.ZKPaths;
 import org.apache.zookeeper.AsyncCallback;
@@ -129,7 +130,13 @@ public class CreateBuilderImpl implements CreateBuilder, CreateBuilder2, Backgro
             @Override
             public PathAndBytesable<T> withACL(List<ACL> aclList)
             {
-                CreateBuilderImpl.this.withACL(aclList);
+                return withACL(aclList, false);
+            }
+
+            @Override
+            public PathAndBytesable<T> withACL(List<ACL> aclList, boolean applyToParents)
+            {
+                CreateBuilderImpl.this.withACL(aclList, applyToParents);
                 return this;
             }
 
@@ -215,6 +222,12 @@ public class CreateBuilderImpl implements CreateBuilder, CreateBuilder2, Backgro
             }
 
             @Override
+            public BackgroundPathAndBytesable<String> withACL(List<ACL> aclList, boolean applyToParents)
+            {
+                return CreateBuilderImpl.this.withACL(aclList, applyToParents);
+            }
+
+            @Override
             public ErrorListenerPathAndBytesable<String> inBackground(BackgroundCallback callback, Object context)
             {
                 return CreateBuilderImpl.this.inBackground(callback, context);
@@ -273,7 +286,13 @@ public class CreateBuilderImpl implements CreateBuilder, CreateBuilder2, Backgro
     @Override
     public ACLBackgroundPathAndBytesable<String> withACL(List<ACL> aclList)
     {
-        acling = new ACLing(client.getAclProvider(), aclList);
+        return withACL(aclList, false);
+    }
+
+    @Override
+    public ACLBackgroundPathAndBytesable<String> withACL(List<ACL> aclList, boolean applyToParents)
+    {
+        acling = new ACLing(client.getAclProvider(), aclList, applyToParents);
         return new ACLBackgroundPathAndBytesable<String>()
         {
             @Override
@@ -283,6 +302,12 @@ public class CreateBuilderImpl implements CreateBuilder, CreateBuilder2, Backgro
             }
 
             @Override
+            public BackgroundPathAndBytesable<String> withACL(List<ACL> aclList, boolean applyToParents)
+            {
+                return CreateBuilderImpl.this.withACL(aclList, applyToParents);
+            }
+
+            @Override
             public ErrorListenerPathAndBytesable<String> inBackground()
             {
                 return CreateBuilderImpl.this.inBackground();
@@ -362,7 +387,13 @@ public class CreateBuilderImpl implements CreateBuilder, CreateBuilder2, Backgro
             @Override
             public BackgroundPathAndBytesable<String> withACL(List<ACL> aclList)
             {
-                return CreateBuilderImpl.this.withACL(aclList);
+                return withACL(aclList, false);
+            }
+
+            @Override
+            public BackgroundPathAndBytesable<String> withACL(List<ACL> aclList, boolean applyToParents)
+            {
+                return CreateBuilderImpl.this.withACL(aclList, applyToParents);
             }
 
             @Override
@@ -420,7 +451,8 @@ public class CreateBuilderImpl implements CreateBuilder, CreateBuilder2, Backgro
             }
 
             @Override
-            public ACLBackgroundPathAndBytesable<String> storingStatIn(Stat stat) {
+            public ACLBackgroundPathAndBytesable<String> storingStatIn(Stat stat)
+            {
                 storingStat = stat;
                 return CreateBuilderImpl.this;
             }
@@ -449,6 +481,12 @@ public class CreateBuilderImpl implements CreateBuilder, CreateBuilder2, Backgro
             }
 
             @Override
+            public PathAndBytesable<String> withACL(List<ACL> aclList, boolean applyToParents)
+            {
+                return CreateBuilderImpl.this.withACL(aclList, applyToParents);
+            }
+
+            @Override
             public String forPath(String path, byte[] data) throws Exception
             {
                 return CreateBuilderImpl.this.forPath(path, data);
@@ -613,7 +651,7 @@ public class CreateBuilderImpl implements CreateBuilder, CreateBuilder2, Backgro
 
                         if ( (rc == KeeperException.Code.NONODE.intValue()) && createParentsIfNeeded )
                         {
-                            backgroundCreateParentsThenNode(client, operationAndData, operationAndData.getData().getPath(), backgrounding, createParentsAsContainers);
+                            backgroundCreateParentsThenNode(client, operationAndData, operationAndData.getData().getPath(), backgrounding, acling.getACLProviderForParents(), createParentsAsContainers);
                         }
                         else if ( (rc == KeeperException.Code.NODEEXISTS.intValue()) && setDataIfExists )
                         {
@@ -642,11 +680,18 @@ public class CreateBuilderImpl implements CreateBuilder, CreateBuilder2, Backgro
         return new CreateProtectACLCreateModePathAndBytesable<String>() {
 
             @Override
-            public BackgroundPathAndBytesable<String> withACL(List<ACL> aclList) {
+            public BackgroundPathAndBytesable<String> withACL(List<ACL> aclList)
+            {
                 return CreateBuilderImpl.this.withACL(aclList);
             }
 
             @Override
+            public BackgroundPathAndBytesable<String> withACL(List<ACL> aclList, boolean applyToParents)
+            {
+                return CreateBuilderImpl.this.withACL(aclList, applyToParents);
+            }
+
+            @Override
             public ErrorListenerPathAndBytesable<String> inBackground() {
                 return CreateBuilderImpl.this.inBackground();
             }
@@ -714,7 +759,7 @@ public class CreateBuilderImpl implements CreateBuilder, CreateBuilder2, Backgro
         return PROTECTED_PREFIX + protectedId + "-";
     }
 
-    static <T> void backgroundCreateParentsThenNode(final CuratorFrameworkImpl client, final OperationAndData<T> mainOperationAndData, final String path, Backgrounding backgrounding, final boolean createParentsAsContainers)
+    static <T> void backgroundCreateParentsThenNode(final CuratorFrameworkImpl client, final OperationAndData<T> mainOperationAndData, final String path, Backgrounding backgrounding, final InternalACLProvider aclProvider, final boolean createParentsAsContainers)
     {
         BackgroundOperation<T> operation = new BackgroundOperation<T>()
         {
@@ -723,7 +768,7 @@ public class CreateBuilderImpl implements CreateBuilder, CreateBuilder2, Backgro
             {
                 try
                 {
-                    ZKPaths.mkdirs(client.getZooKeeper(), path, false, client.getAclProvider(), createParentsAsContainers);
+                    ZKPaths.mkdirs(client.getZooKeeper(), path, false, aclProvider, createParentsAsContainers);
                 }
                 catch ( KeeperException e )
                 {
@@ -801,6 +846,13 @@ public class CreateBuilderImpl implements CreateBuilder, CreateBuilder2, Backgro
             }
 
             @Override
+            public PathAndBytesable<String> withACL(List<ACL> aclList, boolean applyToParents)
+            {
+                CreateBuilderImpl.this.withACL(aclList, applyToParents);
+                return this;
+            }
+
+            @Override
             public ACLPathAndBytesable<String> withMode(CreateMode mode)
             {
                 createMode = mode;
@@ -813,6 +865,12 @@ public class CreateBuilderImpl implements CreateBuilder, CreateBuilder2, Backgro
                     }
 
                     @Override
+                    public PathAndBytesable<String> withACL(List<ACL> aclList, boolean applyToParents)
+                    {
+                        return CreateBuilderImpl.this.withACL(aclList, applyToParents);
+                    }
+
+                    @Override
                     public String forPath(String path, byte[] data) throws Exception
                     {
                         return CreateBuilderImpl.this.forPath(path, data);
@@ -850,6 +908,12 @@ public class CreateBuilderImpl implements CreateBuilder, CreateBuilder2, Backgro
             }
 
             @Override
+            public BackgroundPathAndBytesable<String> withACL(List<ACL> aclList, boolean applyToParents)
+            {
+                return CreateBuilderImpl.this.withACL(aclList, applyToParents);
+            }
+
+            @Override
             public ACLBackgroundPathAndBytesable<String> withMode(CreateMode mode) {
                 return CreateBuilderImpl.this.withMode(mode);
             }
@@ -923,6 +987,13 @@ public class CreateBuilderImpl implements CreateBuilder, CreateBuilder2, Backgro
             }
 
             @Override
+            public BackgroundPathAndBytesable<String> withACL(List<ACL> aclList, boolean applyToParents)
+            {
+                CreateBuilderImpl.this.withACL(aclList, applyToParents);
+                return this;
+            }
+
+            @Override
             public ErrorListenerPathAndBytesable<String> inBackground() {
                 return CreateBuilderImpl.this.inBackground();
             }
@@ -1079,7 +1150,7 @@ public class CreateBuilderImpl implements CreateBuilder, CreateBuilder2, Backgro
                             {
                                 if ( createParentsIfNeeded )
                                 {
-                                    ZKPaths.mkdirs(client.getZooKeeper(), path, false, client.getAclProvider(), createParentsAsContainers);
+                                    ZKPaths.mkdirs(client.getZooKeeper(), path, false, acling.getACLProviderForParents(), createParentsAsContainers);
                                     createdPath = client.getZooKeeper().create(path, data, acling.getAclList(path), createMode, storingStat, ttl);
                                 }
                                 else

http://git-wip-us.apache.org/repos/asf/curator/blob/7e611bd1/curator-framework/src/main/java/org/apache/curator/framework/imps/ExistsBuilderImpl.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/ExistsBuilderImpl.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/ExistsBuilderImpl.java
index 9380948..caebd3d 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/ExistsBuilderImpl.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/ExistsBuilderImpl.java
@@ -25,24 +25,25 @@ import org.apache.curator.utils.ZKPaths;
 import org.apache.zookeeper.AsyncCallback;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Stat;
+
+import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.Executor;
 
-public class ExistsBuilderImpl implements ExistsBuilder, BackgroundOperation<String>, ErrorListenerPathable<Stat>
+public class ExistsBuilderImpl implements ExistsBuilder, BackgroundOperation<String>, ErrorListenerPathable<Stat>, ACLableExistBuilderMain
 {
     private final CuratorFrameworkImpl client;
     private Backgrounding backgrounding;
     private Watching watching;
     private boolean createParentsIfNeeded;
     private boolean createParentContainersIfNeeded;
+    private ACLing acling;
 
     ExistsBuilderImpl(CuratorFrameworkImpl client)
     {
-        this.client = client;
-        backgrounding = new Backgrounding();
-        watching = new Watching(client);
-        createParentContainersIfNeeded = false;
+        this(client, new Backgrounding(), null, false, false);
     }
 
     public ExistsBuilderImpl(CuratorFrameworkImpl client, Backgrounding backgrounding, Watcher watcher, boolean createParentsIfNeeded, boolean createParentContainersIfNeeded)
@@ -52,10 +53,11 @@ public class ExistsBuilderImpl implements ExistsBuilder, BackgroundOperation<Str
         this.watching = new Watching(client, watcher);
         this.createParentsIfNeeded = createParentsIfNeeded;
         this.createParentContainersIfNeeded = createParentContainersIfNeeded;
+        this.acling = new ACLing(client.getAclProvider());
     }
 
     @Override
-    public ExistsBuilderMain creatingParentsIfNeeded()
+    public ACLableExistBuilderMain creatingParentsIfNeeded()
     {
         createParentContainersIfNeeded = false;
         createParentsIfNeeded = true;
@@ -63,7 +65,7 @@ public class ExistsBuilderImpl implements ExistsBuilder, BackgroundOperation<Str
     }
 
     @Override
-    public ExistsBuilderMain creatingParentContainersIfNeeded()
+    public ACLableExistBuilderMain creatingParentContainersIfNeeded()
     {
         createParentContainersIfNeeded = true;
         createParentsIfNeeded = false;
@@ -71,6 +73,13 @@ public class ExistsBuilderImpl implements ExistsBuilder, BackgroundOperation<Str
     }
 
     @Override
+    public ExistsBuilderMain withACL(List<ACL> aclList)
+    {
+        acling = new ACLing(client.getAclProvider(), aclList, true);
+        return this;
+    }
+
+    @Override
     public BackgroundPathable<Stat> watched()
     {
         watching = new Watching(client, true);
@@ -185,7 +194,7 @@ public class ExistsBuilderImpl implements ExistsBuilder, BackgroundOperation<Str
             OperationAndData<String> operationAndData = new OperationAndData<String>(this, path, backgrounding.getCallback(), null, backgrounding.getContext(), watching);
             if ( createParentContainersIfNeeded || createParentsIfNeeded )
             {
-                CreateBuilderImpl.backgroundCreateParentsThenNode(client, operationAndData, operationAndData.getData(), backgrounding, createParentContainersIfNeeded);
+                CreateBuilderImpl.backgroundCreateParentsThenNode(client, operationAndData, operationAndData.getData(), backgrounding, acling.getACLProviderForParents(), createParentContainersIfNeeded);
             }
             else
             {
@@ -218,7 +227,7 @@ public class ExistsBuilderImpl implements ExistsBuilder, BackgroundOperation<Str
                         {
                             try
                             {
-                                ZKPaths.mkdirs(client.getZooKeeper(), parent, true, client.getAclProvider(), createParentContainersIfNeeded);
+                                ZKPaths.mkdirs(client.getZooKeeper(), parent, true, acling.getACLProviderForParents(), createParentContainersIfNeeded);
                             }
                             catch ( KeeperException.NodeExistsException e )
                             {

http://git-wip-us.apache.org/repos/asf/curator/blob/7e611bd1/curator-framework/src/main/java/org/apache/curator/framework/imps/SetACLBuilderImpl.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/SetACLBuilderImpl.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/SetACLBuilderImpl.java
index a80be81..0153a45 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/SetACLBuilderImpl.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/SetACLBuilderImpl.java
@@ -59,7 +59,7 @@ public class SetACLBuilderImpl implements SetACLBuilder, BackgroundPathable<Stat
     @Override
     public BackgroundPathable<Stat> withACL(List<ACL> aclList)
     {
-        acling = new ACLing(client.getAclProvider(), aclList);
+        acling = new ACLing(client.getAclProvider(), aclList, false);
         return this;
     }
 

http://git-wip-us.apache.org/repos/asf/curator/blob/7e611bd1/curator-framework/src/test/java/org/apache/curator/framework/imps/TestCreate.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestCreate.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestCreate.java
new file mode 100644
index 0000000..55ef622
--- /dev/null
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestCreate.java
@@ -0,0 +1,249 @@
+/**
+ * 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.imps;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.api.ACLProvider;
+import org.apache.curator.framework.api.BackgroundCallback;
+import org.apache.curator.framework.api.CuratorEvent;
+import org.apache.curator.retry.RetryOneTime;
+import org.apache.curator.test.BaseClassForTests;
+import org.apache.curator.utils.CloseableUtils;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.zookeeper.ZooDefs.Ids.ANYONE_ID_UNSAFE;
+
+public class TestCreate extends BaseClassForTests
+{
+    private static final List<ACL> READ_CREATE = Collections.singletonList(new ACL(ZooDefs.Perms.CREATE | ZooDefs.Perms.READ, ANYONE_ID_UNSAFE));
+    private static final List<ACL> READ_CREATE_WRITE = Collections.singletonList(new ACL(ZooDefs.Perms.CREATE | ZooDefs.Perms.READ | ZooDefs.Perms.WRITE, ANYONE_ID_UNSAFE));
+
+    private static ACLProvider testACLProvider = new ACLProvider()
+    {
+        @Override
+        public List<ACL> getDefaultAcl()
+        {
+            return ZooDefs.Ids.OPEN_ACL_UNSAFE;
+        }
+
+        @Override
+        public List<ACL> getAclForPath(String path)
+        {
+            switch (path)
+            {
+                case "/bar" : return READ_CREATE;
+                case "/bar/foo" : return READ_CREATE_WRITE;
+            }
+            return null;
+        }
+    };
+
+    private CuratorFramework createClient(ACLProvider aclProvider)
+    {
+        return CuratorFrameworkFactory.builder().
+                aclProvider(aclProvider).
+                connectString(server.getConnectString()).
+                retryPolicy(new RetryOneTime(1)).
+                build();
+    }
+
+    /**
+     * Tests that the ACL list provided to the create builder is used for creating the parents.
+     */
+    @Test
+    public void  testCreateWithParentsWithAcl() throws Exception
+    {
+        CuratorFramework client = createClient(new DefaultACLProvider());
+        try
+        {
+            client.start();
+
+            String path = "/bar/foo";
+            List<ACL> acl = Collections.singletonList(new ACL(ZooDefs.Perms.CREATE | ZooDefs.Perms.READ, ANYONE_ID_UNSAFE));
+            client.create().creatingParentsIfNeeded().withACL(acl).forPath(path);
+            List<ACL> actual_bar_foo = client.getACL().forPath(path);
+            Assert.assertEquals(actual_bar_foo, acl);
+            List<ACL> actual_bar = client.getACL().forPath("/bar");
+            Assert.assertEquals(actual_bar, ZooDefs.Ids.OPEN_ACL_UNSAFE);
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+
+    @Test
+    public void  testCreateWithParentsWithAclApplyToParents() throws Exception
+    {
+        CuratorFramework client = createClient(new DefaultACLProvider());
+        try
+        {
+            client.start();
+
+            String path = "/bar/foo";
+            List<ACL> acl = Collections.singletonList(new ACL(ZooDefs.Perms.CREATE | ZooDefs.Perms.READ, ANYONE_ID_UNSAFE));
+            client.create().creatingParentsIfNeeded().withACL(acl, true).forPath(path);
+            List<ACL> actual_bar_foo = client.getACL().forPath(path);
+            Assert.assertEquals(actual_bar_foo, acl);
+            List<ACL> actual_bar = client.getACL().forPath("/bar");
+            Assert.assertEquals(actual_bar, acl);
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+
+    /**
+     * Tests that the ACL list provided to the create builder is used for creating the parents.
+     */
+    @Test
+    public void  testCreateWithParentsWithAclInBackground() throws Exception
+    {
+        CuratorFramework client = createClient(new DefaultACLProvider());
+        try
+        {
+            client.start();
+            final CountDownLatch latch = new CountDownLatch(1);
+            String path = "/bar/foo";
+            List<ACL> acl = Collections.singletonList(new ACL(ZooDefs.Perms.CREATE | ZooDefs.Perms.READ, ANYONE_ID_UNSAFE));
+            BackgroundCallback callback = new BackgroundCallback()
+            {
+                @Override
+                public void processResult(CuratorFramework client, CuratorEvent event) throws Exception
+                {
+                    latch.countDown();
+                }
+            };
+            client.create().creatingParentsIfNeeded().withACL(acl).inBackground(callback).forPath(path);
+            Assert.assertTrue(latch.await(2000, TimeUnit.MILLISECONDS), "Callback not invoked");
+            List<ACL> actual_bar_foo = client.getACL().forPath(path);
+            Assert.assertEquals(actual_bar_foo, acl);
+            List<ACL> actual_bar = client.getACL().forPath("/bar");
+            Assert.assertEquals(actual_bar, ZooDefs.Ids.OPEN_ACL_UNSAFE);
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+
+    @Test
+    public void  testCreateWithParentsWithAclApplyToParentsInBackground() throws Exception
+    {
+        CuratorFramework client = createClient(new DefaultACLProvider());
+        try
+        {
+            client.start();
+            final CountDownLatch latch = new CountDownLatch(1);
+            String path = "/bar/foo";
+            List<ACL> acl = Collections.singletonList(new ACL(ZooDefs.Perms.CREATE | ZooDefs.Perms.READ, ANYONE_ID_UNSAFE));
+            BackgroundCallback callback = new BackgroundCallback()
+            {
+                @Override
+                public void processResult(CuratorFramework client, CuratorEvent event) throws Exception
+                {
+                    latch.countDown();
+                }
+            };
+            client.create().creatingParentsIfNeeded().withACL(acl, true).inBackground(callback).forPath(path);
+            Assert.assertTrue(latch.await(2000, TimeUnit.MILLISECONDS), "Callback not invoked");
+            List<ACL> actual_bar_foo = client.getACL().forPath(path);
+            Assert.assertEquals(actual_bar_foo, acl);
+            List<ACL> actual_bar = client.getACL().forPath("/bar");
+            Assert.assertEquals(actual_bar, acl);
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+
+    /**
+     * Tests that if no ACL list provided to the create builder, then the ACL list is created based on the client's ACLProvider.
+     */
+    @Test
+    public void  testCreateWithParentsWithoutAcl() throws Exception
+    {
+        CuratorFramework client = createClient(testACLProvider);
+        try
+        {
+            client.start();
+
+            String path = "/bar/foo/boo";
+            client.create().creatingParentsIfNeeded().forPath(path);
+            List<ACL> actual_bar_foo_boo = client.getACL().forPath("/bar/foo/boo");
+            Assert.assertEquals(actual_bar_foo_boo, ZooDefs.Ids.OPEN_ACL_UNSAFE);
+            List<ACL> actual_bar_foo = client.getACL().forPath("/bar/foo");
+            Assert.assertEquals(actual_bar_foo, READ_CREATE_WRITE);
+            List<ACL> actual_bar = client.getACL().forPath("/bar");
+            Assert.assertEquals(actual_bar, READ_CREATE);
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+
+    /**
+     * Tests that if no ACL list provided to the create builder, then the ACL list is created based on the client's ACLProvider.
+     */
+    @Test
+    public void  testCreateWithParentsWithoutAclInBackground() throws Exception
+    {
+        CuratorFramework client = createClient(testACLProvider);
+        try
+        {
+            client.start();
+
+            final CountDownLatch latch = new CountDownLatch(1);
+            BackgroundCallback callback = new BackgroundCallback()
+            {
+                @Override
+                public void processResult(CuratorFramework client, CuratorEvent event) throws Exception
+                {
+                    latch.countDown();
+                }
+            };
+
+            final String path = "/bar/foo/boo";
+            client.create().creatingParentsIfNeeded().inBackground(callback).forPath(path);
+            Assert.assertTrue(latch.await(2000, TimeUnit.MILLISECONDS), "Callback not invoked");
+            List<ACL> actual_bar_foo_boo = client.getACL().forPath(path);
+            Assert.assertEquals(actual_bar_foo_boo, ZooDefs.Ids.OPEN_ACL_UNSAFE);
+            List<ACL> actual_bar_foo = client.getACL().forPath("/bar/foo");
+            Assert.assertEquals(actual_bar_foo, READ_CREATE_WRITE);
+            List<ACL> actual_bar = client.getACL().forPath("/bar");
+            Assert.assertEquals(actual_bar, READ_CREATE);
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/7e611bd1/curator-framework/src/test/java/org/apache/curator/framework/imps/TestExistsBuilder.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestExistsBuilder.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestExistsBuilder.java
new file mode 100644
index 0000000..8557e64
--- /dev/null
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestExistsBuilder.java
@@ -0,0 +1,109 @@
+/**
+ * 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.imps;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.api.ACLProvider;
+import org.apache.curator.framework.api.BackgroundCallback;
+import org.apache.curator.framework.api.CuratorEvent;
+import org.apache.curator.retry.RetryOneTime;
+import org.apache.curator.test.BaseClassForTests;
+import org.apache.curator.utils.CloseableUtils;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.zookeeper.ZooDefs.Ids.ANYONE_ID_UNSAFE;
+import static org.testng.Assert.assertNull;
+
+public class TestExistsBuilder extends BaseClassForTests {
+
+    /**
+     * Tests that the ACL list provided to the exists builder is used for creating the parents, when it is applied to
+     * parents.
+     */
+    @Test
+    public void  testExistsWithParentsWithAclApplyToParents() throws Exception
+    {
+        CuratorFramework client = createClient(new DefaultACLProvider());
+        try
+        {
+            client.start();
+
+            String path = "/bar/foo/test";
+            List<ACL> acl = Collections.singletonList(new ACL(ZooDefs.Perms.CREATE | ZooDefs.Perms.READ, ANYONE_ID_UNSAFE));
+            assertNull(client.checkExists().creatingParentsIfNeeded().withACL(acl).forPath(path));
+            List<ACL> actual_bar = client.getACL().forPath("/bar");
+            Assert.assertEquals(actual_bar, acl);
+            List<ACL> actual_bar_foo = client.getACL().forPath("/bar/foo");
+            Assert.assertEquals(actual_bar_foo, acl);
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+
+    @Test
+    public void  testExistsWithParentsWithAclApplyToParentsInBackground() throws Exception
+    {
+        CuratorFramework client = createClient(new DefaultACLProvider());
+        try
+        {
+            client.start();
+            final CountDownLatch latch = new CountDownLatch(1);
+            String path = "/bar/foo/test";
+            List<ACL> acl = Collections.singletonList(new ACL(ZooDefs.Perms.CREATE | ZooDefs.Perms.READ, ANYONE_ID_UNSAFE));
+            BackgroundCallback callback = new BackgroundCallback()
+            {
+                @Override
+                public void processResult(CuratorFramework client, CuratorEvent event) throws Exception
+                {
+                    latch.countDown();
+                }
+            };
+            client.checkExists().creatingParentsIfNeeded().withACL(acl).inBackground(callback).forPath(path);
+            Assert.assertTrue(latch.await(2000, TimeUnit.MILLISECONDS), "Callback not invoked");
+            List<ACL> actual_bar = client.getACL().forPath("/bar");
+            Assert.assertEquals(actual_bar, acl);
+            List<ACL> actual_bar_foo = client.getACL().forPath("/bar/foo");
+            Assert.assertEquals(actual_bar_foo, acl);
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+
+    private CuratorFramework createClient(ACLProvider aclProvider)
+    {
+        return CuratorFrameworkFactory.builder().
+                aclProvider(aclProvider).
+                connectString(server.getConnectString()).
+                retryPolicy(new RetryOneTime(1)).
+                build();
+    }
+}