You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@curator.apache.org by ca...@apache.org on 2015/08/23 01:33:38 UTC

curator git commit: CURATOR-214 - Implemented new ZK API to allow Stat objects to be returned as part of the creation of a new ZNode.

Repository: curator
Updated Branches:
  refs/heads/CURATOR-214 [created] af369dec7


CURATOR-214 - Implemented new ZK API to allow Stat objects to be returned as part of the creation of a new ZNode.


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

Branch: refs/heads/CURATOR-214
Commit: af369dec7a3ba20c60c68b502198bd851e34906f
Parents: 7d97259
Author: Cam McKenzie <ca...@apache.org>
Authored: Sun Aug 23 09:28:49 2015 +1000
Committer: Cam McKenzie <ca...@apache.org>
Committed: Sun Aug 23 09:28:49 2015 +1000

----------------------------------------------------------------------
 ...reateModeStatBackgroundPathAndBytesable.java |  25 ++
 .../api/CreateBackgroundModeStatACLable.java    |  70 +++++
 .../curator/framework/api/CreateBuilder.java    |   9 +-
 ...ateProtectACLCreateModePathAndBytesable.java |  72 +++++
 .../framework/imps/CreateBuilderImpl.java       | 299 ++++++++++++++++++-
 .../framework/imps/TestCreateReturningStat.java | 199 ++++++++++++
 .../framework/imps/TestFrameworkEdges.java      |   4 +-
 7 files changed, 658 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/af369dec/curator-framework/src/main/java/org/apache/curator/framework/api/ACLCreateModeStatBackgroundPathAndBytesable.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLCreateModeStatBackgroundPathAndBytesable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/ACLCreateModeStatBackgroundPathAndBytesable.java
new file mode 100755
index 0000000..0becfb9
--- /dev/null
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/ACLCreateModeStatBackgroundPathAndBytesable.java
@@ -0,0 +1,25 @@
+/**
+ * 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 ACLCreateModeStatBackgroundPathAndBytesable<T> extends
+    ACLCreateModeBackgroundPathAndBytesable<T>,
+    Statable<ACLCreateModeBackgroundPathAndBytesable<T>>
+{
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/af369dec/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBackgroundModeStatACLable.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBackgroundModeStatACLable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBackgroundModeStatACLable.java
new file mode 100755
index 0000000..2ebe63e
--- /dev/null
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBackgroundModeStatACLable.java
@@ -0,0 +1,70 @@
+/**
+ * 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.CreateMode;
+
+public interface CreateBackgroundModeStatACLable extends
+    BackgroundPathAndBytesable<String>,
+    CreateModable<ACLBackgroundPathAndBytesable<String>>,
+    ACLCreateModeBackgroundPathAndBytesable<String>,
+    Statable<CreateBackgroundModeACLable>
+{
+    /**
+     * Causes any parent nodes to get created if they haven't already been
+     *
+     * @return this
+     */
+    public ACLCreateModePathAndBytesable<String>    creatingParentsIfNeeded();
+
+    /**
+     * Causes any parent nodes to get created using {@link CreateMode#CONTAINER} if they haven't already been.
+     * IMPORTANT NOTE: container creation is a new feature in recent versions of ZooKeeper.
+     * If the ZooKeeper version you're using does not support containers, the parent nodes
+     * are created as ordinary PERSISTENT nodes.
+     *
+     * @return this
+     */
+    public ACLCreateModePathAndBytesable<String> creatingParentContainersIfNeeded();
+
+    /**
+     * <p>
+     *     Hat-tip to https://github.com/sbridges for pointing this out
+     * </p>
+     *
+     * <p>
+     *     It turns out there is an edge case that exists when creating sequential-ephemeral
+     *     nodes. The creation can succeed on the server, but the server can crash before
+     *     the created node name is returned to the client. However, the ZK session is still
+     *     valid so the ephemeral node is not deleted. Thus, there is no way for the client to
+     *     determine what node was created for them.
+     * </p>
+     *
+     * <p>
+     *     Putting the create builder into protected-ephemeral-sequential mode works around this.
+     *     The name of the node that is created is prefixed with a GUID. If node creation fails
+     *     the normal retry mechanism will occur. On the retry, the parent path is first searched
+     *     for a node that has the GUID in it. If that node is found, it is assumed to be the lost
+     *     node that was successfully created on the first try and is returned to the caller.
+     * </p>
+     *
+     * @return this
+     */
+    public ACLPathAndBytesable<String>              withProtectedEphemeralSequential();
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/af369dec/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilder.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilder.java b/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilder.java
index 0db2094..fa1a68e 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilder.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilder.java
@@ -24,14 +24,15 @@ public interface CreateBuilder extends
     BackgroundPathAndBytesable<String>,
     CreateModable<ACLBackgroundPathAndBytesable<String>>,
     ACLCreateModeBackgroundPathAndBytesable<String>,
-    Compressible<CreateBackgroundModeACLable>
+    Compressible<CreateBackgroundModeStatACLable>,
+    Statable<CreateProtectACLCreateModePathAndBytesable<String>>
 {
     /**
      * Causes any parent nodes to get created if they haven't already been
      *
      * @return this
      */
-    public ProtectACLCreateModePathAndBytesable<String> creatingParentsIfNeeded();
+    public ProtectACLCreateModeStatPathAndBytesable<String> creatingParentsIfNeeded();
 
     /**
      * Causes any parent nodes to get created using {@link CreateMode#CONTAINER} if they haven't already been.
@@ -41,7 +42,7 @@ public interface CreateBuilder extends
      *
      * @return this
      */
-    public ProtectACLCreateModePathAndBytesable<String> creatingParentContainersIfNeeded();
+    public ProtectACLCreateModeStatPathAndBytesable<String> creatingParentContainersIfNeeded();
 
     /**
      * @deprecated this has been generalized to support all create modes. Instead, use:
@@ -81,5 +82,5 @@ public interface CreateBuilder extends
      *
      * @return this
      */
-    public ACLCreateModeBackgroundPathAndBytesable<String>    withProtection();
+    public ACLCreateModeStatBackgroundPathAndBytesable<String>    withProtection();
 }

http://git-wip-us.apache.org/repos/asf/curator/blob/af369dec/curator-framework/src/main/java/org/apache/curator/framework/api/CreateProtectACLCreateModePathAndBytesable.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/CreateProtectACLCreateModePathAndBytesable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/CreateProtectACLCreateModePathAndBytesable.java
new file mode 100755
index 0000000..9e0c840
--- /dev/null
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/CreateProtectACLCreateModePathAndBytesable.java
@@ -0,0 +1,72 @@
+/**
+ * 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.CreateMode;
+
+public interface CreateProtectACLCreateModePathAndBytesable<T> extends
+    ProtectACLCreateModePathAndBytesable<T>
+{
+    /**
+     * Causes any parent nodes to get created if they haven't already been
+     *
+     * @return this
+     */
+    public ProtectACLCreateModePathAndBytesable<String> creatingParentsIfNeeded();
+
+    /**
+     * Causes any parent nodes to get created using {@link CreateMode#CONTAINER} if they haven't already been.
+     * IMPORTANT NOTE: container creation is a new feature in recent versions of ZooKeeper.
+     * If the ZooKeeper version you're using does not support containers, the parent nodes
+     * are created as ordinary PERSISTENT nodes.
+     *
+     * @return this
+     */
+    public ProtectACLCreateModePathAndBytesable<String> creatingParentContainersIfNeeded();
+
+    /**
+     * <p>
+     *     Hat-tip to https://github.com/sbridges for pointing this out
+     * </p>
+     *
+     * <p>
+     *     It turns out there is an edge case that exists when creating sequential-ephemeral
+     *     nodes. The creation can succeed on the server, but the server can crash before
+     *     the created node name is returned to the client. However, the ZK session is still
+     *     valid so the ephemeral node is not deleted. Thus, there is no way for the client to
+     *     determine what node was created for them.
+     * </p>
+     *
+     * <p>
+     *     Even without sequential-ephemeral, however, the create can succeed on the sever
+     *     but the client (for various reasons) will not know it.
+     * </p>
+     *
+     * <p>
+     *     Putting the create builder into protection mode works around this.
+     *     The name of the node that is created is prefixed with a GUID. If node creation fails
+     *     the normal retry mechanism will occur. On the retry, the parent path is first searched
+     *     for a node that has the GUID in it. If that node is found, it is assumed to be the lost
+     *     node that was successfully created on the first try and is returned to the caller.
+     * </p>
+     *
+     * @return this
+     */
+    public ACLCreateModeBackgroundPathAndBytesable<String>    withProtection();
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/af369dec/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 7184c39..f75816b 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
@@ -34,6 +34,8 @@ import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.Op;
 import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Stat;
+
 import java.util.List;
 import java.util.UUID;
 import java.util.concurrent.Callable;
@@ -51,6 +53,7 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
     private boolean compress;
     private String protectedId;
     private ACLing acling;
+    private Stat storingStat;
 
     @VisibleForTesting
     boolean failNextCreateForTesting = false;
@@ -69,6 +72,7 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
         compress = false;
         doProtected = false;
         protectedId = null;
+        storingStat = null;
     }
 
     <T> TransactionCreateBuilder<T> asTransactionCreateBuilder(final T context, final CuratorMultiTransactionRecord transaction)
@@ -118,12 +122,18 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
     }
 
     @Override
-    public CreateBackgroundModeACLable compressed()
+    public CreateBackgroundModeStatACLable compressed()
     {
         compress = true;
-        return new CreateBackgroundModeACLable()
+        return new CreateBackgroundModeStatACLable()
         {
             @Override
+            public CreateBackgroundModeACLable storingStatIn(Stat stat) {
+                storingStat = stat;
+                return asCreateBackgroundModeACLable();
+            }
+            
+            @Override
             public ACLCreateModePathAndBytesable<String> creatingParentsIfNeeded()
             {
                 createParentsIfNeeded = true;
@@ -268,7 +278,7 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
     }
 
     @Override
-    public ProtectACLCreateModePathAndBytesable<String> creatingParentContainersIfNeeded()
+    public ProtectACLCreateModeStatPathAndBytesable<String> creatingParentContainersIfNeeded()
     {
         setCreateParentsAsContainers();
         return creatingParentsIfNeeded();
@@ -283,10 +293,10 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
     }
 
     @Override
-    public ProtectACLCreateModePathAndBytesable<String> creatingParentsIfNeeded()
+    public ProtectACLCreateModeStatPathAndBytesable<String> creatingParentsIfNeeded()
     {
         createParentsIfNeeded = true;
-        return new ProtectACLCreateModePathAndBytesable<String>()
+        return new ProtectACLCreateModeStatPathAndBytesable<String>()
         {
             @Override
             public ACLCreateModeBackgroundPathAndBytesable<String> withProtection()
@@ -353,14 +363,20 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
             {
                 return CreateBuilderImpl.this.forPath(path);
             }
+
+            @Override
+            public ACLBackgroundPathAndBytesable<String> storingStatIn(Stat stat) {
+                storingStat = stat;
+                return CreateBuilderImpl.this;
+            }
         };
     }
 
     @Override
-    public ACLCreateModeBackgroundPathAndBytesable<String> withProtection()
+    public ACLCreateModeStatBackgroundPathAndBytesable<String> withProtection()
     {
         setProtected();
-        return this;
+        return asACLCreateModeStatBackgroundPathAndBytesable();
     }
 
     @Override
@@ -501,7 +517,10 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
     public void performBackgroundOperation(final OperationAndData<PathAndBytes> operationAndData) throws Exception
     {
         final TimeTrace trace = client.getZookeeperClient().startTracer("CreateBuilderImpl-Background");
-        client.getZooKeeper().create
+        
+        if(storingStat == null)
+        {
+            client.getZooKeeper().create
             (
                 operationAndData.getData().getPath(),
                 operationAndData.getData().getData(),
@@ -520,12 +539,127 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
                         }
                         else
                         {
-                            sendBackgroundResponse(rc, path, ctx, name, operationAndData);
+                            sendBackgroundResponse(rc, path, ctx, name, null, operationAndData);
                         }
                     }
                 },
                 backgrounding.getContext()
             );
+        }
+        else
+        {
+            client.getZooKeeper().create
+            (
+                operationAndData.getData().getPath(),
+                operationAndData.getData().getData(),
+                acling.getAclList(operationAndData.getData().getPath()),
+                createMode,
+                new AsyncCallback.Create2Callback() {
+                    
+                    @Override
+                    public void processResult(int rc, String path, Object ctx, String name, Stat stat) {
+                        trace.commit();
+                        
+                        //Need to manually populate the provided stat here.
+                        storingStat.setAversion(stat.getAversion());
+                        storingStat.setCtime(stat.getCtime());
+                        storingStat.setCversion(stat.getCversion());
+                        storingStat.setCzxid(stat.getCzxid());
+                        storingStat.setDataLength(stat.getDataLength());
+                        storingStat.setEphemeralOwner(stat.getEphemeralOwner());
+                        storingStat.setMtime(stat.getMtime());
+                        storingStat.setMzxid(stat.getMzxid());
+                        storingStat.setNumChildren(stat.getNumChildren());
+                        storingStat.setPzxid(stat.getPzxid());
+                        storingStat.setVersion(stat.getVersion());
+
+                        if ( (rc == KeeperException.Code.NONODE.intValue()) && createParentsIfNeeded )
+                        {
+                            backgroundCreateParentsThenNode(client, operationAndData, operationAndData.getData().getPath(), backgrounding, createParentsAsContainers);
+                        }
+                        else
+                        {
+                            sendBackgroundResponse(rc, path, ctx, name, stat, operationAndData);
+                        }
+                    }
+                },
+                backgrounding.getContext()
+            );
+        }
+    }
+    
+    @Override
+    public CreateProtectACLCreateModePathAndBytesable<String> storingStatIn(Stat stat) {
+        storingStat = stat;
+        
+        return new CreateProtectACLCreateModePathAndBytesable<String>() {
+
+            @Override
+            public BackgroundPathAndBytesable<String> withACL(List<ACL> aclList) {
+                return CreateBuilderImpl.this.withACL(aclList);
+            }
+
+            @Override
+            public PathAndBytesable<String> inBackground() {
+                return CreateBuilderImpl.this.inBackground();
+            }
+
+            @Override
+            public PathAndBytesable<String> inBackground(Object context) {
+                return CreateBuilderImpl.this.inBackground(context);
+            }
+
+            @Override
+            public PathAndBytesable<String> inBackground(BackgroundCallback callback) {
+                return CreateBuilderImpl.this.inBackground(callback);
+            }
+
+            @Override
+            public PathAndBytesable<String> inBackground(BackgroundCallback callback, Object context) {
+                return CreateBuilderImpl.this.inBackground(callback, context);
+            }
+
+            @Override
+            public PathAndBytesable<String> inBackground(BackgroundCallback callback, Executor executor) {
+                return CreateBuilderImpl.this.inBackground(callback, executor);
+            }
+
+            @Override
+            public PathAndBytesable<String> inBackground(BackgroundCallback callback, Object context,
+                    Executor executor) {
+                return CreateBuilderImpl.this.inBackground(callback, context, executor);
+            }
+
+            @Override
+            public String forPath(String path, byte[] data) throws Exception {
+                return CreateBuilderImpl.this.forPath(path, data);
+            }
+
+            @Override
+            public String forPath(String path) throws Exception {
+                return CreateBuilderImpl.this.forPath(path);
+            }
+
+            @Override
+            public ACLBackgroundPathAndBytesable<String> withMode(CreateMode mode) {
+                return CreateBuilderImpl.this.withMode(mode);
+            }
+
+            @Override
+            public ACLCreateModeBackgroundPathAndBytesable<String> withProtection() {
+                return CreateBuilderImpl.this.withProtection();
+            }
+
+            @Override
+            public ProtectACLCreateModePathAndBytesable<String> creatingParentsIfNeeded() {
+                return CreateBuilderImpl.this.creatingParentsIfNeeded();
+            }
+
+            @Override
+            public ProtectACLCreateModePathAndBytesable<String> creatingParentContainersIfNeeded() {
+                return CreateBuilderImpl.this.creatingParentContainersIfNeeded();
+            }  
+        };
     }
 
     private static String getProtectedPrefix(String protectedId)
@@ -555,12 +689,12 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
         client.queueOperation(parentOperation);
     }
 
-    private void sendBackgroundResponse(int rc, String path, Object ctx, String name, OperationAndData<PathAndBytes> operationAndData)
+    private void sendBackgroundResponse(int rc, String path, Object ctx, String name, Stat stat, OperationAndData<PathAndBytes> operationAndData)
     {
         path = client.unfixForNamespace(path);
         name = client.unfixForNamespace(name);
 
-        CuratorEvent event = new CuratorEventImpl(client, CuratorEventType.CREATE, rc, path, name, ctx, null, null, null, null, null, null);
+        CuratorEvent event = new CuratorEventImpl(client, CuratorEventType.CREATE, rc, path, name, ctx, stat, null, null, null, null, null);
         client.processBackgroundOperation(operationAndData, event);
     }
 
@@ -619,6 +753,141 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
             }
         };
     }
+    
+    private CreateBackgroundModeACLable asCreateBackgroundModeACLable()
+    {
+        return new CreateBackgroundModeACLable() {
+            
+            @Override
+            public BackgroundPathAndBytesable<String> withACL(List<ACL> aclList) {
+                return CreateBuilderImpl.this.withACL(aclList);
+            }
+            
+            @Override
+            public ACLBackgroundPathAndBytesable<String> withMode(CreateMode mode) {
+                return CreateBuilderImpl.this.withMode(mode);
+            }
+            
+            @Override
+            public String forPath(String path) throws Exception {
+                return CreateBuilderImpl.this.forPath(path);
+            }
+            
+            @Override
+            public String forPath(String path, byte[] data) throws Exception {
+                return CreateBuilderImpl.this.forPath(path, data);
+            }
+            
+            @Override
+            public PathAndBytesable<String> inBackground(BackgroundCallback callback, Object context, Executor executor) {
+                return CreateBuilderImpl.this.inBackground(callback, context, executor);
+            }
+            
+            @Override
+            public PathAndBytesable<String> inBackground(BackgroundCallback callback, Executor executor) {
+                return CreateBuilderImpl.this.inBackground(callback, executor);
+            }
+            
+            @Override
+            public PathAndBytesable<String> inBackground(BackgroundCallback callback, Object context) {
+                return CreateBuilderImpl.this.inBackground(callback, context);
+            }
+            
+            @Override
+            public PathAndBytesable<String> inBackground(BackgroundCallback callback) {
+                return CreateBuilderImpl.this.inBackground(callback);
+            }
+            
+            @Override
+            public PathAndBytesable<String> inBackground(Object context) {
+                return CreateBuilderImpl.this.inBackground(context);
+            }
+            
+            @Override
+            public PathAndBytesable<String> inBackground() {
+                return CreateBuilderImpl.this.inBackground();
+            }
+            
+            @Override
+            public ACLPathAndBytesable<String> withProtectedEphemeralSequential() {
+                return CreateBuilderImpl.this.withProtectedEphemeralSequential();
+            }
+            
+            @Override
+            public ACLCreateModePathAndBytesable<String> creatingParentsIfNeeded() {
+                createParentsIfNeeded = true;
+                return asACLCreateModePathAndBytesable();
+            }
+            
+            @Override
+            public ACLCreateModePathAndBytesable<String> creatingParentContainersIfNeeded() {
+                setCreateParentsAsContainers();
+                return asACLCreateModePathAndBytesable();
+            }
+        };
+    }
+    
+    private ACLCreateModeStatBackgroundPathAndBytesable<String> asACLCreateModeStatBackgroundPathAndBytesable()
+    {
+        return new ACLCreateModeStatBackgroundPathAndBytesable<String>()
+        {
+            @Override
+            public BackgroundPathAndBytesable<String> withACL(List<ACL> aclList) {
+                return CreateBuilderImpl.this.withACL(aclList);
+            }
+
+            @Override
+            public PathAndBytesable<String> inBackground() {
+                return CreateBuilderImpl.this.inBackground();
+            }
+
+            @Override
+            public PathAndBytesable<String> inBackground(BackgroundCallback callback, Object context, Executor executor) {
+                return CreateBuilderImpl.this.inBackground(callback, context, executor);
+            }
+            
+            @Override
+            public PathAndBytesable<String> inBackground(BackgroundCallback callback, Executor executor) {
+                return CreateBuilderImpl.this.inBackground(callback, executor);
+            }
+            
+            @Override
+            public PathAndBytesable<String> inBackground(BackgroundCallback callback, Object context) {
+                return CreateBuilderImpl.this.inBackground(callback, context);
+            }
+            
+            @Override
+            public PathAndBytesable<String> inBackground(BackgroundCallback callback) {
+                return CreateBuilderImpl.this.inBackground(callback);
+            }
+            
+            @Override
+            public PathAndBytesable<String> inBackground(Object context) {
+                return CreateBuilderImpl.this.inBackground(context);
+            }
+
+            @Override
+            public String forPath(String path) throws Exception {
+                return CreateBuilderImpl.this.forPath(path);
+            }
+            
+            @Override
+            public String forPath(String path, byte[] data) throws Exception {
+                return CreateBuilderImpl.this.forPath(path, data);
+            }
+
+            @Override
+            public ACLBackgroundPathAndBytesable<String> withMode(CreateMode mode) {
+                return CreateBuilderImpl.this.withMode(mode);
+            }
+
+            @Override
+            public ACLCreateModeBackgroundPathAndBytesable<String> storingStatIn(Stat stat) {
+                storingStat = stat;
+                return CreateBuilderImpl.this;
+            }            
+        };
+    }
 
     @VisibleForTesting
     volatile boolean debugForceFindProtectedNode = false;
@@ -657,14 +926,14 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
                     }
                     catch ( KeeperException.ConnectionLossException e )
                     {
-                        sendBackgroundResponse(KeeperException.Code.CONNECTIONLOSS.intValue(), path, backgrounding.getContext(), null, this);
+                        sendBackgroundResponse(KeeperException.Code.CONNECTIONLOSS.intValue(), path, backgrounding.getContext(), null, null, this);
                         callSuper = false;
                     }
                     if ( createdPath != null )
                     {
                         try
                         {
-                            sendBackgroundResponse(KeeperException.Code.OK.intValue(), createdPath, backgrounding.getContext(), createdPath, this);
+                            sendBackgroundResponse(KeeperException.Code.OK.intValue(), createdPath, backgrounding.getContext(), createdPath, null, this);
                         }
                         catch ( Exception e )
                         {
@@ -716,14 +985,14 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
                         {
                             try
                             {
-                                createdPath = client.getZooKeeper().create(path, data, acling.getAclList(path), createMode);
+                                createdPath = client.getZooKeeper().create(path, data, acling.getAclList(path), createMode, storingStat);
                             }
                             catch ( KeeperException.NoNodeException e )
                             {
                                 if ( createParentsIfNeeded )
                                 {
                                     ZKPaths.mkdirs(client.getZooKeeper(), path, false, client.getAclProvider(), createParentsAsContainers);
-                                    createdPath = client.getZooKeeper().create(path, data, acling.getAclList(path), createMode);
+                                    createdPath = client.getZooKeeper().create(path, data, acling.getAclList(path), createMode, storingStat);
                                 }
                                 else
                                 {

http://git-wip-us.apache.org/repos/asf/curator/blob/af369dec/curator-framework/src/test/java/org/apache/curator/framework/imps/TestCreateReturningStat.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestCreateReturningStat.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestCreateReturningStat.java
new file mode 100755
index 0000000..4e9e78c
--- /dev/null
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestCreateReturningStat.java
@@ -0,0 +1,199 @@
+/**
+ * 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 java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.api.BackgroundCallback;
+import org.apache.curator.framework.api.CuratorEvent;
+import org.apache.curator.framework.api.CuratorEventType;
+import org.apache.curator.framework.api.ExistsBuilder;
+import org.apache.curator.framework.api.PathAndBytesable;
+import org.apache.curator.retry.RetryOneTime;
+import org.apache.curator.test.BaseClassForTests;
+import org.apache.curator.test.Timing;
+import org.apache.curator.utils.CloseableUtils;
+import org.apache.zookeeper.data.Stat;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+public class TestCreateReturningStat extends BaseClassForTests
+{
+    private CuratorFramework createClient()
+    {
+        return CuratorFrameworkFactory.builder().
+                connectString(server.getConnectString()).
+                retryPolicy(new RetryOneTime(1)).
+                build();
+    }
+    
+    private void compare(CuratorFramework client, String path,
+                         Stat expected) throws Exception
+    {
+        Stat queriedStat = client.checkExists().forPath(path);
+        
+        Assert.assertEquals(queriedStat, expected);
+    }
+    
+    @Test
+    public void testCreateReturningStat() throws Exception
+    {
+        CuratorFramework client = createClient();
+        try
+        {
+            client.start();
+            
+            String path = "/bla";
+            Stat stat = new Stat();
+            client.create().storingStatIn(stat).forPath(path);
+            
+            compare(client, path, stat);
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+    
+    @Test
+    public void testCreateReturningStatIncludingParents() throws Exception
+    {
+        CuratorFramework client = createClient();
+        try
+        {
+            client.start();
+            
+            String path = "/bla/bla";
+            Stat stat = new Stat();
+            client.create().creatingParentsIfNeeded().storingStatIn(stat).forPath(path);
+            
+            compare(client, path, stat);
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+    
+    @Test
+    public void testCreateReturningStatIncludingParentsReverse() throws Exception
+    {
+        CuratorFramework client = createClient();
+        try
+        {
+            client.start();
+            
+            String path = "/bla/bla";
+            Stat stat = new Stat();
+            client.create().storingStatIn(stat).creatingParentsIfNeeded().forPath(path);
+            
+            compare(client, path, stat);
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+    
+    @Test
+    public void testCreateReturningStatCompressed() throws Exception
+    {
+        CuratorFramework client = createClient();
+        try
+        {
+            client.start();
+            
+            String path = "/bla";
+            Stat stat = new Stat();
+            client.create().compressed().storingStatIn(stat).forPath(path);
+            
+            compare(client, path, stat);
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+    
+    @Test
+    public void testCreateReturningStatWithProtected() throws Exception
+    {
+        CuratorFramework client = createClient();
+        try
+        {
+            client.start();
+            
+            String path = "/bla";
+            Stat stat = new Stat();
+            path = client.create().withProtection().storingStatIn(stat).forPath(path);
+            
+            compare(client, path, stat);
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+    
+    @Test
+    public void testCreateReturningStatInBackground() throws Exception
+    {
+        Timing timing = new Timing();
+        CuratorFramework client = createClient();
+        try
+        {
+            client.start();
+            
+            String path = "/bla";
+            Stat stat = new Stat();
+
+            final CountDownLatch latch = new CountDownLatch(1);
+            final AtomicReference<Stat> statRef = new AtomicReference<>();
+            BackgroundCallback callback = new BackgroundCallback() {
+                
+                @Override
+                public void processResult(CuratorFramework client, CuratorEvent event) throws Exception {
+                    if(event.getType() == CuratorEventType.CREATE)
+                    {
+                        statRef.set(event.getStat());
+                    
+                        latch.countDown();
+                    }
+                }
+            };
+            
+            client.create().storingStatIn(stat).inBackground(callback).forPath(path);
+            
+            if(!timing.awaitLatch(latch))
+            {
+                Assert.fail("Timed out awaing latch");
+            }
+            
+            compare(client, path, statRef.get());
+            compare(client, path, stat);
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/af369dec/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkEdges.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkEdges.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkEdges.java
index cd3ae77..bb0fbcb 100644
--- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkEdges.java
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkEdges.java
@@ -93,7 +93,9 @@ public class TestFrameworkEdges extends BaseClassForTests
 
             client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), 1, new RetryOneTime(1));
             client.start();
-            CreateBuilderImpl createBuilder = (CreateBuilderImpl)client.create().withProtection();
+            
+            CreateBuilderImpl createBuilder = (CreateBuilderImpl)client.create();
+            createBuilder.withProtection();
 
             client.create().forPath(createBuilder.adjustPath(TEST_PATH));